LUCENE-4055: remove extra booleans

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4055@1339732 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2012-05-17 17:34:49 +00:00
parent 7c9361ff9e
commit e099cb8164
20 changed files with 52 additions and 158 deletions

View File

@ -163,9 +163,8 @@ class Lucene3xSegmentInfosReader extends SegmentInfosReader {
}
}
// nocommit 3x always has norms?
return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset,
docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile,
delCount, hasProx, codec, diagnostics, hasVectors, false, true, true);
delCount, hasProx, codec, diagnostics, hasVectors);
}
}

View File

@ -67,19 +67,16 @@ public class Lucene40DocValuesConsumer extends DocValuesWriterBase {
}
public static void files(SegmentInfo segmentInfo, Set<String> files) throws IOException {
if (segmentInfo.getHasDocValues()) {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
assert segmentInfo.dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
assert segmentInfo.dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION));
}
files.add(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
}
@Override
public void abort() {
try {
close();
} catch (IOException ignored) {}
} catch (IOException ignored) {
}
IOUtils.deleteFilesIgnoringExceptions(mainDirectory, IndexFileNames.segmentFileName(
segmentName, segmentSuffix, IndexFileNames.COMPOUND_FILE_EXTENSION),
IndexFileNames.segmentFileName(segmentName, segmentSuffix,

View File

@ -123,12 +123,10 @@ public class Lucene40NormsFormat extends NormsFormat {
}
public static void files(SegmentInfo segmentInfo, Set<String> files) throws IOException {
if (segmentInfo.getHasNorms()) {
final String normsFileName = IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION);
final String normsEntriesFileName = IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
files.add(normsFileName);
files.add(normsEntriesFileName);
}
final String normsFileName = IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION);
final String normsEntriesFileName = IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
files.add(normsFileName);
files.add(normsEntriesFileName);
}
}

View File

@ -86,13 +86,9 @@ public class Lucene40SegmentInfosReader extends SegmentInfosReader {
final Codec codec = Codec.forName(input.readString());
final Map<String,String> diagnostics = input.readStringStringMap();
final boolean hasVectors = input.readByte() == 1;
final boolean hasDocValues = input.readByte() == 1;
final boolean hasNorms = input.readByte() == 1;
final boolean hasFreq = input.readByte() == 1;
return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset,
docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile,
delCount, hasProx, codec, diagnostics, hasVectors, hasDocValues, hasNorms,
hasFreq);
delCount, hasProx, codec, diagnostics, hasVectors);
}
}

View File

@ -99,9 +99,6 @@ public class Lucene40SegmentInfosWriter extends SegmentInfosWriter {
output.writeString(si.getCodec().getName());
output.writeStringStringMap(si.getDiagnostics());
output.writeByte((byte) (si.getHasVectors() ? 1 : 0));
output.writeByte((byte) (si.getHasDocValues() ? 1 : 0));
output.writeByte((byte) (si.getHasNorms() ? 1 : 0));
output.writeByte((byte) (si.getHasFreq() ? 1 : 0));
}
protected IndexOutput createOutput(Directory dir, String segmentFileName, IOContext context)

View File

@ -66,7 +66,6 @@ public class SepDocValuesConsumer extends DocValuesWriterBase {
@Override
public void abort() {
Set<String> files = new HashSet<String>();
assert false: "sep is broken for now!!";
files(segmentName, files);
IOUtils.deleteFilesIgnoringExceptions(directory, SegmentInfo.findMatchingFiles(directory, files).toArray(new String[0]));
}

View File

@ -24,8 +24,9 @@ import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.TermState;
@ -60,7 +61,7 @@ public class SepPostingsReader extends PostingsReaderBase {
int maxSkipLevels;
int skipMinimum;
public SepPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext context, IntStreamFactory intFactory, String segmentSuffix) throws IOException {
public SepPostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext context, IntStreamFactory intFactory, String segmentSuffix) throws IOException {
boolean success = false;
try {
@ -69,7 +70,7 @@ public class SepPostingsReader extends PostingsReaderBase {
skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.SKIP_EXTENSION), context);
if (segmentInfo.getHasFreq()) {
if (fieldInfos.hasFreq()) {
freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION), context);
} else {
freqIn = null;
@ -93,9 +94,7 @@ public class SepPostingsReader extends PostingsReaderBase {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.DOC_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.SKIP_EXTENSION));
if (segmentInfo.getHasFreq()) {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION));
}
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION));
if (segmentInfo.getHasProx()) {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.POS_EXTENSION));

View File

@ -129,24 +129,22 @@ public class SimpleTextNormsFormat extends NormsFormat {
@Override
public void abort() {
Set<String> files = new HashSet<String>();
filesInternal(state.fieldInfos.hasNorms(), state.segmentName, files, segmentSuffix);
filesInternal(state.segmentName, files, segmentSuffix);
IOUtils.deleteFilesIgnoringExceptions(state.directory,
SegmentInfo.findMatchingFiles(state.directory, files).toArray(new String[0]));
}
public static void files(SegmentInfo segmentInfo, Set<String> files)
throws IOException {
filesInternal(segmentInfo.getHasNorms(), segmentInfo.name, files,
filesInternal(segmentInfo.name, files,
NORMS_SEG_SUFFIX);
}
public static void filesInternal(boolean hasNorms, String segmentName,
public static void filesInternal(String segmentName,
Set<String> files, String segmentSuffix) {
if (hasNorms) {
String id = docValuesIdRegexp(segmentName);
files.add(IndexFileNames.segmentFileName(id, "",
segmentSuffix));
}
String id = docValuesIdRegexp(segmentName);
files.add(IndexFileNames.segmentFileName(id, "",
segmentSuffix));
}
}
}

View File

@ -110,18 +110,6 @@ public class SimpleTextSegmentInfosReader extends SegmentInfosReader {
assert StringHelper.startsWith(scratch, SI_HASVECTORS);
final boolean hasVectors = Boolean.parseBoolean(readString(SI_HASVECTORS.length, scratch));
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_HASDOCVALUES);
final boolean hasDocValues = Boolean.parseBoolean(readString(SI_HASDOCVALUES.length, scratch));
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_HASNORMS);
final boolean hasNorms = Boolean.parseBoolean(readString(SI_HASNORMS.length, scratch));
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_HASFREQS);
final boolean hasFreqs = Boolean.parseBoolean(readString(SI_HASFREQS.length, scratch));
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, SI_USECOMPOUND);
final boolean isCompoundFile = Boolean.parseBoolean(readString(SI_USECOMPOUND.length, scratch));
@ -180,7 +168,7 @@ public class SimpleTextSegmentInfosReader extends SegmentInfosReader {
return new SegmentInfo(directory, version, name, docCount, delGen, dsOffset,
dsSegment, dsCompoundFile, normGen, isCompoundFile,
delCount, hasProx, codec, diagnostics, hasVectors, hasDocValues, hasNorms, hasFreqs);
delCount, hasProx, codec, diagnostics, hasVectors);
}
private String readString(int offset, BytesRef scratch) {

View File

@ -53,9 +53,6 @@ public class SimpleTextSegmentInfosWriter extends SegmentInfosWriter {
final static BytesRef SI_DELCOUNT = new BytesRef(" number of deletions ");
final static BytesRef SI_HASPROX = new BytesRef(" has prox ");
final static BytesRef SI_HASVECTORS = new BytesRef(" has vectors ");
final static BytesRef SI_HASDOCVALUES = new BytesRef(" has doc values ");
final static BytesRef SI_HASNORMS = new BytesRef(" has norms ");
final static BytesRef SI_HASFREQS = new BytesRef(" has freqs ");
final static BytesRef SI_USECOMPOUND = new BytesRef(" uses compound file ");
final static BytesRef SI_DSOFFSET = new BytesRef(" docstore offset ");
final static BytesRef SI_DSSEGMENT = new BytesRef(" docstore segment ");
@ -157,18 +154,6 @@ public class SimpleTextSegmentInfosWriter extends SegmentInfosWriter {
SimpleTextUtil.write(output, si.getHasVectors() ? "true" : "false", scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_HASDOCVALUES);
SimpleTextUtil.write(output, si.getHasDocValues() ? "true" : "false", scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_HASNORMS);
SimpleTextUtil.write(output, si.getHasNorms() ? "true" : "false", scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_HASFREQS);
SimpleTextUtil.write(output, si.getHasFreq() ? "true" : "false", scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_USECOMPOUND);
SimpleTextUtil.write(output, Boolean.toString(si.getUseCompoundFile()), scratch);
SimpleTextUtil.writeNewline(output);

View File

@ -479,18 +479,15 @@ class DocumentsWriterPerThread {
SegmentInfo.NO, -1, segment, false, null, false, 0,
flushState.fieldInfos.hasProx(), flushState.codec,
null,
flushState.fieldInfos.hasVectors(),
flushState.fieldInfos.hasDocValues(),
flushState.fieldInfos.hasNorms(),
flushState.fieldInfos.hasFreq());
flushState.fieldInfos.hasVectors());
if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.delCountOnFlush)) + " deleted docs");
infoStream.message("DWPT", "new segment has " +
(newSegment.getHasVectors() ? "vectors" : "no vectors") + "; " +
(newSegment.getHasNorms() ? "norms" : "no norms") + "; " +
(newSegment.getHasDocValues() ? "docValues" : "no docValues") + "; " +
(newSegment.getHasProx() ? "prox" : "no prox") + "; " +
(newSegment.getHasProx() ? "freqs" : "no freqs"));
(flushState.fieldInfos.hasVectors() ? "vectors" : "no vectors") + "; " +
(flushState.fieldInfos.hasNorms() ? "norms" : "no norms") + "; " +
(flushState.fieldInfos.hasDocValues() ? "docValues" : "no docValues") + "; " +
(flushState.fieldInfos.hasProx() ? "prox" : "no prox") + "; " +
(flushState.fieldInfos.hasFreq() ? "freqs" : "no freqs"));
infoStream.message("DWPT", "flushedFiles=" + newSegment.files());
infoStream.message("DWPT", "flushed codec=" + newSegment.getCodec());
}

View File

@ -2291,10 +2291,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
SegmentInfo info = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergedName, docCount,
SegmentInfo.NO, -1, mergedName, false, null, false, 0,
mergeState.fieldInfos.hasProx(), codec, null,
mergeState.fieldInfos.hasVectors(),
mergeState.fieldInfos.hasDocValues(),
mergeState.fieldInfos.hasNorms(),
mergeState.fieldInfos.hasFreq());
mergeState.fieldInfos.hasVectors());
setDiagnostics(info, "addIndexes(IndexReader...)");
@ -3441,9 +3438,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// LUCENE-3403: set hasVectors after merge(), so that it is properly set.
merge.info.setHasVectors(mergeState.fieldInfos.hasVectors());
merge.info.setHasProx(mergeState.fieldInfos.hasProx());
merge.info.setHasFreq(mergeState.fieldInfos.hasFreq());
merge.info.setHasDocValues(mergeState.fieldInfos.hasDocValues());
merge.info.setHasNorms(mergeState.fieldInfos.hasNorms());
// Record which codec was used to write the segment
@ -3453,11 +3447,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "merge codec=" + codec + " docCount=" + mergedDocCount + "; merged segment has " +
(merge.info.getHasVectors() ? "vectors" : "no vectors") + "; " +
(merge.info.getHasNorms() ? "norms" : "no norms") + "; " +
(merge.info.getHasDocValues() ? "docValues" : "no docValues") + "; " +
(merge.info.getHasProx() ? "prox" : "no prox") + "; " +
(merge.info.getHasProx() ? "freqs" : "no freqs"));
(mergeState.fieldInfos.hasVectors() ? "vectors" : "no vectors") + "; " +
(mergeState.fieldInfos.hasNorms() ? "norms" : "no norms") + "; " +
(mergeState.fieldInfos.hasDocValues() ? "docValues" : "no docValues") + "; " +
(mergeState.fieldInfos.hasProx() ? "prox" : "no prox") + "; " +
(mergeState.fieldInfos.hasProx() ? "freqs" : "no freqs"));
}
// Very important to do this before opening the reader

View File

@ -71,10 +71,6 @@ public final class SegmentInfo implements Cloneable {
private boolean isCompoundFile;
// nocommit should we stop caching the files!?
private volatile List<String> files; // cached list of files that this segment uses
// in the Directory
private volatile long sizeInBytes = -1; // total byte size of all files (computed on demand)
//TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0)
@ -89,10 +85,7 @@ public final class SegmentInfo implements Cloneable {
private int delCount; // How many deleted docs in this segment
private boolean hasVectors; // True if this segment has any term vectors fields
private boolean hasDocValues; // True if this segment has any doc values fields
private boolean hasFreq; // True if this segment has any fields with docFreq information
private boolean hasProx; // True if this segment has any fields with positional information
private boolean hasNorms; // True if this segment has any fields with norms enabled
private Codec codec;
@ -139,8 +132,7 @@ public final class SegmentInfo implements Cloneable {
*/
public SegmentInfo(Directory dir, String version, String name, int docCount, long delGen, int docStoreOffset,
String docStoreSegment, boolean docStoreIsCompoundFile, Map<Integer,Long> normGen, boolean isCompoundFile,
int delCount, boolean hasProx, Codec codec, Map<String,String> diagnostics, boolean hasVectors, boolean hasDocValues,
boolean hasNorms, boolean hasFreq) {
int delCount, boolean hasProx, Codec codec, Map<String,String> diagnostics, boolean hasVectors) {
this.dir = dir;
this.version = version;
this.name = name;
@ -157,33 +149,22 @@ public final class SegmentInfo implements Cloneable {
this.diagnostics = diagnostics;
// nocommit remove these now that we can do regexp instead!
this.hasVectors = hasVectors;
this.hasDocValues = hasDocValues;
this.hasNorms = hasNorms;
this.hasFreq = hasFreq;
}
/**
* Returns total size in bytes of all of files used by this segment
*/
public long sizeInBytes() throws IOException {
long sum = 0;
for (final String fileName : files()) {
sum += dir.fileLength(fileName);
if (sizeInBytes == -1) {
long sum = 0;
for (final String fileName : files()) {
sum += dir.fileLength(fileName);
}
sizeInBytes = sum;
}
sizeInBytes = sum;
return sizeInBytes;
}
// nocommit: ideally codec stores this info privately:
public boolean getHasFreq() throws IOException {
return hasFreq;
}
public void setHasFreq(boolean hasFreq) {
this.hasFreq = hasFreq;
clearFilesCache();
}
// nocommit: ideally codec stores this info privately:
public boolean getHasProx() throws IOException {
return hasProx;
@ -204,26 +185,6 @@ public final class SegmentInfo implements Cloneable {
clearFilesCache();
}
// nocommit: ideally codec stores this info privately:
public boolean getHasDocValues() throws IOException {
return hasDocValues;
}
public void setHasDocValues(boolean hasDocValues) {
this.hasDocValues = hasDocValues;
clearFilesCache();
}
// nocommit: ideally codec stores this info privately:
public boolean getHasNorms() throws IOException {
return hasNorms;
}
public void setHasNorms(boolean hasNorms) {
this.hasNorms = hasNorms;
clearFilesCache();
}
public boolean hasDeletions() {
// Cases:
//
@ -270,7 +231,7 @@ public final class SegmentInfo implements Cloneable {
return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset,
docStoreSegment, docStoreIsCompoundFile, clonedNormGen, isCompoundFile,
delCount, hasProx, codec, new HashMap<String,String>(diagnostics),
hasVectors, hasDocValues, hasNorms, hasFreq);
hasVectors);
}
/**
@ -411,21 +372,14 @@ public final class SegmentInfo implements Cloneable {
*/
public List<String> files() throws IOException {
if (files == null) {
// nocommit maybe don't cache...?
// Cache
final Set<String> fileSet = new HashSet<String>();
codec.files(this, fileSet);
files = findMatchingFiles(dir, fileSet);
}
return files;
final Set<String> fileSet = new HashSet<String>();
codec.files(this, fileSet);
return findMatchingFiles(dir, fileSet);
}
/* Called whenever any change is made that affects which
* files this segment has. */
private void clearFilesCache() {
files = null;
sizeInBytes = -1;
}

View File

@ -314,10 +314,7 @@ public class TestCodecs extends LuceneTestCase {
final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, -1, -1,
SEGMENT, false, null, false, 0,
fieldInfos.hasProx(),
codec, null, fieldInfos.hasVectors(),
fieldInfos.hasDocValues(),
fieldInfos.hasNorms(),
fieldInfos.hasFreq());
codec, null, fieldInfos.hasVectors());
if (VERBOSE) {
System.out.println("TEST: now read postings");

View File

@ -203,10 +203,7 @@ public class TestDoc extends LuceneTestCase {
final SegmentInfo info = new SegmentInfo(si1.dir, Constants.LUCENE_MAIN_VERSION, merged,
si1.docCount + si2.docCount, -1, -1, merged,
false, null, false, 0, mergeState.fieldInfos.hasProx(), codec, null,
mergeState.fieldInfos.hasVectors(),
mergeState.fieldInfos.hasDocValues(),
mergeState.fieldInfos.hasNorms(),
mergeState.fieldInfos.hasFreq());
mergeState.fieldInfos.hasVectors());
if (useCompoundFile) {
Collection<String> filesToDelete = IndexWriter.createCompoundFile(dir, merged + ".cfs", MergeState.CheckAbort.NONE, info, newIOContext(random()));

View File

@ -86,10 +86,7 @@ public class TestSegmentMerger extends LuceneTestCase {
//Should be able to open a new SegmentReader against the new directory
SegmentReader mergedReader = new SegmentReader(new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, docsMerged, -1, -1, mergedSegment,
false, null, false, 0, mergeState.fieldInfos.hasProx(), codec, null,
mergeState.fieldInfos.hasVectors(),
mergeState.fieldInfos.hasDocValues(),
mergeState.fieldInfos.hasNorms(),
mergeState.fieldInfos.hasFreq()),
mergeState.fieldInfos.hasVectors()),
DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
assertTrue(mergedReader != null);
assertTrue(mergedReader.numDocs() == 2);

View File

@ -156,6 +156,7 @@ public class MockFixedIntBlockPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
state.fieldInfos,
state.segmentInfo,
state.context,
new MockIntFactory(blockSize), state.segmentSuffix);

View File

@ -179,6 +179,7 @@ public class MockVariableIntBlockPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
state.fieldInfos,
state.segmentInfo,
state.context,
new MockIntFactory(baseBlockSize), state.segmentSuffix);

View File

@ -293,7 +293,7 @@ public class MockRandomPostingsFormat extends PostingsFormat {
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading Sep postings");
}
postingsReader = new SepPostingsReader(state.dir, state.segmentInfo,
postingsReader = new SepPostingsReader(state.dir, state.fieldInfos, state.segmentInfo,
state.context, new MockIntStreamFactory(random), state.segmentSuffix);
} else {
if (LuceneTestCase.VERBOSE) {

View File

@ -85,7 +85,7 @@ public class MockSepPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new SepPostingsReader(state.dir, state.segmentInfo,
PostingsReaderBase postingsReader = new SepPostingsReader(state.dir, state.fieldInfos, state.segmentInfo,
state.context, new MockSingleIntFactory(), state.segmentSuffix);
TermsIndexReaderBase indexReader;