mirror of https://github.com/apache/lucene.git
small cleanups
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/docvalues@1132418 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
9b1c6a44a2
commit
ff60d29945
|
@ -135,10 +135,9 @@ public class AppendingCodec extends Codec {
|
|||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files)
|
||||
throws IOException {
|
||||
final String codecIdAsString = "" + codecId;
|
||||
StandardPostingsReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
StandardPostingsReader.files(dir, segmentInfo, codecId, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecId, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.lucene.document;
|
|||
import org.apache.lucene.search.PhraseQuery; // for javadocs
|
||||
import org.apache.lucene.search.spans.SpanQuery; // for javadocs
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.index.FieldInvertState;
|
||||
import org.apache.lucene.index.FieldInvertState; // for javadocs
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.util.StringHelper; // for javadocs
|
||||
|
|
|
@ -19,8 +19,6 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.ValuesEnum;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
|
||||
/** Enumerates indexed fields. You must first call {@link
|
||||
|
|
|
@ -207,7 +207,12 @@ public final class IndexFileNames {
|
|||
return segmentName;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/** Sugar for passing "" + name instead */
|
||||
public static String segmentFileName(String segmentName, int name, String ext) {
|
||||
return segmentFileName(segmentName, ""+name, ext);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the given filename ends with the given extension. One
|
||||
* should provide a <i>pure</i> extension, without '.'.
|
||||
|
|
|
@ -1056,9 +1056,10 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
protected abstract void doSetNorm(int doc, String field, byte value)
|
||||
throws CorruptIndexException, IOException;
|
||||
|
||||
/** Flex API: returns {@link Fields} for this reader.
|
||||
* This method may return null if the reader has no
|
||||
* postings.
|
||||
/**
|
||||
* Returns {@link Fields} for this reader.
|
||||
* This method may return null if the reader has no
|
||||
* postings.
|
||||
*
|
||||
* <p><b>NOTE</b>: if this is a multi reader ({@link
|
||||
* #getSequentialSubReaders} is not null) then this
|
||||
|
@ -1071,9 +1072,9 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
public abstract Fields fields() throws IOException;
|
||||
|
||||
/**
|
||||
* Flex API: returns {@link PerDocValues} for this reader.
|
||||
* This method may return null if the reader has no per-document
|
||||
* values stored.
|
||||
* Returns {@link PerDocValues} for this reader.
|
||||
* This method may return null if the reader has no per-document
|
||||
* values stored.
|
||||
*
|
||||
* <p><b>NOTE</b>: if this is a multi reader ({@link
|
||||
* #getSequentialSubReaders} is not null) then this
|
||||
|
|
|
@ -21,12 +21,8 @@ import java.io.IOException;
|
|||
import java.util.Map;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.MultiIndexDocValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.values.MultiIndexDocValues.DocValuesIndex;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.ReaderUtil.Gather; // for javadocs
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -275,6 +271,5 @@ public final class MultiFields extends Fields {
|
|||
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -32,10 +32,10 @@ import org.apache.lucene.util.ReaderUtil;
|
|||
import org.apache.lucene.util.ReaderUtil.Gather;
|
||||
|
||||
/**
|
||||
* Exposes per-document flex API, merged from per-document flex API of
|
||||
* sub-segments. This is useful when you're interacting with an
|
||||
* {@link IndexReader} implementation that consists of sequential sub-readers
|
||||
* (eg DirectoryReader or {@link MultiReader}).
|
||||
* Exposes per-document values, merged from per-document values API of
|
||||
* sub-segments. This is useful when you're interacting with an {@link IndexReader}
|
||||
* implementation that consists of sequential sub-readers (eg DirectoryReader
|
||||
* or {@link MultiReader}).
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: for multi readers, you'll get better performance by gathering
|
||||
|
|
|
@ -67,8 +67,4 @@ public class PerDocWriteState {
|
|||
this.codecId = codecId;
|
||||
this.bytesUsed = state.bytesUsed;
|
||||
}
|
||||
|
||||
public String codecIdAsString() {
|
||||
return "" + codecId;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,7 +26,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
|
|
|
@ -54,8 +54,4 @@ public class SegmentReadState {
|
|||
this.termsIndexDivisor = termsIndexDivisor;
|
||||
this.codecId = codecId;
|
||||
}
|
||||
|
||||
public String codecIdAsString() {
|
||||
return "" + codecId;
|
||||
}
|
||||
}
|
|
@ -79,8 +79,4 @@ public class SegmentWriteState {
|
|||
this.codecId = codecId;
|
||||
segDeletes = state.segDeletes;
|
||||
}
|
||||
|
||||
public String codecIdAsString() {
|
||||
return "" + codecId;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,7 +35,6 @@ import org.apache.lucene.index.TermState;
|
|||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReader; // javadocs
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -116,7 +115,7 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
termsCache = new DoubleBarrelLRUCache<FieldAndTerm,BlockTermState>(termsCacheSize);
|
||||
|
||||
//this.segment = segment;
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, ""+codecId, BlockTermsWriter.TERMS_EXTENSION),
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTermsWriter.TERMS_EXTENSION),
|
||||
readBufferSize);
|
||||
|
||||
boolean success = false;
|
||||
|
@ -197,7 +196,7 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, String id, Collection<String> files) {
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, int id, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, BlockTermsWriter.TERMS_EXTENSION));
|
||||
}
|
||||
|
||||
|
|
|
@ -70,7 +70,7 @@ public class BlockTermsWriter extends FieldsConsumer {
|
|||
public BlockTermsWriter(TermsIndexWriterBase termsIndexWriter,
|
||||
SegmentWriteState state, PostingsWriterBase postingsWriter)
|
||||
throws IOException {
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), TERMS_EXTENSION);
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION);
|
||||
this.termsIndexWriter = termsIndexWriter;
|
||||
out = state.directory.createOutput(termsFileName);
|
||||
boolean success = false;
|
||||
|
|
|
@ -189,8 +189,9 @@ public class CodecProvider {
|
|||
* NOTE: This method will pass any codec from the given codec to
|
||||
* {@link #register(Codec)} and sets fiels codecs via
|
||||
* {@link #setFieldCodec(String, String)}.
|
||||
* @return this
|
||||
*/
|
||||
public void copyFrom(CodecProvider other) {
|
||||
public CodecProvider copyFrom(CodecProvider other) {
|
||||
final Collection<Codec> values = other.codecs.values();
|
||||
for (Codec codec : values) {
|
||||
register(codec);
|
||||
|
@ -200,5 +201,6 @@ public class CodecProvider {
|
|||
setFieldCodec(entry.getKey(), entry.getValue());
|
||||
}
|
||||
setDefaultFieldCodec(other.getDefaultFieldCodec());
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -58,6 +58,7 @@ public class DefaultDocValuesConsumer extends PerDocConsumer {
|
|||
directory, comparator, bytesUsed);
|
||||
}
|
||||
|
||||
@SuppressWarnings("fallthrough")
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, int codecId,
|
||||
Set<String> files) throws IOException {
|
||||
FieldInfos fieldInfos = segmentInfo.getFieldInfos();
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.util.TreeMap;
|
|||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.Bytes;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.Floats;
|
||||
|
|
|
@ -48,11 +48,10 @@ public abstract class FieldsConsumer implements Closeable {
|
|||
mergeState.fieldInfo = mergeState.fieldInfos.fieldInfo(field);
|
||||
assert mergeState.fieldInfo != null : "FieldInfo for field is null: "+ field;
|
||||
TermsEnum terms = fieldsEnum.terms();
|
||||
if(terms != null) {
|
||||
if (terms != null) {
|
||||
final TermsConsumer termsConsumer = addField(mergeState.fieldInfo);
|
||||
termsConsumer.merge(mergeState, terms);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -73,7 +73,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
|
||||
this.termComp = termComp;
|
||||
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, ""+codecId, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
|
||||
boolean success = false;
|
||||
|
||||
|
@ -406,7 +406,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, String id, Collection<String> files) {
|
||||
public static void files(Directory dir, SegmentInfo info, int id, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, id, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
|
|
|
@ -56,7 +56,7 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
|
|||
private final FieldInfos fieldInfos; // unread
|
||||
|
||||
public FixedGapTermsIndexWriter(SegmentWriteState state) throws IOException {
|
||||
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), TERMS_INDEX_EXTENSION);
|
||||
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
|
||||
termIndexInterval = state.termIndexInterval;
|
||||
out = state.directory.createOutput(indexFileName);
|
||||
boolean success = false;
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.lucene.index.values.IndexDocValues;
|
|||
* Abstract API that consumes per document values. Concrete implementations of
|
||||
* this convert field values into a Codec specific format during indexing.
|
||||
* <p>
|
||||
* The {@link PerDocConsumer} API is accessible through flexible indexing / the
|
||||
* The {@link PerDocConsumer} API is accessible through the
|
||||
* {@link Codec} - API providing per field consumers and producers for inverted
|
||||
* data (terms, postings) as well as per-document data.
|
||||
*
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.lucene.index.values.IndexDocValues;
|
|||
* storage on a per-document basis corresponding to their actual
|
||||
* {@link PerDocConsumer} counterpart.
|
||||
* <p>
|
||||
* The {@link PerDocValues} API is accessible through flexible indexing / the
|
||||
* The {@link PerDocValues} API is accessible through the
|
||||
* {@link Codec} - API providing per field consumers and producers for inverted
|
||||
* data (terms, postings) as well as per-document data.
|
||||
*
|
||||
|
|
|
@ -60,7 +60,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, int codecId)
|
||||
throws IOException {
|
||||
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, ""+codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
this.segment = segment;
|
||||
boolean success = false;
|
||||
|
||||
|
@ -236,7 +236,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, String id, Collection<String> files) {
|
||||
public static void files(Directory dir, SegmentInfo info, int id, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, id, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
|
|
|
@ -158,7 +158,7 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
|
|||
// in the extremes.
|
||||
|
||||
public VariableGapTermsIndexWriter(SegmentWriteState state, IndexTermSelector policy) throws IOException {
|
||||
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), TERMS_INDEX_EXTENSION);
|
||||
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
|
||||
out = state.directory.createOutput(indexFileName);
|
||||
boolean success = false;
|
||||
try {
|
||||
|
|
|
@ -154,10 +154,9 @@ public class PulsingCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
|
||||
final String codecId = "" + id;
|
||||
StandardPostingsReader.files(dir, segmentInfo, codecId, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecId, files);
|
||||
VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
|
||||
StandardPostingsReader.files(dir, segmentInfo, id, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, id, files);
|
||||
VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
|
||||
}
|
||||
|
||||
|
|
|
@ -59,19 +59,18 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
|
|||
int skipMinimum;
|
||||
|
||||
public SepPostingsReaderImpl(Directory dir, SegmentInfo segmentInfo, int readBufferSize, IntStreamFactory intFactory, int codecId) throws IOException {
|
||||
final String codecIdAsString = "" + codecId;
|
||||
boolean success = false;
|
||||
try {
|
||||
|
||||
final String docFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecIdAsString, SepPostingsWriterImpl.DOC_EXTENSION);
|
||||
final String docFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.DOC_EXTENSION);
|
||||
docIn = intFactory.openInput(dir, docFileName);
|
||||
|
||||
skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecIdAsString, SepPostingsWriterImpl.SKIP_EXTENSION), readBufferSize);
|
||||
skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.SKIP_EXTENSION), readBufferSize);
|
||||
|
||||
if (segmentInfo.getHasProx()) {
|
||||
freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecIdAsString, SepPostingsWriterImpl.FREQ_EXTENSION));
|
||||
posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecIdAsString, SepPostingsWriterImpl.POS_EXTENSION), readBufferSize);
|
||||
payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecIdAsString, SepPostingsWriterImpl.PAYLOAD_EXTENSION), readBufferSize);
|
||||
freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.FREQ_EXTENSION));
|
||||
posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.POS_EXTENSION), readBufferSize);
|
||||
payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.PAYLOAD_EXTENSION), readBufferSize);
|
||||
} else {
|
||||
posIn = null;
|
||||
payloadIn = null;
|
||||
|
@ -85,7 +84,7 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(SegmentInfo segmentInfo, String codecId, Collection<String> files) throws IOException {
|
||||
public static void files(SegmentInfo segmentInfo, int codecId, Collection<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.DOC_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.SKIP_EXTENSION));
|
||||
|
||||
|
|
|
@ -117,25 +117,25 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
|
|||
try {
|
||||
this.skipInterval = skipInterval;
|
||||
this.skipMinimum = skipInterval; /* set to the same for now */
|
||||
final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), DOC_EXTENSION);
|
||||
final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, DOC_EXTENSION);
|
||||
docOut = factory.createOutput(state.directory, docFileName);
|
||||
docIndex = docOut.index();
|
||||
|
||||
if (state.fieldInfos.hasProx()) {
|
||||
final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), FREQ_EXTENSION);
|
||||
final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, FREQ_EXTENSION);
|
||||
freqOut = factory.createOutput(state.directory, frqFileName);
|
||||
freqIndex = freqOut.index();
|
||||
|
||||
final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), POS_EXTENSION);
|
||||
final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, POS_EXTENSION);
|
||||
posOut = factory.createOutput(state.directory, posFileName);
|
||||
posIndex = posOut.index();
|
||||
|
||||
// TODO: -- only if at least one field stores payloads?
|
||||
final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), PAYLOAD_EXTENSION);
|
||||
final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, PAYLOAD_EXTENSION);
|
||||
payloadOut = state.directory.createOutput(payloadFileName);
|
||||
}
|
||||
|
||||
final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), SKIP_EXTENSION);
|
||||
final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SKIP_EXTENSION);
|
||||
skipOut = state.directory.createOutput(skipFileName);
|
||||
|
||||
totalNumDocs = state.numDocs;
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
|
@ -63,13 +62,13 @@ public class SimpleTextCodec extends Codec {
|
|||
/** Extension of freq postings file */
|
||||
static final String POSTINGS_EXTENSION = "pst";
|
||||
|
||||
static String getPostingsFileName(String segment, String id) {
|
||||
static String getPostingsFileName(String segment, int id) {
|
||||
return IndexFileNames.segmentFileName(segment, id, POSTINGS_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
|
||||
files.add(getPostingsFileName(segmentInfo.name, ""+id));
|
||||
files.add(getPostingsFileName(segmentInfo.name, id));
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
|
||||
}
|
||||
|
||||
|
|
|
@ -58,7 +58,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
final static BytesRef PAYLOAD = SimpleTextFieldsWriter.PAYLOAD;
|
||||
|
||||
public SimpleTextFieldsReader(SegmentReadState state) throws IOException {
|
||||
in = state.dir.openInput(SimpleTextCodec.getPostingsFileName(state.segmentInfo.name, ""+state.codecId));
|
||||
in = state.dir.openInput(SimpleTextCodec.getPostingsFileName(state.segmentInfo.name, state.codecId));
|
||||
|
||||
fieldInfos = state.fieldInfos;
|
||||
}
|
||||
|
|
|
@ -45,7 +45,7 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
|
|||
final static BytesRef PAYLOAD = new BytesRef(" payload ");
|
||||
|
||||
public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException {
|
||||
final String fileName = SimpleTextCodec.getPostingsFileName(state.segmentName, state.codecIdAsString());
|
||||
final String fileName = SimpleTextCodec.getPostingsFileName(state.segmentName, state.codecId);
|
||||
out = state.directory.createOutput(fileName);
|
||||
}
|
||||
|
||||
|
|
|
@ -25,7 +25,6 @@ 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.DocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
|
@ -40,7 +39,6 @@ import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
|
|||
import org.apache.lucene.index.codecs.BlockTermsWriter;
|
||||
import org.apache.lucene.index.codecs.BlockTermsReader;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.index.values.Writer;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
|
@ -139,10 +137,9 @@ public class StandardCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
|
||||
final String codecId = "" + id;
|
||||
StandardPostingsReader.files(dir, segmentInfo, codecId, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecId, files);
|
||||
VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
|
||||
StandardPostingsReader.files(dir, segmentInfo, id, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, id, files);
|
||||
VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
|
||||
}
|
||||
|
||||
|
|
|
@ -52,13 +52,13 @@ public class StandardPostingsReader extends PostingsReaderBase {
|
|||
//private String segment;
|
||||
|
||||
public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize, int codecId) throws IOException {
|
||||
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, ""+codecId, StandardCodec.FREQ_EXTENSION),
|
||||
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.FREQ_EXTENSION),
|
||||
readBufferSize);
|
||||
//this.segment = segmentInfo.name;
|
||||
if (segmentInfo.getHasProx()) {
|
||||
boolean success = false;
|
||||
try {
|
||||
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, ""+codecId, StandardCodec.PROX_EXTENSION),
|
||||
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.PROX_EXTENSION),
|
||||
readBufferSize);
|
||||
success = true;
|
||||
} finally {
|
||||
|
@ -71,7 +71,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, String id, Collection<String> files) throws IOException {
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, int id, Collection<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, StandardCodec.FREQ_EXTENSION));
|
||||
if (segmentInfo.getHasProx()) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, StandardCodec.PROX_EXTENSION));
|
||||
|
|
|
@ -91,14 +91,14 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
|
|||
this.skipInterval = skipInterval;
|
||||
this.skipMinimum = skipInterval; /* set to the same for now */
|
||||
//this.segment = state.segmentName;
|
||||
String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), StandardCodec.FREQ_EXTENSION);
|
||||
String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
|
||||
freqOut = state.directory.createOutput(fileName);
|
||||
boolean success = false;
|
||||
try {
|
||||
if (state.fieldInfos.hasProx()) {
|
||||
// At least one field does not omit TF, so create the
|
||||
// prox file
|
||||
fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), StandardCodec.PROX_EXTENSION);
|
||||
fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
|
||||
proxOut = state.directory.createOutput(fileName);
|
||||
} else {
|
||||
// Every field omits TF so we will write no prox file
|
||||
|
|
|
@ -70,7 +70,7 @@ class FixedDerefBytesImpl {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, BytesRef bytes) throws IOException {
|
||||
public void add(int docID, BytesRef bytes) throws IOException {
|
||||
if (bytes.length == 0) // default value - skip it
|
||||
return;
|
||||
if (size == -1) {
|
||||
|
|
|
@ -78,8 +78,9 @@ class FixedStraightBytesImpl {
|
|||
if (state.bits == null && state.reader instanceof Reader) {
|
||||
Reader reader = (Reader) state.reader;
|
||||
final int maxDocs = reader.maxDoc;
|
||||
if (maxDocs == 0)
|
||||
if (maxDocs == 0) {
|
||||
return;
|
||||
}
|
||||
if (size == -1) {
|
||||
size = reader.size;
|
||||
datOut.writeInt(size);
|
||||
|
@ -124,7 +125,6 @@ class FixedStraightBytesImpl {
|
|||
public long ramBytesUsed() {
|
||||
return oneRecord == null ? 0 : oneRecord.length;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static class Reader extends BytesReaderBase {
|
||||
|
|
|
@ -95,7 +95,6 @@ public class Floats {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
}
|
||||
|
@ -140,7 +139,6 @@ public class Floats {
|
|||
public void files(Collection<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(id, "", Writer.DATA_EXTENSION));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// Writes 4 bytes (float) per value
|
||||
|
|
|
@ -66,10 +66,6 @@ public class MultiIndexDocValues extends IndexDocValues {
|
|||
return new MultiSource(docValuesIdx, starts);
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
}
|
||||
|
||||
public IndexDocValues reset(DocValuesIndex[] docValuesIdx) {
|
||||
int[] start = new int[docValuesIdx.length];
|
||||
for (int i = 0; i < docValuesIdx.length; i++) {
|
||||
|
@ -103,11 +99,6 @@ public class MultiIndexDocValues extends IndexDocValues {
|
|||
public ValueType type() {
|
||||
return emptySoruce.type();
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class MultiValuesEnum extends ValuesEnum {
|
||||
|
|
|
@ -18,11 +18,11 @@ package org.apache.lucene.util;
|
|||
*/
|
||||
|
||||
/**
|
||||
* Represents double[], as a slice (offset + length) into an existing float[].
|
||||
* Represents double[], as a slice (offset + length) into an existing double[].
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public final class FloatsRef implements Cloneable{
|
||||
public final class FloatsRef implements Cloneable {
|
||||
public double[] floats;
|
||||
public int offset;
|
||||
public int length;
|
||||
|
|
|
@ -93,12 +93,12 @@ final class PackedReaderIterator implements PackedInts.ReaderIterator {
|
|||
final long bits = (long) bitsPerValue;
|
||||
final int posToSkip = ord - 1 - position;
|
||||
final long bitsToSkip = (bits * (long)posToSkip);
|
||||
if(bitsToSkip < pendingBitsLeft ){ // enough bits left - no seek required
|
||||
if (bitsToSkip < pendingBitsLeft) { // enough bits left - no seek required
|
||||
pendingBitsLeft -= bitsToSkip;
|
||||
}else {
|
||||
} else {
|
||||
final long skip = bitsToSkip-pendingBitsLeft;
|
||||
final long closestByte = (skip >> 6) << 3;
|
||||
if(closestByte != 0) { // need to seek
|
||||
if (closestByte != 0) { // need to seek
|
||||
final long filePointer = in.getFilePointer();
|
||||
in.seek(filePointer + closestByte);
|
||||
}
|
||||
|
|
|
@ -65,6 +65,14 @@ public class RandomIndexWriter implements Closeable {
|
|||
// called from different threads; else test failures may
|
||||
// not be reproducible from the original seed
|
||||
this.r = new Random(r.nextInt());
|
||||
|
||||
if (r.nextBoolean()) {
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("NOTE: RIW ctor is setting new CodecProvider().copyFrom");
|
||||
}
|
||||
// Just to test CP.copyFrom:
|
||||
conf.setCodecProvider(new CodecProvider().copyFrom(conf.getCodecProvider()));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -34,7 +34,6 @@ 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.DocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
|
||||
|
@ -204,10 +203,9 @@ public class MockFixedIntBlockCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
|
||||
final String codecIdAsString = "" + codecId;
|
||||
SepPostingsReaderImpl.files(segmentInfo, codecIdAsString, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
SepPostingsReaderImpl.files(segmentInfo, codecId, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecId, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files);
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,6 @@ 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.DocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
|
||||
|
@ -227,10 +226,9 @@ public class MockVariableIntBlockCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
|
||||
final String codecIdAsString = "" + codecId;
|
||||
SepPostingsReaderImpl.files(segmentInfo, codecIdAsString, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
SepPostingsReaderImpl.files(segmentInfo, codecId, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecId, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files);
|
||||
}
|
||||
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
import org.apache.lucene.index.codecs.BlockTermsReader;
|
||||
import org.apache.lucene.index.codecs.BlockTermsWriter;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
|
@ -140,7 +139,7 @@ public class MockRandomCodec extends Codec {
|
|||
System.out.println("MockRandomCodec: writing to seg=" + state.segmentName + " seed=" + seed);
|
||||
}
|
||||
|
||||
final String seedFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), SEED_EXT);
|
||||
final String seedFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SEED_EXT);
|
||||
final IndexOutput out = state.directory.createOutput(seedFileName);
|
||||
try {
|
||||
out.writeLong(seed);
|
||||
|
@ -241,7 +240,7 @@ public class MockRandomCodec extends Codec {
|
|||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
|
||||
final String seedFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecIdAsString(), SEED_EXT);
|
||||
final String seedFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, SEED_EXT);
|
||||
final IndexInput in = state.dir.openInput(seedFileName);
|
||||
final long seed = in.readLong();
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
|
@ -348,14 +347,13 @@ public class MockRandomCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
|
||||
final String codecIdAsString = codecId + "";
|
||||
final String seedFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecIdAsString, SEED_EXT);
|
||||
final String seedFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SEED_EXT);
|
||||
files.add(seedFileName);
|
||||
SepPostingsReaderImpl.files(segmentInfo, codecIdAsString, files);
|
||||
StandardPostingsReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
VariableGapTermsIndexReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
SepPostingsReaderImpl.files(segmentInfo, codecId, files);
|
||||
StandardPostingsReader.files(dir, segmentInfo, codecId, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecId, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
|
||||
VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files);
|
||||
// hackish!
|
||||
Iterator<String> it = files.iterator();
|
||||
|
|
|
@ -136,10 +136,9 @@ public class MockSepCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
|
||||
final String codecIdAsString = "" + codecId;
|
||||
SepPostingsReaderImpl.files(segmentInfo, codecIdAsString, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, codecIdAsString, files);
|
||||
SepPostingsReaderImpl.files(segmentInfo, codecId, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecId, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files);
|
||||
}
|
||||
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.lucene.index.codecs.BlockTermsReader;
|
|||
import org.apache.lucene.index.codecs.BlockTermsWriter;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.CoreCodecProvider;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
|
@ -198,9 +197,9 @@ public class TestDocTermOrds extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
|
||||
StandardPostingsReader.files(dir, segmentInfo, ""+id, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, ""+id, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, ""+id, files);
|
||||
StandardPostingsReader.files(dir, segmentInfo, id, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, id, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, id, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue