From 5c816ad8ae6b3412cdc7d3ae33619770fdea5266 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Thu, 17 May 2012 22:28:12 +0000 Subject: [PATCH] LUCENE-4055: remove last redundant SI/FI boolean git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4055@1339895 13f79535-47bb-0310-9956-ffa450edef68 --- .../appending/AppendingPostingsFormat.java | 2 +- .../codecs/lucene3x/Lucene3xFields.java | 18 ++++++++---------- .../lucene3x/Lucene3xSegmentInfosReader.java | 7 +++++-- .../lucene40/Lucene40PostingsBaseFormat.java | 2 +- .../lucene40/Lucene40PostingsFormat.java | 2 +- .../lucene40/Lucene40PostingsReader.java | 11 +++++------ .../lucene40/Lucene40SegmentInfosReader.java | 3 +-- .../lucene40/Lucene40SegmentInfosWriter.java | 1 - .../lucene/codecs/sep/SepPostingsReader.java | 8 +++----- .../SimpleTextSegmentInfosReader.java | 6 +----- .../SimpleTextSegmentInfosWriter.java | 5 ----- .../org/apache/lucene/index/CheckIndex.java | 7 ------- .../index/DocumentsWriterPerThread.java | 2 +- .../org/apache/lucene/index/IndexWriter.java | 5 +---- .../org/apache/lucene/index/SegmentInfo.java | 19 ++----------------- .../org/apache/lucene/index/TestCodecs.java | 1 - .../test/org/apache/lucene/index/TestDoc.java | 2 +- .../lucene/index/TestSegmentMerger.java | 2 +- .../lucene3x/PreFlexRWSegmentInfosWriter.java | 4 +++- .../codecs/lucene40ords/Lucene40WithOrds.java | 2 +- .../mockrandom/MockRandomPostingsFormat.java | 2 +- .../NestedPulsingPostingsFormat.java | 2 +- 22 files changed, 38 insertions(+), 75 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java index b7d77828e1e..da4fc149f2b 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java @@ -60,7 +60,7 @@ class AppendingPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix); + PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix); boolean success = false; try { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java index 52cfb11388c..b19f22007e5 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java @@ -137,16 +137,14 @@ class Lucene3xFields extends FieldsProducer { files.add(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.TERMS_EXTENSION)); files.add(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION)); files.add(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.FREQ_EXTENSION)); - if (info.getHasProx()) { - // LUCENE-1739: for certain versions of 2.9-dev, - // hasProx would be incorrectly computed during - // indexing as true, and then stored into the segments - // file, when it should have been false. So we do the - // extra check, here: - final String prx = IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.PROX_EXTENSION); - if (info.dir.fileExists(prx)) { - files.add(prx); - } + // LUCENE-1739: for certain versions of 2.9-dev, + // hasProx would be incorrectly computed during + // indexing as true, and then stored into the segments + // file, when it should have been false. So we do the + // extra check, here: + final String prx = IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.PROX_EXTENSION); + if (info.dir.fileExists(prx)) { + files.add(prx); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xSegmentInfosReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xSegmentInfosReader.java index b0a55fe354d..bd311ca9d52 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xSegmentInfosReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xSegmentInfosReader.java @@ -132,7 +132,10 @@ class Lucene3xSegmentInfosReader extends SegmentInfosReader { final Codec codec = Codec.forName("Lucene3x"); final Map diagnostics = input.readStringStringMap(); - // nocommit cleane up + // nocommit we can use hasProx/hasVectors from the 3.x + // si... if we can pass this to the other components...? + + // nocommit clean up final boolean hasVectors; if (format <= SegmentInfos.FORMAT_HAS_VECTORS) { hasVectors = input.readByte() == 1; @@ -153,6 +156,6 @@ class Lucene3xSegmentInfosReader extends SegmentInfosReader { return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset, docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile, - delCount, hasProx, codec, diagnostics); + delCount, codec, diagnostics); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java index f9faa267f68..c72b6ac7cd7 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java @@ -42,7 +42,7 @@ public final class Lucene40PostingsBaseFormat extends PostingsBaseFormat { @Override public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException { - return new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix); + return new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java index 59b40c47eec..2a20110af69 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java @@ -307,7 +307,7 @@ public class Lucene40PostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix); + PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix); boolean success = false; try { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java index 808e6593c4f..6919cc0081e 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java @@ -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; @@ -57,11 +58,11 @@ public class Lucene40PostingsReader extends PostingsReaderBase { // private String segment; - public Lucene40PostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException { + public Lucene40PostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException { freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION), ioContext); // this.segment = segmentInfo.name; - if (segmentInfo.getHasProx()) { + if (fieldInfos.hasProx()) { boolean success = false; try { proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION), @@ -79,9 +80,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase { public static void files(SegmentInfo segmentInfo, String segmentSuffix, Collection files) throws IOException { files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION)); - if (segmentInfo.getHasProx()) { - files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION)); - } + files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION)); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosReader.java index 20de4177d1c..935bfb64afe 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosReader.java @@ -82,12 +82,11 @@ public class Lucene40SegmentInfosReader extends SegmentInfosReader { final int delCount = input.readInt(); assert delCount <= docCount; - final boolean hasProx = input.readByte() == 1; final Codec codec = Codec.forName(input.readString()); final Map diagnostics = input.readStringStringMap(); return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset, docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile, - delCount, hasProx, codec, diagnostics); + delCount, codec, diagnostics); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosWriter.java index 4706d8ac0dd..32179b6f4b0 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosWriter.java @@ -95,7 +95,6 @@ public class Lucene40SegmentInfosWriter extends SegmentInfosWriter { output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO)); output.writeInt(si.getDelCount()); - output.writeByte((byte) (si.getHasProx() ? 1 : 0)); output.writeString(si.getCodec().getName()); output.writeStringStringMap(si.getDiagnostics()); } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java index bcca2be3b65..bb3799cc879 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java @@ -75,7 +75,7 @@ public class SepPostingsReader extends PostingsReaderBase { } else { freqIn = null; } - if (segmentInfo.getHasProx()) { + if (fieldInfos.hasProx()) { posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.POS_EXTENSION), context); payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.PAYLOAD_EXTENSION), context); } else { @@ -96,10 +96,8 @@ public class SepPostingsReader extends PostingsReaderBase { files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION)); - if (segmentInfo.getHasProx()) { - files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.POS_EXTENSION)); - files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.PAYLOAD_EXTENSION)); - } + files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.POS_EXTENSION)); + files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.PAYLOAD_EXTENSION)); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosReader.java b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosReader.java index a4621ff2a12..31084a99e24 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosReader.java @@ -102,10 +102,6 @@ public class SimpleTextSegmentInfosReader extends SegmentInfosReader { assert StringHelper.startsWith(scratch, SI_DELCOUNT); final int delCount = Integer.parseInt(readString(SI_DELCOUNT.length, scratch)); - SimpleTextUtil.readLine(input, scratch); - assert StringHelper.startsWith(scratch, SI_HASPROX); - final boolean hasProx = Boolean.parseBoolean(readString(SI_HASPROX.length, scratch)); - SimpleTextUtil.readLine(input, scratch); assert StringHelper.startsWith(scratch, SI_USECOMPOUND); final boolean isCompoundFile = Boolean.parseBoolean(readString(SI_USECOMPOUND.length, scratch)); @@ -164,7 +160,7 @@ public class SimpleTextSegmentInfosReader extends SegmentInfosReader { return new SegmentInfo(directory, version, name, docCount, delGen, dsOffset, dsSegment, dsCompoundFile, normGen, isCompoundFile, - delCount, hasProx, codec, diagnostics); + delCount, codec, diagnostics); } private String readString(int offset, BytesRef scratch) { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosWriter.java index 5a0421495cf..069db2755ee 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosWriter.java @@ -51,7 +51,6 @@ public class SimpleTextSegmentInfosWriter extends SegmentInfosWriter { final static BytesRef SI_VERSION = new BytesRef(" version "); final static BytesRef SI_DOCCOUNT = new BytesRef(" number of documents "); final static BytesRef SI_DELCOUNT = new BytesRef(" number of deletions "); - final static BytesRef SI_HASPROX = new BytesRef(" has prox "); 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 "); @@ -145,10 +144,6 @@ public class SimpleTextSegmentInfosWriter extends SegmentInfosWriter { SimpleTextUtil.write(output, Integer.toString(si.getDelCount()), scratch); SimpleTextUtil.writeNewline(output); - SimpleTextUtil.write(output, SI_HASPROX); - SimpleTextUtil.write(output, si.getHasProx() ? "true" : "false", scratch); - SimpleTextUtil.writeNewline(output); - SimpleTextUtil.write(output, SI_USECOMPOUND); SimpleTextUtil.write(output, Boolean.toString(si.getUseCompoundFile()), scratch); SimpleTextUtil.writeNewline(output); diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java index d1404d85ece..71fdc67990d 100644 --- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java +++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java @@ -185,11 +185,6 @@ public class CheckIndex { /** Number of fields in this segment. */ int numFields; - /** True if at least one of the fields in this segment - * has position data - * @see FieldType#setIndexOptions(org.apache.lucene.index.FieldInfo.IndexOptions) */ - public boolean hasProx; - /** Map that includes certain * debugging details that IndexWriter records into * each segment it creates */ @@ -505,8 +500,6 @@ public class CheckIndex { segInfoStat.codec = codec; msg(" compound=" + info.getUseCompoundFile()); segInfoStat.compound = info.getUseCompoundFile(); - msg(" hasProx=" + info.getHasProx()); - segInfoStat.hasProx = info.getHasProx(); msg(" numFiles=" + info.files().size()); segInfoStat.numFiles = info.files().size(); segInfoStat.sizeMB = info.sizeInBytes()/(1024.*1024.); diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java index dc54395766c..1491b3213c6 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java @@ -477,7 +477,7 @@ class DocumentsWriterPerThread { pendingDeletes.terms.clear(); final SegmentInfo newSegment = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, segment, flushState.numDocs, SegmentInfo.NO, -1, segment, false, null, false, 0, - flushState.fieldInfos.hasProx(), flushState.codec, + flushState.codec, null); if (infoStream.isEnabled("DWPT")) { infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.delCountOnFlush)) + " deleted docs"); diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java index 35a6e4acfb1..0f81ece1660 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -2290,7 +2290,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit { int docCount = mergeState.mergedDocCount; SegmentInfo info = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergedName, docCount, SegmentInfo.NO, -1, mergedName, false, null, false, 0, - mergeState.fieldInfos.hasProx(), codec, null); + codec, null); setDiagnostics(info, "addIndexes(IndexReader...)"); @@ -3434,9 +3434,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit { MergeState mergeState = merger.merge(); mergedDocCount = merge.info.docCount = mergeState.mergedDocCount; - // LUCENE-3403: set hasVectors after merge(), so that it is properly set. - merge.info.setHasProx(mergeState.fieldInfos.hasProx()); - // Record which codec was used to write the segment // nocommit stop doing this once we call non-wimpy diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java index 866887f1ab5..a35d5bcaa57 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java @@ -42,7 +42,6 @@ import org.apache.lucene.util.Constants; * @lucene.experimental */ public final class SegmentInfo implements Cloneable { - // TODO: remove with hasVector and hasProx public static final int CHECK_FIELDINFO = -2; // TODO: remove these from this class, for now this is the representation @@ -87,8 +86,6 @@ public final class SegmentInfo implements Cloneable { private int delCount; // How many deleted docs in this segment - private boolean hasProx; // True if this segment has any fields with positional information - private Codec codec; private Map diagnostics; @@ -134,7 +131,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 normGen, boolean isCompoundFile, - int delCount, boolean hasProx, Codec codec, Map diagnostics) { + int delCount, Codec codec, Map diagnostics) { this.dir = dir; this.version = version; this.name = name; @@ -146,8 +143,6 @@ public final class SegmentInfo implements Cloneable { this.normGen = normGen; this.isCompoundFile = isCompoundFile; this.delCount = delCount; - // nocommit remove these now that we can do regexp instead! - this.hasProx = hasProx; this.codec = codec; this.diagnostics = diagnostics; } @@ -166,16 +161,6 @@ public final class SegmentInfo implements Cloneable { return sizeInBytes; } - // nocommit: ideally codec stores this info privately: - public boolean getHasProx() throws IOException { - return hasProx; - } - - public void setHasProx(boolean hasProx) { - this.hasProx = hasProx; - clearFilesCache(); - } - public boolean hasDeletions() { // Cases: // @@ -221,7 +206,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(diagnostics)); + delCount, codec, new HashMap(diagnostics)); } /** diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java b/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java index ed4c2befb81..d44e9df16eb 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java @@ -313,7 +313,6 @@ public class TestCodecs extends LuceneTestCase { Codec codec = Codec.getDefault(); final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, -1, -1, SEGMENT, false, null, false, 0, - fieldInfos.hasProx(), codec, null); if (VERBOSE) { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDoc.java b/lucene/core/src/test/org/apache/lucene/index/TestDoc.java index cc4b83e04c7..3d6aa162baa 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestDoc.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestDoc.java @@ -202,7 +202,7 @@ public class TestDoc extends LuceneTestCase { r2.close(); 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); + false, null, false, 0, codec, null); if (useCompoundFile) { Collection filesToDelete = IndexWriter.createCompoundFile(dir, merged + ".cfs", MergeState.CheckAbort.NONE, info, newIOContext(random())); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java index 929d76d6e8e..5984271f799 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java @@ -85,7 +85,7 @@ public class TestSegmentMerger extends LuceneTestCase { assertTrue(docsMerged == 2); //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), + false, null, false, 0, codec, null), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random())); assertTrue(mergedReader != null); assertTrue(mergedReader.numDocs() == 2); diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWSegmentInfosWriter.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWSegmentInfosWriter.java index 52ecac4713c..acd7a6bdd49 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWSegmentInfosWriter.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWSegmentInfosWriter.java @@ -93,8 +93,10 @@ class PreFlexRWSegmentInfosWriter extends SegmentInfosWriter { output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO)); output.writeInt(si.getDelCount()); - output.writeByte((byte) (si.getHasProx() ? 1 : 0)); + // hasProx: + output.writeByte((byte) 1); output.writeStringStringMap(si.getDiagnostics()); + // hasVectors: output.writeByte((byte) 1); } diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java index 0accf47ce60..90e791b1cfb 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java @@ -88,7 +88,7 @@ public class Lucene40WithOrds extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix); + PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java index eed2d5b6cf3..2fcfd2c0185 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java @@ -299,7 +299,7 @@ public class MockRandomPostingsFormat extends PostingsFormat { if (LuceneTestCase.VERBOSE) { System.out.println("MockRandomCodec: reading Standard postings"); } - postingsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix); + postingsReader = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix); } if (random.nextBoolean()) { diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java index 37925b808e6..15126582f0e 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java @@ -70,7 +70,7 @@ public class NestedPulsingPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase docsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix); + PostingsReaderBase docsReader = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix); PostingsReaderBase pulsingReaderInner = new PulsingPostingsReader(docsReader); PostingsReaderBase pulsingReader = new PulsingPostingsReader(pulsingReaderInner); boolean success = false;