mirror of https://github.com/apache/lucene.git
LUCENE-2647: refactor reusable components out of standard codec
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@998627 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
93f8a01df7
commit
7fe5f4bef7
|
@ -26,14 +26,14 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardCodec;
|
||||
import org.apache.lucene.index.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReaderImpl;
|
||||
import org.apache.lucene.index.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsWriterImpl;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsDictReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
|
||||
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
|
@ -58,7 +58,7 @@ public class AppendingCodec extends Codec {
|
|||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state)
|
||||
throws IOException {
|
||||
StandardPostingsWriter docsWriter = new StandardPostingsWriterImpl(state);
|
||||
PostingsWriterBase docsWriter = new StandardPostingsWriter(state);
|
||||
boolean success = false;
|
||||
AppendingTermsIndexWriter indexWriter = null;
|
||||
try {
|
||||
|
@ -88,8 +88,8 @@ public class AppendingCodec extends Codec {
|
|||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state)
|
||||
throws IOException {
|
||||
StandardPostingsReader docsReader = new StandardPostingsReaderImpl(state.dir, state.segmentInfo, state.readBufferSize);
|
||||
StandardTermsIndexReader indexReader;
|
||||
PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize);
|
||||
TermsIndexReaderBase indexReader;
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
|
@ -128,9 +128,9 @@ public class AppendingCodec extends Codec {
|
|||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, Set<String> files)
|
||||
throws IOException {
|
||||
StandardPostingsReaderImpl.files(dir, segmentInfo, files);
|
||||
StandardTermsDictReader.files(dir, segmentInfo, files);
|
||||
SimpleStandardTermsIndexReader.files(dir, segmentInfo, files);
|
||||
StandardPostingsReader.files(dir, segmentInfo, files);
|
||||
PrefixCodedTermsReader.files(dir, segmentInfo, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,20 +21,20 @@ import java.io.IOException;
|
|||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsDictReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsDictWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsIndexReader;
|
||||
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.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 StandardTermsDictReader {
|
||||
public class AppendingTermsDictReader extends PrefixCodedTermsReader {
|
||||
|
||||
public AppendingTermsDictReader(StandardTermsIndexReader indexReader,
|
||||
public AppendingTermsDictReader(TermsIndexReaderBase indexReader,
|
||||
Directory dir, FieldInfos fieldInfos, String segment,
|
||||
StandardPostingsReader postingsReader, int readBufferSize,
|
||||
PostingsReaderBase postingsReader, int readBufferSize,
|
||||
Comparator<BytesRef> termComp, int termsCacheSize) throws IOException {
|
||||
super(indexReader, dir, fieldInfos, segment, postingsReader, readBufferSize,
|
||||
termComp, termsCacheSize);
|
||||
|
@ -43,7 +43,7 @@ public class AppendingTermsDictReader extends StandardTermsDictReader {
|
|||
@Override
|
||||
protected void readHeader(IndexInput in) throws IOException {
|
||||
CodecUtil.checkHeader(in, AppendingTermsDictWriter.CODEC_NAME,
|
||||
StandardTermsDictWriter.VERSION_START, StandardTermsDictWriter.VERSION_CURRENT);
|
||||
PrefixCodedTermsWriter.VERSION_START, PrefixCodedTermsWriter.VERSION_CURRENT);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,18 +21,18 @@ import java.io.IOException;
|
|||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsDictWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsIndexWriter;
|
||||
import org.apache.lucene.index.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
|
||||
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 StandardTermsDictWriter {
|
||||
public class AppendingTermsDictWriter extends PrefixCodedTermsWriter {
|
||||
final static String CODEC_NAME = "APPENDING_TERMS_DICT";
|
||||
|
||||
public AppendingTermsDictWriter(StandardTermsIndexWriter indexWriter,
|
||||
SegmentWriteState state, StandardPostingsWriter postingsWriter,
|
||||
public AppendingTermsDictWriter(TermsIndexWriterBase indexWriter,
|
||||
SegmentWriteState state, PostingsWriterBase postingsWriter,
|
||||
Comparator<BytesRef> termComp) throws IOException {
|
||||
super(indexWriter, state, postingsWriter, termComp);
|
||||
}
|
||||
|
|
|
@ -21,13 +21,13 @@ import java.io.IOException;
|
|||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
|
||||
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 AppendingTermsIndexReader extends SimpleStandardTermsIndexReader {
|
||||
public class AppendingTermsIndexReader extends FixedGapTermsIndexReader {
|
||||
|
||||
public AppendingTermsIndexReader(Directory dir, FieldInfos fieldInfos,
|
||||
String segment, int indexDivisor, Comparator<BytesRef> termComp)
|
||||
|
|
|
@ -20,11 +20,11 @@ package org.apache.lucene.index.codecs.appending;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexWriter;
|
||||
import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
|
||||
public class AppendingTermsIndexWriter extends SimpleStandardTermsIndexWriter {
|
||||
public class AppendingTermsIndexWriter extends FixedGapTermsIndexWriter {
|
||||
final static String CODEC_NAME = "APPENDING_TERMS_INDEX";
|
||||
final static int VERSION_START = 0;
|
||||
final static int VERSION_CURRENT = VERSION_START;
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -59,7 +59,7 @@ import java.io.IOException;
|
|||
import org.apache.lucene.index.IndexFileNames;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public class SimpleStandardTermsIndexReader extends StandardTermsIndexReader {
|
||||
public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
||||
|
||||
// NOTE: long is overkill here, since this number is 128
|
||||
// by default and only indexDivisor * 128 if you change
|
||||
|
@ -90,12 +90,12 @@ public class SimpleStandardTermsIndexReader extends StandardTermsIndexReader {
|
|||
// start of the field info data
|
||||
protected long dirOffset;
|
||||
|
||||
public SimpleStandardTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp)
|
||||
public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp)
|
||||
throws IOException {
|
||||
|
||||
this.termComp = termComp;
|
||||
|
||||
IndexInput in = dir.openInput(IndexFileNames.segmentFileName(segment, "", StandardCodec.TERMS_INDEX_EXTENSION));
|
||||
IndexInput in = dir.openInput(IndexFileNames.segmentFileName(segment, "", FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
|
||||
boolean success = false;
|
||||
|
||||
|
@ -147,8 +147,8 @@ public class SimpleStandardTermsIndexReader extends StandardTermsIndexReader {
|
|||
}
|
||||
|
||||
protected void readHeader(IndexInput input) throws IOException {
|
||||
CodecUtil.checkHeader(input, SimpleStandardTermsIndexWriter.CODEC_NAME,
|
||||
SimpleStandardTermsIndexWriter.VERSION_START, SimpleStandardTermsIndexWriter.VERSION_START);
|
||||
CodecUtil.checkHeader(input, FixedGapTermsIndexWriter.CODEC_NAME,
|
||||
FixedGapTermsIndexWriter.VERSION_START, FixedGapTermsIndexWriter.VERSION_START);
|
||||
dirOffset = input.readLong();
|
||||
}
|
||||
|
||||
|
@ -179,7 +179,7 @@ public class SimpleStandardTermsIndexReader extends StandardTermsIndexReader {
|
|||
this.numIndexTerms = numIndexTerms;
|
||||
|
||||
// We still create the indexReader when indexDivisor
|
||||
// is -1, so that StandardTermsDictReader can call
|
||||
// is -1, so that PrefixCodedTermsReader can call
|
||||
// isIndexTerm for each field:
|
||||
if (indexDivisor > 0) {
|
||||
coreIndex = new CoreFieldIndex(indexStart,
|
||||
|
@ -437,11 +437,11 @@ public class SimpleStandardTermsIndexReader extends StandardTermsIndexReader {
|
|||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", StandardCodec.TERMS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
public static void getIndexExtensions(Collection<String> extensions) {
|
||||
extensions.add(StandardCodec.TERMS_INDEX_EXTENSION);
|
||||
extensions.add(FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -32,9 +32,12 @@ import java.util.ArrayList;
|
|||
import java.io.IOException;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public class SimpleStandardTermsIndexWriter extends StandardTermsIndexWriter {
|
||||
public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
|
||||
protected final IndexOutput out;
|
||||
|
||||
/** Extension of terms index file */
|
||||
static final String TERMS_INDEX_EXTENSION = "tii";
|
||||
|
||||
final static String CODEC_NAME = "SIMPLE_STANDARD_TERMS_INDEX";
|
||||
final static int VERSION_START = 0;
|
||||
final static int VERSION_CURRENT = VERSION_START;
|
||||
|
@ -45,8 +48,8 @@ public class SimpleStandardTermsIndexWriter extends StandardTermsIndexWriter {
|
|||
private final FieldInfos fieldInfos; // unread
|
||||
private IndexOutput termsOut;
|
||||
|
||||
public SimpleStandardTermsIndexWriter(SegmentWriteState state) throws IOException {
|
||||
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, "", StandardCodec.TERMS_INDEX_EXTENSION);
|
||||
public FixedGapTermsIndexWriter(SegmentWriteState state) throws IOException {
|
||||
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, "", TERMS_INDEX_EXTENSION);
|
||||
state.flushedFiles.add(indexFileName);
|
||||
termIndexInterval = state.termIndexInterval;
|
||||
out = state.directory.createOutput(indexFileName);
|
|
@ -0,0 +1,56 @@
|
|||
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 org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
/** PrefixCodedTermsReader interacts with a single instance
|
||||
* of this 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
|
||||
* time.
|
||||
* @lucene.experimental */
|
||||
|
||||
public abstract class PostingsReaderBase implements Closeable {
|
||||
|
||||
public abstract void init(IndexInput termsIn) throws IOException;
|
||||
|
||||
/** Return a newly created empty TermState */
|
||||
public abstract TermState newTermState() throws IOException;
|
||||
|
||||
public abstract void readTerm(IndexInput termsIn, FieldInfo fieldInfo, TermState state, boolean isIndexTerm) throws IOException;
|
||||
|
||||
/** Must fully consume state, since after this call that
|
||||
* TermState may be reused. */
|
||||
public abstract DocsEnum docs(FieldInfo fieldInfo, TermState 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, TermState state, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException;
|
||||
|
||||
public abstract void close() throws IOException;
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
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 org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public abstract class PostingsWriterBase extends PostingsConsumer implements Closeable {
|
||||
|
||||
public abstract void start(IndexOutput termsOut) throws IOException;
|
||||
|
||||
public abstract void startTerm() throws IOException;
|
||||
|
||||
/** Finishes the current term */
|
||||
public abstract void finishTerm(int numDocs, boolean isIndexTerm) throws IOException;
|
||||
|
||||
public abstract void setField(FieldInfo fieldInfo);
|
||||
|
||||
public abstract void close() throws IOException;
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -33,7 +33,6 @@ import org.apache.lucene.index.IndexFileNames;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -49,17 +48,17 @@ import org.apache.lucene.util.CodecUtil;
|
|||
* make their own terms dict impl).
|
||||
*
|
||||
* <p>This class also interacts with an instance of {@link
|
||||
* StandardTermsIndexReader}, to abstract away the specific
|
||||
* TermsIndexReaderBase}, to abstract away the specific
|
||||
* implementation of the terms dict index.
|
||||
* @lucene.experimental */
|
||||
|
||||
public class StandardTermsDictReader extends FieldsProducer {
|
||||
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 StandardPostingsReader postingsReader;
|
||||
private final PostingsReaderBase postingsReader;
|
||||
|
||||
private final TreeMap<String,FieldReader> fields = new TreeMap<String,FieldReader>();
|
||||
|
||||
|
@ -70,7 +69,7 @@ public class StandardTermsDictReader extends FieldsProducer {
|
|||
private final DoubleBarrelLRUCache<FieldAndTerm,TermState> termsCache;
|
||||
|
||||
// Reads the terms index
|
||||
private StandardTermsIndexReader indexReader;
|
||||
private TermsIndexReaderBase indexReader;
|
||||
|
||||
// keeps the dirStart offset
|
||||
protected long dirOffset;
|
||||
|
@ -105,7 +104,7 @@ public class StandardTermsDictReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
|
||||
public StandardTermsDictReader(StandardTermsIndexReader indexReader, Directory dir, FieldInfos fieldInfos, String segment, StandardPostingsReader postingsReader, int readBufferSize,
|
||||
public PrefixCodedTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, int readBufferSize,
|
||||
Comparator<BytesRef> termComp, int termsCacheSize)
|
||||
throws IOException {
|
||||
|
||||
|
@ -114,7 +113,7 @@ public class StandardTermsDictReader extends FieldsProducer {
|
|||
|
||||
this.termComp = termComp;
|
||||
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, "", StandardCodec.TERMS_EXTENSION),
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, "", PrefixCodedTermsWriter.TERMS_EXTENSION),
|
||||
readBufferSize);
|
||||
|
||||
boolean success = false;
|
||||
|
@ -134,7 +133,7 @@ public class StandardTermsDictReader extends FieldsProducer {
|
|||
final long numTerms = in.readLong();
|
||||
assert numTerms >= 0;
|
||||
final long termsStartPointer = in.readLong();
|
||||
final StandardTermsIndexReader.FieldReader fieldIndexReader;
|
||||
final TermsIndexReaderBase.FieldReader fieldIndexReader;
|
||||
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
|
||||
fieldIndexReader = indexReader.getField(fieldInfo);
|
||||
if (numTerms > 0) {
|
||||
|
@ -153,8 +152,8 @@ public class StandardTermsDictReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
protected void readHeader(IndexInput input) throws IOException {
|
||||
CodecUtil.checkHeader(in, StandardTermsDictWriter.CODEC_NAME,
|
||||
StandardTermsDictWriter.VERSION_START, StandardTermsDictWriter.VERSION_CURRENT);
|
||||
CodecUtil.checkHeader(in, PrefixCodedTermsWriter.CODEC_NAME,
|
||||
PrefixCodedTermsWriter.VERSION_START, PrefixCodedTermsWriter.VERSION_CURRENT);
|
||||
dirOffset = in.readLong();
|
||||
}
|
||||
|
||||
|
@ -198,11 +197,11 @@ public class StandardTermsDictReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.TERMS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, "", PrefixCodedTermsWriter.TERMS_EXTENSION));
|
||||
}
|
||||
|
||||
public static void getExtensions(Collection<String> extensions) {
|
||||
extensions.add(StandardCodec.TERMS_EXTENSION);
|
||||
extensions.add(PrefixCodedTermsWriter.TERMS_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -245,9 +244,9 @@ public class StandardTermsDictReader extends FieldsProducer {
|
|||
final long numTerms;
|
||||
final FieldInfo fieldInfo;
|
||||
final long termsStartPointer;
|
||||
final StandardTermsIndexReader.FieldReader fieldIndexReader;
|
||||
final TermsIndexReaderBase.FieldReader fieldIndexReader;
|
||||
|
||||
FieldReader(StandardTermsIndexReader.FieldReader fieldIndexReader, FieldInfo fieldInfo, long numTerms, long termsStartPointer) {
|
||||
FieldReader(TermsIndexReaderBase.FieldReader fieldIndexReader, FieldInfo fieldInfo, long numTerms, long termsStartPointer) {
|
||||
assert numTerms > 0;
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.numTerms = numTerms;
|
||||
|
@ -281,11 +280,11 @@ public class StandardTermsDictReader extends FieldsProducer {
|
|||
private final DeltaBytesReader bytesReader;
|
||||
private final TermState state;
|
||||
private boolean seekPending;
|
||||
private final StandardTermsIndexReader.TermsIndexResult indexResult = new StandardTermsIndexReader.TermsIndexResult();
|
||||
private final TermsIndexReaderBase.TermsIndexResult indexResult = new TermsIndexReaderBase.TermsIndexResult();
|
||||
private final FieldAndTerm fieldTerm = new FieldAndTerm();
|
||||
|
||||
SegmentTermsEnum() throws IOException {
|
||||
in = (IndexInput) StandardTermsDictReader.this.in.clone();
|
||||
in = (IndexInput) PrefixCodedTermsReader.this.in.clone();
|
||||
in.seek(termsStartPointer);
|
||||
bytesReader = new DeltaBytesReader(in);
|
||||
fieldTerm.field = fieldInfo.name;
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -27,9 +27,6 @@ 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.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.PostingsConsumer;
|
||||
import org.apache.lucene.index.codecs.TermsConsumer;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
|
||||
|
@ -44,7 +41,7 @@ import org.apache.lucene.util.CodecUtil;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public class StandardTermsDictWriter extends FieldsConsumer {
|
||||
public class PrefixCodedTermsWriter extends FieldsConsumer {
|
||||
|
||||
final static String CODEC_NAME = "STANDARD_TERMS_DICT";
|
||||
|
||||
|
@ -53,23 +50,26 @@ public class StandardTermsDictWriter extends FieldsConsumer {
|
|||
|
||||
public static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
/** Extension of terms file */
|
||||
static final String TERMS_EXTENSION = "tis";
|
||||
|
||||
private final DeltaBytesWriter termWriter;
|
||||
|
||||
protected final IndexOutput out;
|
||||
final StandardPostingsWriter postingsWriter;
|
||||
final PostingsWriterBase postingsWriter;
|
||||
final FieldInfos fieldInfos;
|
||||
FieldInfo currentField;
|
||||
private final StandardTermsIndexWriter termsIndexWriter;
|
||||
private final TermsIndexWriterBase termsIndexWriter;
|
||||
private final List<TermsConsumer> fields = new ArrayList<TermsConsumer>();
|
||||
private final Comparator<BytesRef> termComp;
|
||||
|
||||
public StandardTermsDictWriter(
|
||||
StandardTermsIndexWriter termsIndexWriter,
|
||||
public PrefixCodedTermsWriter(
|
||||
TermsIndexWriterBase termsIndexWriter,
|
||||
SegmentWriteState state,
|
||||
StandardPostingsWriter postingsWriter,
|
||||
PostingsWriterBase postingsWriter,
|
||||
Comparator<BytesRef> termComp) throws IOException
|
||||
{
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, "", StandardCodec.TERMS_EXTENSION);
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, "", TERMS_EXTENSION);
|
||||
this.termsIndexWriter = termsIndexWriter;
|
||||
this.termComp = termComp;
|
||||
out = state.directory.createOutput(termsFileName);
|
||||
|
@ -96,7 +96,7 @@ public class StandardTermsDictWriter extends FieldsConsumer {
|
|||
public TermsConsumer addField(FieldInfo field) {
|
||||
assert currentField == null || currentField.name.compareTo(field.name) < 0;
|
||||
currentField = field;
|
||||
StandardTermsIndexWriter.FieldWriter fieldIndexWriter = termsIndexWriter.addField(field);
|
||||
TermsIndexWriterBase.FieldWriter fieldIndexWriter = termsIndexWriter.addField(field);
|
||||
TermsConsumer terms = new TermsWriter(fieldIndexWriter, field, postingsWriter);
|
||||
fields.add(terms);
|
||||
return terms;
|
||||
|
@ -139,15 +139,15 @@ public class StandardTermsDictWriter extends FieldsConsumer {
|
|||
|
||||
class TermsWriter extends TermsConsumer {
|
||||
private final FieldInfo fieldInfo;
|
||||
private final StandardPostingsWriter postingsWriter;
|
||||
private final PostingsWriterBase postingsWriter;
|
||||
private final long termsStartPointer;
|
||||
private long numTerms;
|
||||
private final StandardTermsIndexWriter.FieldWriter fieldIndexWriter;
|
||||
private final TermsIndexWriterBase.FieldWriter fieldIndexWriter;
|
||||
|
||||
TermsWriter(
|
||||
StandardTermsIndexWriter.FieldWriter fieldIndexWriter,
|
||||
TermsIndexWriterBase.FieldWriter fieldIndexWriter,
|
||||
FieldInfo fieldInfo,
|
||||
StandardPostingsWriter postingsWriter)
|
||||
PostingsWriterBase postingsWriter)
|
||||
{
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.fieldIndexWriter = fieldIndexWriter;
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -38,7 +38,7 @@ import java.util.Collection;
|
|||
* text.
|
||||
* @lucene.experimental */
|
||||
|
||||
public abstract class StandardTermsIndexReader {
|
||||
public abstract class TermsIndexReaderBase {
|
||||
|
||||
static class TermsIndexResult {
|
||||
long position;
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -23,7 +23,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
import java.io.IOException;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public abstract class StandardTermsIndexWriter {
|
||||
public abstract class TermsIndexWriterBase {
|
||||
|
||||
public abstract void setTermsOutput(IndexOutput out);
|
||||
|
|
@ -24,18 +24,18 @@ import org.apache.lucene.index.SegmentInfo;
|
|||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsWriterImpl;
|
||||
import org.apache.lucene.index.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReaderImpl;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsDictReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsDictWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsIndexWriter;
|
||||
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.TermsIndexReaderBase;
|
||||
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardCodec;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -66,19 +66,19 @@ public class PulsingCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
// We wrap StandardPostingsWriterImpl, but any StandardPostingsWriter
|
||||
// We wrap StandardPostingsWriter, but any StandardPostingsWriter
|
||||
// will work:
|
||||
StandardPostingsWriter docsWriter = new StandardPostingsWriterImpl(state);
|
||||
PostingsWriterBase docsWriter = new StandardPostingsWriter(state);
|
||||
|
||||
// Terms that have <= freqCutoff number of docs are
|
||||
// "pulsed" (inlined):
|
||||
StandardPostingsWriter pulsingWriter = new PulsingPostingsWriterImpl(freqCutoff, docsWriter);
|
||||
PostingsWriterBase pulsingWriter = new PulsingPostingsWriterImpl(freqCutoff, docsWriter);
|
||||
|
||||
// Terms dict index
|
||||
StandardTermsIndexWriter indexWriter;
|
||||
TermsIndexWriterBase indexWriter;
|
||||
boolean success = false;
|
||||
try {
|
||||
indexWriter = new SimpleStandardTermsIndexWriter(state);
|
||||
indexWriter = new FixedGapTermsIndexWriter(state);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -89,7 +89,7 @@ public class PulsingCodec extends Codec {
|
|||
// Terms dict
|
||||
success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new StandardTermsDictWriter(indexWriter, state, pulsingWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, pulsingWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
|
@ -106,17 +106,17 @@ public class PulsingCodec extends Codec {
|
|||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
|
||||
// We wrap StandardPostingsReaderImpl, but any StandardPostingsReader
|
||||
// We wrap StandardPostingsReader, but any StandardPostingsReader
|
||||
// will work:
|
||||
StandardPostingsReader docsReader = new StandardPostingsReaderImpl(state.dir, state.segmentInfo, state.readBufferSize);
|
||||
StandardPostingsReader pulsingReader = new PulsingPostingsReaderImpl(docsReader);
|
||||
PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize);
|
||||
PostingsReaderBase pulsingReader = new PulsingPostingsReaderImpl(docsReader);
|
||||
|
||||
// Terms dict index reader
|
||||
StandardTermsIndexReader indexReader;
|
||||
TermsIndexReaderBase indexReader;
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
indexReader = new SimpleStandardTermsIndexReader(state.dir,
|
||||
indexReader = new FixedGapTermsIndexReader(state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
state.termsIndexDivisor,
|
||||
|
@ -131,7 +131,7 @@ public class PulsingCodec extends Codec {
|
|||
// Terms dict reader
|
||||
success = false;
|
||||
try {
|
||||
FieldsProducer ret = new StandardTermsDictReader(indexReader,
|
||||
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
|
||||
state.dir, state.fieldInfos, state.segmentInfo.name,
|
||||
pulsingReader,
|
||||
state.readBufferSize,
|
||||
|
@ -152,9 +152,9 @@ public class PulsingCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) throws IOException {
|
||||
StandardPostingsReaderImpl.files(dir, segmentInfo, files);
|
||||
StandardTermsDictReader.files(dir, segmentInfo, files);
|
||||
SimpleStandardTermsIndexReader.files(dir, segmentInfo, files);
|
||||
StandardPostingsReader.files(dir, segmentInfo, files);
|
||||
PrefixCodedTermsReader.files(dir, segmentInfo, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,8 +22,8 @@ import java.io.IOException;
|
|||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.codecs.standard.TermState;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
|
||||
import org.apache.lucene.index.codecs.TermState;
|
||||
import org.apache.lucene.index.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl.Document;
|
||||
import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl.Position;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -39,13 +39,13 @@ import org.apache.lucene.util.CodecUtil;
|
|||
// create two separate docs readers, one that also reads
|
||||
// prox and one that doesn't?
|
||||
|
||||
public class PulsingPostingsReaderImpl extends StandardPostingsReader {
|
||||
public class PulsingPostingsReaderImpl extends PostingsReaderBase {
|
||||
|
||||
// Fallback reader for non-pulsed terms:
|
||||
final StandardPostingsReader wrappedPostingsReader;
|
||||
final PostingsReaderBase wrappedPostingsReader;
|
||||
int maxPulsingDocFreq;
|
||||
|
||||
public PulsingPostingsReaderImpl(StandardPostingsReader wrappedPostingsReader) throws IOException {
|
||||
public PulsingPostingsReaderImpl(PostingsReaderBase wrappedPostingsReader) throws IOException {
|
||||
this.wrappedPostingsReader = wrappedPostingsReader;
|
||||
}
|
||||
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
|
||||
import org.apache.lucene.index.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -34,7 +34,7 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
// presumably rare in practice...
|
||||
|
||||
/** @lucene.experimental */
|
||||
public final class PulsingPostingsWriterImpl extends StandardPostingsWriter {
|
||||
public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
|
||||
|
||||
final static String CODEC = "PulsedPostings";
|
||||
|
||||
|
@ -110,11 +110,11 @@ public final class PulsingPostingsWriterImpl extends StandardPostingsWriter {
|
|||
// 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:
|
||||
final StandardPostingsWriter wrappedPostingsWriter;
|
||||
final PostingsWriterBase wrappedPostingsWriter;
|
||||
|
||||
/** If docFreq <= maxPulsingDocFreq, its postings are
|
||||
* inlined into terms dict */
|
||||
public PulsingPostingsWriterImpl(int maxPulsingDocFreq, StandardPostingsWriter wrappedPostingsWriter) throws IOException {
|
||||
public PulsingPostingsWriterImpl(int maxPulsingDocFreq, PostingsWriterBase wrappedPostingsWriter) throws IOException {
|
||||
super();
|
||||
|
||||
pendingDocs = new Document[maxPulsingDocFreq];
|
||||
|
|
|
@ -25,8 +25,8 @@ import org.apache.lucene.index.DocsAndPositionsEnum;
|
|||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
|
||||
import org.apache.lucene.index.codecs.standard.TermState;
|
||||
import org.apache.lucene.index.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.codecs.TermState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -43,7 +43,7 @@ import org.apache.lucene.util.CodecUtil;
|
|||
// create two separate docs readers, one that also reads
|
||||
// prox and one that doesn't?
|
||||
|
||||
public class SepPostingsReaderImpl extends StandardPostingsReader {
|
||||
public class SepPostingsReaderImpl extends PostingsReaderBase {
|
||||
|
||||
final IntIndexInput freqIn;
|
||||
final IntIndexInput docIn;
|
||||
|
|
|
@ -24,7 +24,7 @@ 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.standard.StandardPostingsWriter;
|
||||
import org.apache.lucene.index.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
|
@ -33,7 +33,7 @@ import org.apache.lucene.util.CodecUtil;
|
|||
* to .pyl, skip data to .skp
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public final class SepPostingsWriterImpl extends StandardPostingsWriter {
|
||||
public final class SepPostingsWriterImpl extends PostingsWriterBase {
|
||||
final static String CODEC = "SepDocFreqSkip";
|
||||
|
||||
final static String DOC_EXTENSION = "doc";
|
||||
|
|
|
@ -27,6 +27,14 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.index.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
|
||||
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
|
||||
import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
|
||||
import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
|
||||
import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/** Default codec.
|
||||
|
@ -39,16 +47,16 @@ public class StandardCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
StandardPostingsWriter docs = new StandardPostingsWriterImpl(state);
|
||||
PostingsWriterBase docs = new StandardPostingsWriter(state);
|
||||
|
||||
// TODO: should we make the terms index more easily
|
||||
// pluggable? Ie so that this codec would record which
|
||||
// index impl was used, and switch on loading?
|
||||
// Or... you must make a new Codec for this?
|
||||
StandardTermsIndexWriter indexWriter;
|
||||
TermsIndexWriterBase indexWriter;
|
||||
boolean success = false;
|
||||
try {
|
||||
indexWriter = new SimpleStandardTermsIndexWriter(state);
|
||||
indexWriter = new FixedGapTermsIndexWriter(state);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -58,7 +66,7 @@ public class StandardCodec extends Codec {
|
|||
|
||||
success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new StandardTermsDictWriter(indexWriter, state, docs, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, docs, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
|
@ -76,12 +84,12 @@ public class StandardCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
StandardPostingsReader postings = new StandardPostingsReaderImpl(state.dir, state.segmentInfo, state.readBufferSize);
|
||||
StandardTermsIndexReader indexReader;
|
||||
PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize);
|
||||
TermsIndexReaderBase indexReader;
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
indexReader = new SimpleStandardTermsIndexReader(state.dir,
|
||||
indexReader = new FixedGapTermsIndexReader(state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
state.termsIndexDivisor,
|
||||
|
@ -95,7 +103,7 @@ public class StandardCodec extends Codec {
|
|||
|
||||
success = false;
|
||||
try {
|
||||
FieldsProducer ret = new StandardTermsDictReader(indexReader,
|
||||
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
|
||||
state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
|
@ -122,17 +130,11 @@ public class StandardCodec extends Codec {
|
|||
/** Extension of prox postings file */
|
||||
static final String PROX_EXTENSION = "prx";
|
||||
|
||||
/** Extension of terms file */
|
||||
static final String TERMS_EXTENSION = "tis";
|
||||
|
||||
/** Extension of terms index file */
|
||||
static final String TERMS_INDEX_EXTENSION = "tii";
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) throws IOException {
|
||||
StandardPostingsReaderImpl.files(dir, segmentInfo, files);
|
||||
StandardTermsDictReader.files(dir, segmentInfo, files);
|
||||
SimpleStandardTermsIndexReader.files(dir, segmentInfo, files);
|
||||
StandardPostingsReader.files(dir, segmentInfo, files);
|
||||
PrefixCodedTermsReader.files(dir, segmentInfo, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -143,7 +145,7 @@ public class StandardCodec extends Codec {
|
|||
public static void getStandardExtensions(Set<String> extensions) {
|
||||
extensions.add(FREQ_EXTENSION);
|
||||
extensions.add(PROX_EXTENSION);
|
||||
StandardTermsDictReader.getExtensions(extensions);
|
||||
SimpleStandardTermsIndexReader.getIndexExtensions(extensions);
|
||||
PrefixCodedTermsReader.getExtensions(extensions);
|
||||
FixedGapTermsIndexReader.getIndexExtensions(extensions);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,39 +18,596 @@ package org.apache.lucene.index.codecs.standard;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Closeable;
|
||||
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.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.codecs.TermState;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
|
||||
/** StandardTermsDictReader interacts with a single instance
|
||||
* of this 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
|
||||
* time.
|
||||
/** Concrete class that reads the current doc/freq/skip
|
||||
* postings format.
|
||||
* @lucene.experimental */
|
||||
|
||||
public abstract class StandardPostingsReader implements Closeable {
|
||||
public class StandardPostingsReader extends PostingsReaderBase {
|
||||
|
||||
public abstract void init(IndexInput termsIn) throws IOException;
|
||||
private final IndexInput freqIn;
|
||||
private final IndexInput proxIn;
|
||||
|
||||
/** Return a newly created empty TermState */
|
||||
public abstract TermState newTermState() throws IOException;
|
||||
int skipInterval;
|
||||
int maxSkipLevels;
|
||||
|
||||
public abstract void readTerm(IndexInput termsIn, FieldInfo fieldInfo, TermState state, boolean isIndexTerm) throws IOException;
|
||||
public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize) throws IOException {
|
||||
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.FREQ_EXTENSION),
|
||||
readBufferSize);
|
||||
if (segmentInfo.getHasProx()) {
|
||||
boolean success = false;
|
||||
try {
|
||||
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.PROX_EXTENSION),
|
||||
readBufferSize);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
freqIn.close();
|
||||
}
|
||||
}
|
||||
} else {
|
||||
proxIn = null;
|
||||
}
|
||||
}
|
||||
|
||||
/** Must fully consume state, since after this call that
|
||||
* TermState may be reused. */
|
||||
public abstract DocsEnum docs(FieldInfo fieldInfo, TermState state, Bits skipDocs, DocsEnum reuse) throws IOException;
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, Collection<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.FREQ_EXTENSION));
|
||||
if (segmentInfo.getHasProx()) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.PROX_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
||||
/** Must fully consume state, since after this call that
|
||||
* TermState may be reused. */
|
||||
public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, TermState state, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException;
|
||||
@Override
|
||||
public void init(IndexInput termsIn) throws IOException {
|
||||
|
||||
public abstract void close() throws IOException;
|
||||
// Make sure we are talking to the matching past writer
|
||||
CodecUtil.checkHeader(termsIn, StandardPostingsWriter.CODEC,
|
||||
StandardPostingsWriter.VERSION_START, StandardPostingsWriter.VERSION_START);
|
||||
|
||||
skipInterval = termsIn.readInt();
|
||||
maxSkipLevels = termsIn.readInt();
|
||||
}
|
||||
|
||||
private static class DocTermState extends TermState {
|
||||
long freqOffset;
|
||||
long proxOffset;
|
||||
int skipOffset;
|
||||
|
||||
public Object clone() {
|
||||
DocTermState other = (DocTermState) super.clone();
|
||||
other.freqOffset = freqOffset;
|
||||
other.proxOffset = proxOffset;
|
||||
other.skipOffset = skipOffset;
|
||||
return other;
|
||||
}
|
||||
|
||||
public void copy(TermState _other) {
|
||||
super.copy(_other);
|
||||
DocTermState other = (DocTermState) _other;
|
||||
freqOffset = other.freqOffset;
|
||||
proxOffset = other.proxOffset;
|
||||
skipOffset = other.skipOffset;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return super.toString() + " freqFP=" + freqOffset + " proxFP=" + proxOffset + " skipOffset=" + skipOffset;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermState newTermState() {
|
||||
return new DocTermState();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
if (freqIn != null) {
|
||||
freqIn.close();
|
||||
}
|
||||
} finally {
|
||||
if (proxIn != null) {
|
||||
proxIn.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readTerm(IndexInput termsIn, FieldInfo fieldInfo, TermState termState, boolean isIndexTerm)
|
||||
throws IOException {
|
||||
|
||||
final DocTermState docTermState = (DocTermState) termState;
|
||||
|
||||
if (isIndexTerm) {
|
||||
docTermState.freqOffset = termsIn.readVLong();
|
||||
} else {
|
||||
docTermState.freqOffset += termsIn.readVLong();
|
||||
}
|
||||
|
||||
if (docTermState.docFreq >= skipInterval) {
|
||||
docTermState.skipOffset = termsIn.readVInt();
|
||||
} else {
|
||||
docTermState.skipOffset = 0;
|
||||
}
|
||||
|
||||
if (!fieldInfo.omitTermFreqAndPositions) {
|
||||
if (isIndexTerm) {
|
||||
docTermState.proxOffset = termsIn.readVLong();
|
||||
} else {
|
||||
docTermState.proxOffset += termsIn.readVLong();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsEnum docs(FieldInfo fieldInfo, TermState termState, Bits skipDocs, DocsEnum reuse) throws IOException {
|
||||
SegmentDocsEnum docsEnum;
|
||||
if (reuse == null || !(reuse instanceof SegmentDocsEnum)) {
|
||||
docsEnum = new SegmentDocsEnum(freqIn);
|
||||
} else {
|
||||
docsEnum = (SegmentDocsEnum) reuse;
|
||||
if (docsEnum.startFreqIn != freqIn) {
|
||||
// If you are using ParellelReader, and pass in a
|
||||
// reused DocsEnum, it could have come from another
|
||||
// reader also using standard codec
|
||||
docsEnum = new SegmentDocsEnum(freqIn);
|
||||
}
|
||||
}
|
||||
return docsEnum.reset(fieldInfo, (DocTermState) termState, skipDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, TermState termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
|
||||
if (fieldInfo.omitTermFreqAndPositions) {
|
||||
return null;
|
||||
}
|
||||
SegmentDocsAndPositionsEnum docsEnum;
|
||||
if (reuse == null || !(reuse instanceof SegmentDocsAndPositionsEnum)) {
|
||||
docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn);
|
||||
} else {
|
||||
docsEnum = (SegmentDocsAndPositionsEnum) reuse;
|
||||
if (docsEnum.startFreqIn != freqIn) {
|
||||
// If you are using ParellelReader, and pass in a
|
||||
// reused DocsEnum, it could have come from another
|
||||
// reader also using standard codec
|
||||
docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn);
|
||||
}
|
||||
}
|
||||
return docsEnum.reset(fieldInfo, (DocTermState) termState, skipDocs);
|
||||
}
|
||||
|
||||
// Decodes only docs
|
||||
private class SegmentDocsEnum extends DocsEnum {
|
||||
final IndexInput freqIn;
|
||||
final IndexInput startFreqIn;
|
||||
|
||||
boolean omitTF; // does current field omit term freq?
|
||||
boolean storePayloads; // does current field store payloads?
|
||||
|
||||
int limit; // number of docs in this posting
|
||||
int ord; // how many docs we've read
|
||||
int doc; // doc we last read
|
||||
int freq; // freq we last read
|
||||
|
||||
Bits skipDocs;
|
||||
|
||||
long freqOffset;
|
||||
int skipOffset;
|
||||
|
||||
boolean skipped;
|
||||
DefaultSkipListReader skipper;
|
||||
|
||||
public SegmentDocsEnum(IndexInput freqIn) throws IOException {
|
||||
startFreqIn = freqIn;
|
||||
this.freqIn = (IndexInput) freqIn.clone();
|
||||
}
|
||||
|
||||
public SegmentDocsEnum reset(FieldInfo fieldInfo, DocTermState termState, Bits skipDocs) throws IOException {
|
||||
omitTF = fieldInfo.omitTermFreqAndPositions;
|
||||
if (omitTF) {
|
||||
freq = 1;
|
||||
}
|
||||
storePayloads = fieldInfo.storePayloads;
|
||||
this.skipDocs = skipDocs;
|
||||
freqOffset = termState.freqOffset;
|
||||
skipOffset = termState.skipOffset;
|
||||
|
||||
// TODO: for full enum case (eg segment merging) this
|
||||
// seek is unnecessary; maybe we can avoid in such
|
||||
// cases
|
||||
freqIn.seek(termState.freqOffset);
|
||||
limit = termState.docFreq;
|
||||
ord = 0;
|
||||
doc = 0;
|
||||
|
||||
skipped = false;
|
||||
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
while(true) {
|
||||
if (ord == limit) {
|
||||
return doc = NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
ord++;
|
||||
|
||||
// Decode next doc/freq pair
|
||||
final int code = freqIn.readVInt();
|
||||
if (omitTF) {
|
||||
doc += code;
|
||||
} else {
|
||||
doc += code >>> 1; // shift off low bit
|
||||
if ((code & 1) != 0) { // if low bit is set
|
||||
freq = 1; // freq is one
|
||||
} else {
|
||||
freq = freqIn.readVInt(); // else read freq
|
||||
}
|
||||
}
|
||||
|
||||
if (skipDocs == null || !skipDocs.get(doc)) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
|
||||
final int[] docs = bulkResult.docs.ints;
|
||||
final int[] freqs = bulkResult.freqs.ints;
|
||||
int i = 0;
|
||||
final int length = docs.length;
|
||||
while (i < length && ord < limit) {
|
||||
ord++;
|
||||
// manually inlined call to next() for speed
|
||||
final int code = freqIn.readVInt();
|
||||
if (omitTF) {
|
||||
doc += code;
|
||||
} else {
|
||||
doc += code >>> 1; // shift off low bit
|
||||
if ((code & 1) != 0) { // if low bit is set
|
||||
freq = 1; // freq is one
|
||||
} else {
|
||||
freq = freqIn.readVInt(); // else read freq
|
||||
}
|
||||
}
|
||||
|
||||
if (skipDocs == null || !skipDocs.get(doc)) {
|
||||
docs[i] = doc;
|
||||
freqs[i] = freq;
|
||||
++i;
|
||||
}
|
||||
}
|
||||
|
||||
return i;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() {
|
||||
return freq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
|
||||
// TODO: jump right to next() if target is < X away
|
||||
// from where we are now?
|
||||
|
||||
if (skipOffset > 0) {
|
||||
|
||||
// There are enough docs in the posting to have
|
||||
// skip data
|
||||
|
||||
if (skipper == null) {
|
||||
// This is the first time this enum has ever been used for skipping -- do lazy init
|
||||
skipper = new DefaultSkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
|
||||
}
|
||||
|
||||
if (!skipped) {
|
||||
|
||||
// This is the first time this posting has
|
||||
// skipped since reset() was called, so now we
|
||||
// load the skip data for this posting
|
||||
|
||||
skipper.init(freqOffset + skipOffset,
|
||||
freqOffset, 0,
|
||||
limit, storePayloads);
|
||||
|
||||
skipped = true;
|
||||
}
|
||||
|
||||
final int newOrd = skipper.skipTo(target);
|
||||
|
||||
if (newOrd > ord) {
|
||||
// Skipper moved
|
||||
|
||||
ord = newOrd;
|
||||
doc = skipper.getDoc();
|
||||
freqIn.seek(skipper.getFreqPointer());
|
||||
}
|
||||
}
|
||||
|
||||
// scan for the rest:
|
||||
do {
|
||||
nextDoc();
|
||||
} while (target > doc);
|
||||
|
||||
return doc;
|
||||
}
|
||||
}
|
||||
|
||||
// Decodes docs & positions
|
||||
private class SegmentDocsAndPositionsEnum extends DocsAndPositionsEnum {
|
||||
final IndexInput startFreqIn;
|
||||
private final IndexInput freqIn;
|
||||
private final IndexInput proxIn;
|
||||
|
||||
boolean storePayloads; // does current field store payloads?
|
||||
|
||||
int limit; // number of docs in this posting
|
||||
int ord; // how many docs we've read
|
||||
int doc; // doc we last read
|
||||
int freq; // freq we last read
|
||||
int position;
|
||||
|
||||
Bits skipDocs;
|
||||
|
||||
long freqOffset;
|
||||
int skipOffset;
|
||||
long proxOffset;
|
||||
|
||||
int posPendingCount;
|
||||
int payloadLength;
|
||||
boolean payloadPending;
|
||||
|
||||
boolean skipped;
|
||||
DefaultSkipListReader skipper;
|
||||
private BytesRef payload;
|
||||
private long lazyProxPointer;
|
||||
|
||||
public SegmentDocsAndPositionsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException {
|
||||
startFreqIn = freqIn;
|
||||
this.freqIn = (IndexInput) freqIn.clone();
|
||||
this.proxIn = (IndexInput) proxIn.clone();
|
||||
}
|
||||
|
||||
public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, DocTermState termState, Bits skipDocs) throws IOException {
|
||||
assert !fieldInfo.omitTermFreqAndPositions;
|
||||
storePayloads = fieldInfo.storePayloads;
|
||||
if (storePayloads && payload == null) {
|
||||
payload = new BytesRef();
|
||||
payload.bytes = new byte[1];
|
||||
}
|
||||
|
||||
this.skipDocs = skipDocs;
|
||||
|
||||
// TODO: for full enum case (eg segment merging) this
|
||||
// seek is unnecessary; maybe we can avoid in such
|
||||
// cases
|
||||
freqIn.seek(termState.freqOffset);
|
||||
lazyProxPointer = termState.proxOffset;
|
||||
|
||||
limit = termState.docFreq;
|
||||
ord = 0;
|
||||
doc = 0;
|
||||
position = 0;
|
||||
|
||||
skipped = false;
|
||||
posPendingCount = 0;
|
||||
payloadPending = false;
|
||||
|
||||
freqOffset = termState.freqOffset;
|
||||
proxOffset = termState.proxOffset;
|
||||
skipOffset = termState.skipOffset;
|
||||
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
while(true) {
|
||||
if (ord == limit) {
|
||||
return doc = NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
ord++;
|
||||
|
||||
// Decode next doc/freq pair
|
||||
final int code = freqIn.readVInt();
|
||||
|
||||
doc += code >>> 1; // shift off low bit
|
||||
if ((code & 1) != 0) { // if low bit is set
|
||||
freq = 1; // freq is one
|
||||
} else {
|
||||
freq = freqIn.readVInt(); // else read freq
|
||||
}
|
||||
posPendingCount += freq;
|
||||
|
||||
if (skipDocs == null || !skipDocs.get(doc)) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
position = 0;
|
||||
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() {
|
||||
return freq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
|
||||
// TODO: jump right to next() if target is < X away
|
||||
// from where we are now?
|
||||
|
||||
if (skipOffset > 0) {
|
||||
|
||||
// There are enough docs in the posting to have
|
||||
// skip data
|
||||
|
||||
if (skipper == null) {
|
||||
// This is the first time this enum has ever been used for skipping -- do lazy init
|
||||
skipper = new DefaultSkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
|
||||
}
|
||||
|
||||
if (!skipped) {
|
||||
|
||||
// This is the first time this posting has
|
||||
// skipped, since reset() was called, so now we
|
||||
// load the skip data for this posting
|
||||
|
||||
skipper.init(freqOffset+skipOffset,
|
||||
freqOffset, proxOffset,
|
||||
limit, storePayloads);
|
||||
|
||||
skipped = true;
|
||||
}
|
||||
|
||||
final int newOrd = skipper.skipTo(target);
|
||||
|
||||
if (newOrd > ord) {
|
||||
// Skipper moved
|
||||
ord = newOrd;
|
||||
doc = skipper.getDoc();
|
||||
freqIn.seek(skipper.getFreqPointer());
|
||||
lazyProxPointer = skipper.getProxPointer();
|
||||
posPendingCount = 0;
|
||||
position = 0;
|
||||
payloadPending = false;
|
||||
payloadLength = skipper.getPayloadLength();
|
||||
}
|
||||
}
|
||||
|
||||
// Now, linear scan for the rest:
|
||||
do {
|
||||
nextDoc();
|
||||
} while (target > doc);
|
||||
|
||||
return doc;
|
||||
}
|
||||
|
||||
public int nextPosition() throws IOException {
|
||||
|
||||
if (lazyProxPointer != -1) {
|
||||
proxIn.seek(lazyProxPointer);
|
||||
lazyProxPointer = -1;
|
||||
}
|
||||
|
||||
if (payloadPending && payloadLength > 0) {
|
||||
// payload of last position as never retrieved -- skip it
|
||||
proxIn.seek(proxIn.getFilePointer() + payloadLength);
|
||||
payloadPending = false;
|
||||
}
|
||||
|
||||
// scan over any docs that were iterated without their positions
|
||||
while(posPendingCount > freq) {
|
||||
|
||||
final int code = proxIn.readVInt();
|
||||
|
||||
if (storePayloads) {
|
||||
if ((code & 1) != 0) {
|
||||
// new payload length
|
||||
payloadLength = proxIn.readVInt();
|
||||
assert payloadLength >= 0;
|
||||
}
|
||||
assert payloadLength != -1;
|
||||
proxIn.seek(proxIn.getFilePointer() + payloadLength);
|
||||
}
|
||||
|
||||
posPendingCount--;
|
||||
position = 0;
|
||||
payloadPending = false;
|
||||
}
|
||||
|
||||
// read next position
|
||||
if (storePayloads) {
|
||||
|
||||
if (payloadPending && payloadLength > 0) {
|
||||
// payload wasn't retrieved for last position
|
||||
proxIn.seek(proxIn.getFilePointer()+payloadLength);
|
||||
}
|
||||
|
||||
final int code = proxIn.readVInt();
|
||||
if ((code & 1) != 0) {
|
||||
// new payload length
|
||||
payloadLength = proxIn.readVInt();
|
||||
assert payloadLength >= 0;
|
||||
}
|
||||
assert payloadLength != -1;
|
||||
|
||||
payloadPending = true;
|
||||
position += code >>> 1;
|
||||
} else {
|
||||
position += proxIn.readVInt();
|
||||
}
|
||||
|
||||
posPendingCount--;
|
||||
|
||||
assert posPendingCount >= 0: "nextPosition() was called too many times (more than freq() times) posPendingCount=" + posPendingCount;
|
||||
|
||||
return position;
|
||||
}
|
||||
|
||||
/** Returns length of payload at current position */
|
||||
public int getPayloadLength() {
|
||||
assert lazyProxPointer == -1;
|
||||
assert posPendingCount < freq;
|
||||
return payloadLength;
|
||||
}
|
||||
|
||||
/** Returns the payload at this position, or null if no
|
||||
* payload was indexed. */
|
||||
public BytesRef getPayload() throws IOException {
|
||||
assert lazyProxPointer == -1;
|
||||
assert posPendingCount < freq;
|
||||
if (!payloadPending) {
|
||||
throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once.");
|
||||
}
|
||||
if (payloadLength > payload.bytes.length) {
|
||||
payload.grow(payloadLength);
|
||||
}
|
||||
proxIn.readBytes(payload.bytes, 0, payloadLength);
|
||||
payload.length = payloadLength;
|
||||
payloadPending = false;
|
||||
|
||||
return payload;
|
||||
}
|
||||
|
||||
public boolean hasPayload() {
|
||||
return payloadPending && payloadLength > 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,611 +0,0 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
|
||||
/**
|
||||
* 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.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.IndexFileNames;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
|
||||
/** Concrete class that reads the current doc/freq/skip
|
||||
* postings format.
|
||||
* @lucene.experimental */
|
||||
|
||||
public class StandardPostingsReaderImpl extends StandardPostingsReader {
|
||||
|
||||
private final IndexInput freqIn;
|
||||
private final IndexInput proxIn;
|
||||
|
||||
int skipInterval;
|
||||
int maxSkipLevels;
|
||||
|
||||
public StandardPostingsReaderImpl(Directory dir, SegmentInfo segmentInfo, int readBufferSize) throws IOException {
|
||||
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.FREQ_EXTENSION),
|
||||
readBufferSize);
|
||||
if (segmentInfo.getHasProx()) {
|
||||
boolean success = false;
|
||||
try {
|
||||
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.PROX_EXTENSION),
|
||||
readBufferSize);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
freqIn.close();
|
||||
}
|
||||
}
|
||||
} else {
|
||||
proxIn = null;
|
||||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, Collection<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.FREQ_EXTENSION));
|
||||
if (segmentInfo.getHasProx()) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.PROX_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(IndexInput termsIn) throws IOException {
|
||||
|
||||
// Make sure we are talking to the matching past writer
|
||||
CodecUtil.checkHeader(termsIn, StandardPostingsWriterImpl.CODEC,
|
||||
StandardPostingsWriterImpl.VERSION_START, StandardPostingsWriterImpl.VERSION_START);
|
||||
|
||||
skipInterval = termsIn.readInt();
|
||||
maxSkipLevels = termsIn.readInt();
|
||||
}
|
||||
|
||||
private static class DocTermState extends TermState {
|
||||
long freqOffset;
|
||||
long proxOffset;
|
||||
int skipOffset;
|
||||
|
||||
public Object clone() {
|
||||
DocTermState other = (DocTermState) super.clone();
|
||||
other.freqOffset = freqOffset;
|
||||
other.proxOffset = proxOffset;
|
||||
other.skipOffset = skipOffset;
|
||||
return other;
|
||||
}
|
||||
|
||||
public void copy(TermState _other) {
|
||||
super.copy(_other);
|
||||
DocTermState other = (DocTermState) _other;
|
||||
freqOffset = other.freqOffset;
|
||||
proxOffset = other.proxOffset;
|
||||
skipOffset = other.skipOffset;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return super.toString() + " freqFP=" + freqOffset + " proxFP=" + proxOffset + " skipOffset=" + skipOffset;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermState newTermState() {
|
||||
return new DocTermState();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
if (freqIn != null) {
|
||||
freqIn.close();
|
||||
}
|
||||
} finally {
|
||||
if (proxIn != null) {
|
||||
proxIn.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readTerm(IndexInput termsIn, FieldInfo fieldInfo, TermState termState, boolean isIndexTerm)
|
||||
throws IOException {
|
||||
|
||||
final DocTermState docTermState = (DocTermState) termState;
|
||||
|
||||
if (isIndexTerm) {
|
||||
docTermState.freqOffset = termsIn.readVLong();
|
||||
} else {
|
||||
docTermState.freqOffset += termsIn.readVLong();
|
||||
}
|
||||
|
||||
if (docTermState.docFreq >= skipInterval) {
|
||||
docTermState.skipOffset = termsIn.readVInt();
|
||||
} else {
|
||||
docTermState.skipOffset = 0;
|
||||
}
|
||||
|
||||
if (!fieldInfo.omitTermFreqAndPositions) {
|
||||
if (isIndexTerm) {
|
||||
docTermState.proxOffset = termsIn.readVLong();
|
||||
} else {
|
||||
docTermState.proxOffset += termsIn.readVLong();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsEnum docs(FieldInfo fieldInfo, TermState termState, Bits skipDocs, DocsEnum reuse) throws IOException {
|
||||
SegmentDocsEnum docsEnum;
|
||||
if (reuse == null || !(reuse instanceof SegmentDocsEnum)) {
|
||||
docsEnum = new SegmentDocsEnum(freqIn);
|
||||
} else {
|
||||
docsEnum = (SegmentDocsEnum) reuse;
|
||||
if (docsEnum.startFreqIn != freqIn) {
|
||||
// If you are using ParellelReader, and pass in a
|
||||
// reused DocsEnum, it could have come from another
|
||||
// reader also using standard codec
|
||||
docsEnum = new SegmentDocsEnum(freqIn);
|
||||
}
|
||||
}
|
||||
return docsEnum.reset(fieldInfo, (DocTermState) termState, skipDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, TermState termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
|
||||
if (fieldInfo.omitTermFreqAndPositions) {
|
||||
return null;
|
||||
}
|
||||
SegmentDocsAndPositionsEnum docsEnum;
|
||||
if (reuse == null || !(reuse instanceof SegmentDocsAndPositionsEnum)) {
|
||||
docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn);
|
||||
} else {
|
||||
docsEnum = (SegmentDocsAndPositionsEnum) reuse;
|
||||
if (docsEnum.startFreqIn != freqIn) {
|
||||
// If you are using ParellelReader, and pass in a
|
||||
// reused DocsEnum, it could have come from another
|
||||
// reader also using standard codec
|
||||
docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn);
|
||||
}
|
||||
}
|
||||
return docsEnum.reset(fieldInfo, (DocTermState) termState, skipDocs);
|
||||
}
|
||||
|
||||
// Decodes only docs
|
||||
private class SegmentDocsEnum extends DocsEnum {
|
||||
final IndexInput freqIn;
|
||||
final IndexInput startFreqIn;
|
||||
|
||||
boolean omitTF; // does current field omit term freq?
|
||||
boolean storePayloads; // does current field store payloads?
|
||||
|
||||
int limit; // number of docs in this posting
|
||||
int ord; // how many docs we've read
|
||||
int doc; // doc we last read
|
||||
int freq; // freq we last read
|
||||
|
||||
Bits skipDocs;
|
||||
|
||||
long freqOffset;
|
||||
int skipOffset;
|
||||
|
||||
boolean skipped;
|
||||
DefaultSkipListReader skipper;
|
||||
|
||||
public SegmentDocsEnum(IndexInput freqIn) throws IOException {
|
||||
startFreqIn = freqIn;
|
||||
this.freqIn = (IndexInput) freqIn.clone();
|
||||
}
|
||||
|
||||
public SegmentDocsEnum reset(FieldInfo fieldInfo, DocTermState termState, Bits skipDocs) throws IOException {
|
||||
omitTF = fieldInfo.omitTermFreqAndPositions;
|
||||
if (omitTF) {
|
||||
freq = 1;
|
||||
}
|
||||
storePayloads = fieldInfo.storePayloads;
|
||||
this.skipDocs = skipDocs;
|
||||
freqOffset = termState.freqOffset;
|
||||
skipOffset = termState.skipOffset;
|
||||
|
||||
// TODO: for full enum case (eg segment merging) this
|
||||
// seek is unnecessary; maybe we can avoid in such
|
||||
// cases
|
||||
freqIn.seek(termState.freqOffset);
|
||||
limit = termState.docFreq;
|
||||
ord = 0;
|
||||
doc = 0;
|
||||
|
||||
skipped = false;
|
||||
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
while(true) {
|
||||
if (ord == limit) {
|
||||
return doc = NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
ord++;
|
||||
|
||||
// Decode next doc/freq pair
|
||||
final int code = freqIn.readVInt();
|
||||
if (omitTF) {
|
||||
doc += code;
|
||||
} else {
|
||||
doc += code >>> 1; // shift off low bit
|
||||
if ((code & 1) != 0) { // if low bit is set
|
||||
freq = 1; // freq is one
|
||||
} else {
|
||||
freq = freqIn.readVInt(); // else read freq
|
||||
}
|
||||
}
|
||||
|
||||
if (skipDocs == null || !skipDocs.get(doc)) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
|
||||
final int[] docs = bulkResult.docs.ints;
|
||||
final int[] freqs = bulkResult.freqs.ints;
|
||||
int i = 0;
|
||||
final int length = docs.length;
|
||||
while (i < length && ord < limit) {
|
||||
ord++;
|
||||
// manually inlined call to next() for speed
|
||||
final int code = freqIn.readVInt();
|
||||
if (omitTF) {
|
||||
doc += code;
|
||||
} else {
|
||||
doc += code >>> 1; // shift off low bit
|
||||
if ((code & 1) != 0) { // if low bit is set
|
||||
freq = 1; // freq is one
|
||||
} else {
|
||||
freq = freqIn.readVInt(); // else read freq
|
||||
}
|
||||
}
|
||||
|
||||
if (skipDocs == null || !skipDocs.get(doc)) {
|
||||
docs[i] = doc;
|
||||
freqs[i] = freq;
|
||||
++i;
|
||||
}
|
||||
}
|
||||
|
||||
return i;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() {
|
||||
return freq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
|
||||
// TODO: jump right to next() if target is < X away
|
||||
// from where we are now?
|
||||
|
||||
if (skipOffset > 0) {
|
||||
|
||||
// There are enough docs in the posting to have
|
||||
// skip data
|
||||
|
||||
if (skipper == null) {
|
||||
// This is the first time this enum has ever been used for skipping -- do lazy init
|
||||
skipper = new DefaultSkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
|
||||
}
|
||||
|
||||
if (!skipped) {
|
||||
|
||||
// This is the first time this posting has
|
||||
// skipped since reset() was called, so now we
|
||||
// load the skip data for this posting
|
||||
|
||||
skipper.init(freqOffset + skipOffset,
|
||||
freqOffset, 0,
|
||||
limit, storePayloads);
|
||||
|
||||
skipped = true;
|
||||
}
|
||||
|
||||
final int newOrd = skipper.skipTo(target);
|
||||
|
||||
if (newOrd > ord) {
|
||||
// Skipper moved
|
||||
|
||||
ord = newOrd;
|
||||
doc = skipper.getDoc();
|
||||
freqIn.seek(skipper.getFreqPointer());
|
||||
}
|
||||
}
|
||||
|
||||
// scan for the rest:
|
||||
do {
|
||||
nextDoc();
|
||||
} while (target > doc);
|
||||
|
||||
return doc;
|
||||
}
|
||||
}
|
||||
|
||||
// Decodes docs & positions
|
||||
private class SegmentDocsAndPositionsEnum extends DocsAndPositionsEnum {
|
||||
final IndexInput startFreqIn;
|
||||
private final IndexInput freqIn;
|
||||
private final IndexInput proxIn;
|
||||
|
||||
boolean storePayloads; // does current field store payloads?
|
||||
|
||||
int limit; // number of docs in this posting
|
||||
int ord; // how many docs we've read
|
||||
int doc; // doc we last read
|
||||
int freq; // freq we last read
|
||||
int position;
|
||||
|
||||
Bits skipDocs;
|
||||
|
||||
long freqOffset;
|
||||
int skipOffset;
|
||||
long proxOffset;
|
||||
|
||||
int posPendingCount;
|
||||
int payloadLength;
|
||||
boolean payloadPending;
|
||||
|
||||
boolean skipped;
|
||||
DefaultSkipListReader skipper;
|
||||
private BytesRef payload;
|
||||
private long lazyProxPointer;
|
||||
|
||||
public SegmentDocsAndPositionsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException {
|
||||
startFreqIn = freqIn;
|
||||
this.freqIn = (IndexInput) freqIn.clone();
|
||||
this.proxIn = (IndexInput) proxIn.clone();
|
||||
}
|
||||
|
||||
public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, DocTermState termState, Bits skipDocs) throws IOException {
|
||||
assert !fieldInfo.omitTermFreqAndPositions;
|
||||
storePayloads = fieldInfo.storePayloads;
|
||||
if (storePayloads && payload == null) {
|
||||
payload = new BytesRef();
|
||||
payload.bytes = new byte[1];
|
||||
}
|
||||
|
||||
this.skipDocs = skipDocs;
|
||||
|
||||
// TODO: for full enum case (eg segment merging) this
|
||||
// seek is unnecessary; maybe we can avoid in such
|
||||
// cases
|
||||
freqIn.seek(termState.freqOffset);
|
||||
lazyProxPointer = termState.proxOffset;
|
||||
|
||||
limit = termState.docFreq;
|
||||
ord = 0;
|
||||
doc = 0;
|
||||
position = 0;
|
||||
|
||||
skipped = false;
|
||||
posPendingCount = 0;
|
||||
payloadPending = false;
|
||||
|
||||
freqOffset = termState.freqOffset;
|
||||
proxOffset = termState.proxOffset;
|
||||
skipOffset = termState.skipOffset;
|
||||
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
while(true) {
|
||||
if (ord == limit) {
|
||||
return doc = NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
ord++;
|
||||
|
||||
// Decode next doc/freq pair
|
||||
final int code = freqIn.readVInt();
|
||||
|
||||
doc += code >>> 1; // shift off low bit
|
||||
if ((code & 1) != 0) { // if low bit is set
|
||||
freq = 1; // freq is one
|
||||
} else {
|
||||
freq = freqIn.readVInt(); // else read freq
|
||||
}
|
||||
posPendingCount += freq;
|
||||
|
||||
if (skipDocs == null || !skipDocs.get(doc)) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
position = 0;
|
||||
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() {
|
||||
return freq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
|
||||
// TODO: jump right to next() if target is < X away
|
||||
// from where we are now?
|
||||
|
||||
if (skipOffset > 0) {
|
||||
|
||||
// There are enough docs in the posting to have
|
||||
// skip data
|
||||
|
||||
if (skipper == null) {
|
||||
// This is the first time this enum has ever been used for skipping -- do lazy init
|
||||
skipper = new DefaultSkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
|
||||
}
|
||||
|
||||
if (!skipped) {
|
||||
|
||||
// This is the first time this posting has
|
||||
// skipped, since reset() was called, so now we
|
||||
// load the skip data for this posting
|
||||
|
||||
skipper.init(freqOffset+skipOffset,
|
||||
freqOffset, proxOffset,
|
||||
limit, storePayloads);
|
||||
|
||||
skipped = true;
|
||||
}
|
||||
|
||||
final int newOrd = skipper.skipTo(target);
|
||||
|
||||
if (newOrd > ord) {
|
||||
// Skipper moved
|
||||
ord = newOrd;
|
||||
doc = skipper.getDoc();
|
||||
freqIn.seek(skipper.getFreqPointer());
|
||||
lazyProxPointer = skipper.getProxPointer();
|
||||
posPendingCount = 0;
|
||||
position = 0;
|
||||
payloadPending = false;
|
||||
payloadLength = skipper.getPayloadLength();
|
||||
}
|
||||
}
|
||||
|
||||
// Now, linear scan for the rest:
|
||||
do {
|
||||
nextDoc();
|
||||
} while (target > doc);
|
||||
|
||||
return doc;
|
||||
}
|
||||
|
||||
public int nextPosition() throws IOException {
|
||||
|
||||
if (lazyProxPointer != -1) {
|
||||
proxIn.seek(lazyProxPointer);
|
||||
lazyProxPointer = -1;
|
||||
}
|
||||
|
||||
if (payloadPending && payloadLength > 0) {
|
||||
// payload of last position as never retrieved -- skip it
|
||||
proxIn.seek(proxIn.getFilePointer() + payloadLength);
|
||||
payloadPending = false;
|
||||
}
|
||||
|
||||
// scan over any docs that were iterated without their positions
|
||||
while(posPendingCount > freq) {
|
||||
|
||||
final int code = proxIn.readVInt();
|
||||
|
||||
if (storePayloads) {
|
||||
if ((code & 1) != 0) {
|
||||
// new payload length
|
||||
payloadLength = proxIn.readVInt();
|
||||
assert payloadLength >= 0;
|
||||
}
|
||||
assert payloadLength != -1;
|
||||
proxIn.seek(proxIn.getFilePointer() + payloadLength);
|
||||
}
|
||||
|
||||
posPendingCount--;
|
||||
position = 0;
|
||||
payloadPending = false;
|
||||
}
|
||||
|
||||
// read next position
|
||||
if (storePayloads) {
|
||||
|
||||
if (payloadPending && payloadLength > 0) {
|
||||
// payload wasn't retrieved for last position
|
||||
proxIn.seek(proxIn.getFilePointer()+payloadLength);
|
||||
}
|
||||
|
||||
final int code = proxIn.readVInt();
|
||||
if ((code & 1) != 0) {
|
||||
// new payload length
|
||||
payloadLength = proxIn.readVInt();
|
||||
assert payloadLength >= 0;
|
||||
}
|
||||
assert payloadLength != -1;
|
||||
|
||||
payloadPending = true;
|
||||
position += code >>> 1;
|
||||
} else {
|
||||
position += proxIn.readVInt();
|
||||
}
|
||||
|
||||
posPendingCount--;
|
||||
|
||||
assert posPendingCount >= 0: "nextPosition() was called too many times (more than freq() times) posPendingCount=" + posPendingCount;
|
||||
|
||||
return position;
|
||||
}
|
||||
|
||||
/** Returns length of payload at current position */
|
||||
public int getPayloadLength() {
|
||||
assert lazyProxPointer == -1;
|
||||
assert posPendingCount < freq;
|
||||
return payloadLength;
|
||||
}
|
||||
|
||||
/** Returns the payload at this position, or null if no
|
||||
* payload was indexed. */
|
||||
public BytesRef getPayload() throws IOException {
|
||||
assert lazyProxPointer == -1;
|
||||
assert posPendingCount < freq;
|
||||
if (!payloadPending) {
|
||||
throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once.");
|
||||
}
|
||||
if (payloadLength > payload.bytes.length) {
|
||||
payload.grow(payloadLength);
|
||||
}
|
||||
proxIn.readBytes(payload.bytes, 0, payloadLength);
|
||||
payload.length = payloadLength;
|
||||
payloadPending = false;
|
||||
|
||||
return payload;
|
||||
}
|
||||
|
||||
public boolean hasPayload() {
|
||||
return payloadPending && payloadLength > 0;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,27 +17,219 @@ package org.apache.lucene.index.codecs.standard;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/** Consumes doc & freq, writing them using the current
|
||||
* index file format */
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Closeable;
|
||||
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.codecs.PostingsConsumer;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
/** @lucene.experimental */
|
||||
public final class StandardPostingsWriter extends PostingsWriterBase {
|
||||
final static String CODEC = "StandardPostingsWriterImpl";
|
||||
|
||||
public abstract class StandardPostingsWriter extends PostingsConsumer implements Closeable {
|
||||
// Increment version to change it:
|
||||
final static int VERSION_START = 0;
|
||||
final static int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
public abstract void start(IndexOutput termsOut) throws IOException;
|
||||
final IndexOutput freqOut;
|
||||
final IndexOutput proxOut;
|
||||
final DefaultSkipListWriter skipListWriter;
|
||||
final int skipInterval;
|
||||
final int maxSkipLevels;
|
||||
final int totalNumDocs;
|
||||
IndexOutput termsOut;
|
||||
|
||||
public abstract void startTerm() throws IOException;
|
||||
boolean omitTermFreqAndPositions;
|
||||
boolean storePayloads;
|
||||
// Starts a new term
|
||||
long lastFreqStart;
|
||||
long freqStart;
|
||||
long lastProxStart;
|
||||
long proxStart;
|
||||
FieldInfo fieldInfo;
|
||||
int lastPayloadLength;
|
||||
int lastPosition;
|
||||
|
||||
/** Finishes the current term */
|
||||
public abstract void finishTerm(int numDocs, boolean isIndexTerm) throws IOException;
|
||||
public StandardPostingsWriter(SegmentWriteState state) throws IOException {
|
||||
super();
|
||||
String fileName = IndexFileNames.segmentFileName(state.segmentName, "", StandardCodec.FREQ_EXTENSION);
|
||||
state.flushedFiles.add(fileName);
|
||||
freqOut = state.directory.createOutput(fileName);
|
||||
|
||||
public abstract void setField(FieldInfo fieldInfo);
|
||||
if (state.fieldInfos.hasProx()) {
|
||||
// At least one field does not omit TF, so create the
|
||||
// prox file
|
||||
fileName = IndexFileNames.segmentFileName(state.segmentName, "", StandardCodec.PROX_EXTENSION);
|
||||
state.flushedFiles.add(fileName);
|
||||
proxOut = state.directory.createOutput(fileName);
|
||||
} else {
|
||||
// Every field omits TF so we will write no prox file
|
||||
proxOut = null;
|
||||
}
|
||||
|
||||
public abstract void close() throws IOException;
|
||||
totalNumDocs = state.numDocs;
|
||||
|
||||
skipListWriter = new DefaultSkipListWriter(state.skipInterval,
|
||||
state.maxSkipLevels,
|
||||
state.numDocs,
|
||||
freqOut,
|
||||
proxOut);
|
||||
|
||||
skipInterval = state.skipInterval;
|
||||
maxSkipLevels = state.maxSkipLevels;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start(IndexOutput termsOut) throws IOException {
|
||||
this.termsOut = termsOut;
|
||||
CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT);
|
||||
termsOut.writeInt(skipInterval); // write skipInterval
|
||||
termsOut.writeInt(maxSkipLevels); // write maxSkipLevels
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startTerm() {
|
||||
freqStart = freqOut.getFilePointer();
|
||||
if (proxOut != null) {
|
||||
proxStart = proxOut.getFilePointer();
|
||||
// force first payload to write its length
|
||||
lastPayloadLength = -1;
|
||||
}
|
||||
skipListWriter.resetSkip();
|
||||
}
|
||||
|
||||
// Currently, this instance is re-used across fields, so
|
||||
// our parent calls setField whenever the field changes
|
||||
@Override
|
||||
public void setField(FieldInfo fieldInfo) {
|
||||
this.fieldInfo = fieldInfo;
|
||||
omitTermFreqAndPositions = fieldInfo.omitTermFreqAndPositions;
|
||||
storePayloads = fieldInfo.storePayloads;
|
||||
}
|
||||
|
||||
int lastDocID;
|
||||
int df;
|
||||
|
||||
/** 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 {
|
||||
|
||||
final int delta = docID - lastDocID;
|
||||
|
||||
if (docID < 0 || (df > 0 && delta <= 0)) {
|
||||
throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )");
|
||||
}
|
||||
|
||||
if ((++df % skipInterval) == 0) {
|
||||
skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength);
|
||||
skipListWriter.bufferSkip(df);
|
||||
}
|
||||
|
||||
assert docID < totalNumDocs: "docID=" + docID + " totalNumDocs=" + totalNumDocs;
|
||||
|
||||
lastDocID = docID;
|
||||
if (omitTermFreqAndPositions) {
|
||||
freqOut.writeVInt(delta);
|
||||
} else if (1 == termDocFreq) {
|
||||
freqOut.writeVInt((delta<<1) | 1);
|
||||
} else {
|
||||
freqOut.writeVInt(delta<<1);
|
||||
freqOut.writeVInt(termDocFreq);
|
||||
}
|
||||
|
||||
lastPosition = 0;
|
||||
}
|
||||
|
||||
/** Add a new position & payload */
|
||||
@Override
|
||||
public void addPosition(int position, BytesRef payload) throws IOException {
|
||||
assert !omitTermFreqAndPositions: "omitTermFreqAndPositions is true";
|
||||
assert proxOut != null;
|
||||
|
||||
final int delta = position - lastPosition;
|
||||
|
||||
assert delta > 0 || position == 0 || position == -1: "position=" + position + " lastPosition=" + lastPosition; // not quite right (if pos=0 is repeated twice we don't catch it)
|
||||
|
||||
lastPosition = position;
|
||||
|
||||
if (storePayloads) {
|
||||
final int payloadLength = payload == null ? 0 : payload.length;
|
||||
|
||||
if (payloadLength != lastPayloadLength) {
|
||||
lastPayloadLength = payloadLength;
|
||||
proxOut.writeVInt((delta<<1)|1);
|
||||
proxOut.writeVInt(payloadLength);
|
||||
} else {
|
||||
proxOut.writeVInt(delta << 1);
|
||||
}
|
||||
|
||||
if (payloadLength > 0) {
|
||||
proxOut.writeBytes(payload.bytes, payload.offset, payloadLength);
|
||||
}
|
||||
} else {
|
||||
proxOut.writeVInt(delta);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishDoc() {
|
||||
}
|
||||
|
||||
/** Called when we are done adding docs to this term */
|
||||
@Override
|
||||
public void finishTerm(int docCount, boolean isIndexTerm) throws IOException {
|
||||
assert docCount > 0;
|
||||
|
||||
// TODO: wasteful we are counting this (counting # docs
|
||||
// for this term) in two places?
|
||||
assert docCount == df;
|
||||
|
||||
if (isIndexTerm) {
|
||||
// Write absolute at seek points
|
||||
termsOut.writeVLong(freqStart);
|
||||
} else {
|
||||
// Write delta between seek points
|
||||
termsOut.writeVLong(freqStart - lastFreqStart);
|
||||
}
|
||||
|
||||
lastFreqStart = freqStart;
|
||||
|
||||
if (df >= skipInterval) {
|
||||
termsOut.writeVInt((int) (skipListWriter.writeSkip(freqOut)-freqStart));
|
||||
}
|
||||
|
||||
if (!omitTermFreqAndPositions) {
|
||||
if (isIndexTerm) {
|
||||
// Write absolute at seek points
|
||||
termsOut.writeVLong(proxStart);
|
||||
} else {
|
||||
// Write delta between seek points
|
||||
termsOut.writeVLong(proxStart - lastProxStart);
|
||||
}
|
||||
lastProxStart = proxStart;
|
||||
}
|
||||
|
||||
lastDocID = 0;
|
||||
df = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
freqOut.close();
|
||||
} finally {
|
||||
if (proxOut != null) {
|
||||
proxOut.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,234 +0,0 @@
|
|||
package org.apache.lucene.index.codecs.standard;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/** Consumes doc & freq, writing them using the current
|
||||
* index file format */
|
||||
|
||||
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.util.BytesRef;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public final class StandardPostingsWriterImpl extends StandardPostingsWriter {
|
||||
final static String CODEC = "StandardPostingsWriterImpl";
|
||||
|
||||
// Increment version to change it:
|
||||
final static int VERSION_START = 0;
|
||||
final static int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
final IndexOutput freqOut;
|
||||
final IndexOutput proxOut;
|
||||
final DefaultSkipListWriter skipListWriter;
|
||||
final int skipInterval;
|
||||
final int maxSkipLevels;
|
||||
final int totalNumDocs;
|
||||
IndexOutput termsOut;
|
||||
|
||||
boolean omitTermFreqAndPositions;
|
||||
boolean storePayloads;
|
||||
// Starts a new term
|
||||
long lastFreqStart;
|
||||
long freqStart;
|
||||
long lastProxStart;
|
||||
long proxStart;
|
||||
FieldInfo fieldInfo;
|
||||
int lastPayloadLength;
|
||||
int lastPosition;
|
||||
|
||||
public StandardPostingsWriterImpl(SegmentWriteState state) throws IOException {
|
||||
super();
|
||||
String fileName = IndexFileNames.segmentFileName(state.segmentName, "", StandardCodec.FREQ_EXTENSION);
|
||||
state.flushedFiles.add(fileName);
|
||||
freqOut = state.directory.createOutput(fileName);
|
||||
|
||||
if (state.fieldInfos.hasProx()) {
|
||||
// At least one field does not omit TF, so create the
|
||||
// prox file
|
||||
fileName = IndexFileNames.segmentFileName(state.segmentName, "", StandardCodec.PROX_EXTENSION);
|
||||
state.flushedFiles.add(fileName);
|
||||
proxOut = state.directory.createOutput(fileName);
|
||||
} else {
|
||||
// Every field omits TF so we will write no prox file
|
||||
proxOut = null;
|
||||
}
|
||||
|
||||
totalNumDocs = state.numDocs;
|
||||
|
||||
skipListWriter = new DefaultSkipListWriter(state.skipInterval,
|
||||
state.maxSkipLevels,
|
||||
state.numDocs,
|
||||
freqOut,
|
||||
proxOut);
|
||||
|
||||
skipInterval = state.skipInterval;
|
||||
maxSkipLevels = state.maxSkipLevels;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start(IndexOutput termsOut) throws IOException {
|
||||
this.termsOut = termsOut;
|
||||
CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT);
|
||||
termsOut.writeInt(skipInterval); // write skipInterval
|
||||
termsOut.writeInt(maxSkipLevels); // write maxSkipLevels
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startTerm() {
|
||||
freqStart = freqOut.getFilePointer();
|
||||
if (proxOut != null) {
|
||||
proxStart = proxOut.getFilePointer();
|
||||
// force first payload to write its length
|
||||
lastPayloadLength = -1;
|
||||
}
|
||||
skipListWriter.resetSkip();
|
||||
}
|
||||
|
||||
// Currently, this instance is re-used across fields, so
|
||||
// our parent calls setField whenever the field changes
|
||||
@Override
|
||||
public void setField(FieldInfo fieldInfo) {
|
||||
this.fieldInfo = fieldInfo;
|
||||
omitTermFreqAndPositions = fieldInfo.omitTermFreqAndPositions;
|
||||
storePayloads = fieldInfo.storePayloads;
|
||||
}
|
||||
|
||||
int lastDocID;
|
||||
int df;
|
||||
|
||||
/** 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 {
|
||||
|
||||
final int delta = docID - lastDocID;
|
||||
|
||||
if (docID < 0 || (df > 0 && delta <= 0)) {
|
||||
throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )");
|
||||
}
|
||||
|
||||
if ((++df % skipInterval) == 0) {
|
||||
skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength);
|
||||
skipListWriter.bufferSkip(df);
|
||||
}
|
||||
|
||||
assert docID < totalNumDocs: "docID=" + docID + " totalNumDocs=" + totalNumDocs;
|
||||
|
||||
lastDocID = docID;
|
||||
if (omitTermFreqAndPositions) {
|
||||
freqOut.writeVInt(delta);
|
||||
} else if (1 == termDocFreq) {
|
||||
freqOut.writeVInt((delta<<1) | 1);
|
||||
} else {
|
||||
freqOut.writeVInt(delta<<1);
|
||||
freqOut.writeVInt(termDocFreq);
|
||||
}
|
||||
|
||||
lastPosition = 0;
|
||||
}
|
||||
|
||||
/** Add a new position & payload */
|
||||
@Override
|
||||
public void addPosition(int position, BytesRef payload) throws IOException {
|
||||
assert !omitTermFreqAndPositions: "omitTermFreqAndPositions is true";
|
||||
assert proxOut != null;
|
||||
|
||||
final int delta = position - lastPosition;
|
||||
|
||||
assert delta > 0 || position == 0 || position == -1: "position=" + position + " lastPosition=" + lastPosition; // not quite right (if pos=0 is repeated twice we don't catch it)
|
||||
|
||||
lastPosition = position;
|
||||
|
||||
if (storePayloads) {
|
||||
final int payloadLength = payload == null ? 0 : payload.length;
|
||||
|
||||
if (payloadLength != lastPayloadLength) {
|
||||
lastPayloadLength = payloadLength;
|
||||
proxOut.writeVInt((delta<<1)|1);
|
||||
proxOut.writeVInt(payloadLength);
|
||||
} else {
|
||||
proxOut.writeVInt(delta << 1);
|
||||
}
|
||||
|
||||
if (payloadLength > 0) {
|
||||
proxOut.writeBytes(payload.bytes, payload.offset, payloadLength);
|
||||
}
|
||||
} else {
|
||||
proxOut.writeVInt(delta);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishDoc() {
|
||||
}
|
||||
|
||||
/** Called when we are done adding docs to this term */
|
||||
@Override
|
||||
public void finishTerm(int docCount, boolean isIndexTerm) throws IOException {
|
||||
assert docCount > 0;
|
||||
|
||||
// TODO: wasteful we are counting this (counting # docs
|
||||
// for this term) in two places?
|
||||
assert docCount == df;
|
||||
|
||||
if (isIndexTerm) {
|
||||
// Write absolute at seek points
|
||||
termsOut.writeVLong(freqStart);
|
||||
} else {
|
||||
// Write delta between seek points
|
||||
termsOut.writeVLong(freqStart - lastFreqStart);
|
||||
}
|
||||
|
||||
lastFreqStart = freqStart;
|
||||
|
||||
if (df >= skipInterval) {
|
||||
termsOut.writeVInt((int) (skipListWriter.writeSkip(freqOut)-freqStart));
|
||||
}
|
||||
|
||||
if (!omitTermFreqAndPositions) {
|
||||
if (isIndexTerm) {
|
||||
// Write absolute at seek points
|
||||
termsOut.writeVLong(proxStart);
|
||||
} else {
|
||||
// Write delta between seek points
|
||||
termsOut.writeVLong(proxStart - lastProxStart);
|
||||
}
|
||||
lastProxStart = proxStart;
|
||||
}
|
||||
|
||||
lastDocID = 0;
|
||||
df = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
freqOut.close();
|
||||
} finally {
|
||||
if (proxOut != null) {
|
||||
proxOut.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -497,18 +497,18 @@ public class TestExternalCodecs extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
StandardPostingsWriter docsWriter = new StandardPostingsWriterImpl(state);
|
||||
PostingsWriterBase docsWriter = new StandardPostingsWriter(state);
|
||||
|
||||
// Terms that have <= freqCutoff number of docs are
|
||||
// "pulsed" (inlined):
|
||||
final int freqCutoff = 1;
|
||||
StandardPostingsWriter pulsingWriter = new PulsingPostingsWriterImpl(freqCutoff, docsWriter);
|
||||
PostingsWriterBase pulsingWriter = new PulsingPostingsWriterImpl(freqCutoff, docsWriter);
|
||||
|
||||
// Terms dict index
|
||||
StandardTermsIndexWriter indexWriter;
|
||||
TermsIndexWriterBase indexWriter;
|
||||
boolean success = false;
|
||||
try {
|
||||
indexWriter = new SimpleStandardTermsIndexWriter(state);
|
||||
indexWriter = new FixedGapTermsIndexWriter(state);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -519,7 +519,7 @@ public class TestExternalCodecs extends LuceneTestCase {
|
|||
// Terms dict
|
||||
success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new StandardTermsDictWriter(indexWriter, state, pulsingWriter, reverseUnicodeComparator);
|
||||
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, pulsingWriter, reverseUnicodeComparator);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
|
@ -536,15 +536,15 @@ public class TestExternalCodecs extends LuceneTestCase {
|
|||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
|
||||
StandardPostingsReader docsReader = new StandardPostingsReaderImpl(state.dir, state.segmentInfo, state.readBufferSize);
|
||||
StandardPostingsReader pulsingReader = new PulsingPostingsReaderImpl(docsReader);
|
||||
PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize);
|
||||
PostingsReaderBase pulsingReader = new PulsingPostingsReaderImpl(docsReader);
|
||||
|
||||
// Terms dict index reader
|
||||
StandardTermsIndexReader indexReader;
|
||||
TermsIndexReaderBase indexReader;
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
indexReader = new SimpleStandardTermsIndexReader(state.dir,
|
||||
indexReader = new FixedGapTermsIndexReader(state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
state.termsIndexDivisor,
|
||||
|
@ -559,7 +559,7 @@ public class TestExternalCodecs extends LuceneTestCase {
|
|||
// Terms dict reader
|
||||
success = false;
|
||||
try {
|
||||
FieldsProducer ret = new StandardTermsDictReader(indexReader,
|
||||
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
|
||||
state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
|
@ -582,9 +582,9 @@ public class TestExternalCodecs extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) throws IOException {
|
||||
StandardPostingsReaderImpl.files(dir, segmentInfo, files);
|
||||
StandardTermsDictReader.files(dir, segmentInfo, files);
|
||||
SimpleStandardTermsIndexReader.files(dir, segmentInfo, files);
|
||||
StandardPostingsReader.files(dir, segmentInfo, files);
|
||||
PrefixCodedTermsReader.files(dir, segmentInfo, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -33,14 +33,14 @@ import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
|
|||
import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
|
||||
import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
|
||||
import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
|
||||
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsDictReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsDictWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsIndexWriter;
|
||||
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.TermsIndexReaderBase;
|
||||
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardCodec;
|
||||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -105,12 +105,12 @@ public class MockFixedIntBlockCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
StandardPostingsWriter postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory());
|
||||
PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory());
|
||||
|
||||
boolean success = false;
|
||||
StandardTermsIndexWriter indexWriter;
|
||||
TermsIndexWriterBase indexWriter;
|
||||
try {
|
||||
indexWriter = new SimpleStandardTermsIndexWriter(state);
|
||||
indexWriter = new FixedGapTermsIndexWriter(state);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -120,7 +120,7 @@ public class MockFixedIntBlockCodec extends Codec {
|
|||
|
||||
success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new StandardTermsDictWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
|
@ -136,15 +136,15 @@ public class MockFixedIntBlockCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
StandardPostingsReader postingsReader = new SepPostingsReaderImpl(state.dir,
|
||||
PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
|
||||
state.segmentInfo,
|
||||
state.readBufferSize,
|
||||
new MockIntFactory());
|
||||
|
||||
StandardTermsIndexReader indexReader;
|
||||
TermsIndexReaderBase indexReader;
|
||||
boolean success = false;
|
||||
try {
|
||||
indexReader = new SimpleStandardTermsIndexReader(state.dir,
|
||||
indexReader = new FixedGapTermsIndexReader(state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
state.termsIndexDivisor,
|
||||
|
@ -158,7 +158,7 @@ public class MockFixedIntBlockCodec extends Codec {
|
|||
|
||||
success = false;
|
||||
try {
|
||||
FieldsProducer ret = new StandardTermsDictReader(indexReader,
|
||||
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
|
||||
state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
|
@ -182,14 +182,14 @@ public class MockFixedIntBlockCodec extends Codec {
|
|||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) {
|
||||
SepPostingsReaderImpl.files(segmentInfo, files);
|
||||
StandardTermsDictReader.files(dir, segmentInfo, files);
|
||||
SimpleStandardTermsIndexReader.files(dir, segmentInfo, files);
|
||||
PrefixCodedTermsReader.files(dir, segmentInfo, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Set<String> extensions) {
|
||||
SepPostingsWriterImpl.getExtensions(extensions);
|
||||
StandardTermsDictReader.getExtensions(extensions);
|
||||
SimpleStandardTermsIndexReader.getIndexExtensions(extensions);
|
||||
PrefixCodedTermsReader.getExtensions(extensions);
|
||||
FixedGapTermsIndexReader.getIndexExtensions(extensions);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,14 +33,14 @@ import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
|
|||
import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
|
||||
import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexInput;
|
||||
import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexOutput;
|
||||
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsDictReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsDictWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsIndexWriter;
|
||||
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.TermsIndexReaderBase;
|
||||
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardCodec;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -128,12 +128,12 @@ public class MockVariableIntBlockCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
StandardPostingsWriter postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory());
|
||||
PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory());
|
||||
|
||||
boolean success = false;
|
||||
StandardTermsIndexWriter indexWriter;
|
||||
TermsIndexWriterBase indexWriter;
|
||||
try {
|
||||
indexWriter = new SimpleStandardTermsIndexWriter(state);
|
||||
indexWriter = new FixedGapTermsIndexWriter(state);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -143,7 +143,7 @@ public class MockVariableIntBlockCodec extends Codec {
|
|||
|
||||
success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new StandardTermsDictWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
|
@ -159,15 +159,15 @@ public class MockVariableIntBlockCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
StandardPostingsReader postingsReader = new SepPostingsReaderImpl(state.dir,
|
||||
PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
|
||||
state.segmentInfo,
|
||||
state.readBufferSize,
|
||||
new MockIntFactory());
|
||||
|
||||
StandardTermsIndexReader indexReader;
|
||||
TermsIndexReaderBase indexReader;
|
||||
boolean success = false;
|
||||
try {
|
||||
indexReader = new SimpleStandardTermsIndexReader(state.dir,
|
||||
indexReader = new FixedGapTermsIndexReader(state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
state.termsIndexDivisor,
|
||||
|
@ -181,7 +181,7 @@ public class MockVariableIntBlockCodec extends Codec {
|
|||
|
||||
success = false;
|
||||
try {
|
||||
FieldsProducer ret = new StandardTermsDictReader(indexReader,
|
||||
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
|
||||
state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
|
@ -205,14 +205,14 @@ public class MockVariableIntBlockCodec extends Codec {
|
|||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) {
|
||||
SepPostingsReaderImpl.files(segmentInfo, files);
|
||||
StandardTermsDictReader.files(dir, segmentInfo, files);
|
||||
SimpleStandardTermsIndexReader.files(dir, segmentInfo, files);
|
||||
PrefixCodedTermsReader.files(dir, segmentInfo, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Set<String> extensions) {
|
||||
SepPostingsWriterImpl.getExtensions(extensions);
|
||||
StandardTermsDictReader.getExtensions(extensions);
|
||||
SimpleStandardTermsIndexReader.getIndexExtensions(extensions);
|
||||
PrefixCodedTermsReader.getExtensions(extensions);
|
||||
FixedGapTermsIndexReader.getIndexExtensions(extensions);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,14 +26,14 @@ import org.apache.lucene.index.SegmentReadState;
|
|||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsDictReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsDictWriter;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.standard.StandardTermsIndexWriter;
|
||||
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.TermsIndexReaderBase;
|
||||
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardCodec;
|
||||
import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
|
||||
import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
|
||||
|
@ -55,12 +55,12 @@ public class MockSepCodec extends Codec {
|
|||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
|
||||
StandardPostingsWriter postingsWriter = new SepPostingsWriterImpl(state, new MockSingleIntFactory());
|
||||
PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockSingleIntFactory());
|
||||
|
||||
boolean success = false;
|
||||
StandardTermsIndexWriter indexWriter;
|
||||
TermsIndexWriterBase indexWriter;
|
||||
try {
|
||||
indexWriter = new SimpleStandardTermsIndexWriter(state);
|
||||
indexWriter = new FixedGapTermsIndexWriter(state);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -70,7 +70,7 @@ public class MockSepCodec extends Codec {
|
|||
|
||||
success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new StandardTermsDictWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
|
@ -87,12 +87,12 @@ public class MockSepCodec extends Codec {
|
|||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
|
||||
StandardPostingsReader postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo, state.readBufferSize, new MockSingleIntFactory());
|
||||
PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo, state.readBufferSize, new MockSingleIntFactory());
|
||||
|
||||
StandardTermsIndexReader indexReader;
|
||||
TermsIndexReaderBase indexReader;
|
||||
boolean success = false;
|
||||
try {
|
||||
indexReader = new SimpleStandardTermsIndexReader(state.dir,
|
||||
indexReader = new FixedGapTermsIndexReader(state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
state.termsIndexDivisor,
|
||||
|
@ -106,7 +106,7 @@ public class MockSepCodec extends Codec {
|
|||
|
||||
success = false;
|
||||
try {
|
||||
FieldsProducer ret = new StandardTermsDictReader(indexReader,
|
||||
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
|
||||
state.dir,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
|
@ -130,8 +130,8 @@ public class MockSepCodec extends Codec {
|
|||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) {
|
||||
SepPostingsReaderImpl.files(segmentInfo, files);
|
||||
StandardTermsDictReader.files(dir, segmentInfo, files);
|
||||
SimpleStandardTermsIndexReader.files(dir, segmentInfo, files);
|
||||
PrefixCodedTermsReader.files(dir, segmentInfo, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -141,7 +141,7 @@ public class MockSepCodec extends Codec {
|
|||
|
||||
public static void getSepExtensions(Set<String> extensions) {
|
||||
SepPostingsWriterImpl.getExtensions(extensions);
|
||||
StandardTermsDictReader.getExtensions(extensions);
|
||||
SimpleStandardTermsIndexReader.getIndexExtensions(extensions);
|
||||
PrefixCodedTermsReader.getExtensions(extensions);
|
||||
FixedGapTermsIndexReader.getIndexExtensions(extensions);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue