mirror of https://github.com/apache/lucene.git
Merge with trunk.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene6238@1663427 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
332c478c62
|
@ -195,6 +195,11 @@ API Changes
|
|||
* LUCENE-6317: Deprecate DataOutput.writeStringSet and writeStringStringMap.
|
||||
Use writeSetOfStrings/Maps instead. (Mike McCandless, Robert Muir)
|
||||
|
||||
* LUCENE-6307: Rename SegmentInfo.getDocCount -> .maxDoc,
|
||||
SegmentInfos.totalDocCount -> .totalMaxDoc, MergeInfo.totalDocCount
|
||||
-> .totalMaxDoc and MergePolicy.OneMerge.totalDocCount ->
|
||||
.totalMaxDoc (Adrien Grand, Robert Muir, Mike McCandless)
|
||||
|
||||
Other
|
||||
|
||||
* LUCENE-6248: Remove unused odd constants from StandardSyntaxParser.jj
|
||||
|
|
|
@ -144,8 +144,8 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
final long sumDocFreq = in.readVLong();
|
||||
final int docCount = in.readVInt();
|
||||
final int longsSize = in.readVInt();
|
||||
if (docCount < 0 || docCount > state.segmentInfo.getDocCount()) { // #docs with field must be <= #docs
|
||||
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.getDocCount(), in);
|
||||
if (docCount < 0 || docCount > state.segmentInfo.maxDoc()) { // #docs with field must be <= #docs
|
||||
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.maxDoc(), in);
|
||||
}
|
||||
if (sumDocFreq < docCount) { // #postings must be >= #docs with field
|
||||
throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount, in);
|
||||
|
|
|
@ -105,7 +105,7 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
|
|||
throws IOException {
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
|
||||
this.termsIndexWriter = termsIndexWriter;
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
out = state.directory.createOutput(termsFileName, state.context);
|
||||
boolean success = false;
|
||||
try {
|
||||
|
|
|
@ -134,8 +134,8 @@ public final class OrdsBlockTreeTermsReader extends FieldsProducer {
|
|||
|
||||
BytesRef minTerm = readBytesRef(in);
|
||||
BytesRef maxTerm = readBytesRef(in);
|
||||
if (docCount < 0 || docCount > state.segmentInfo.getDocCount()) { // #docs with field must be <= #docs
|
||||
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.getDocCount(), in);
|
||||
if (docCount < 0 || docCount > state.segmentInfo.maxDoc()) { // #docs with field must be <= #docs
|
||||
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.maxDoc(), in);
|
||||
}
|
||||
if (sumDocFreq < docCount) { // #postings must be >= #docs with field
|
||||
throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount, in);
|
||||
|
|
|
@ -182,7 +182,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
|
|||
{
|
||||
BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock);
|
||||
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
|
||||
out = state.directory.createOutput(termsFileName, state.context);
|
||||
|
|
|
@ -28,7 +28,7 @@ public class DefaultBloomFilterFactory extends BloomFilterFactory {
|
|||
@Override
|
||||
public FuzzySet getSetForField(SegmentWriteState state,FieldInfo info) {
|
||||
//Assume all of the docs have a unique term (e.g. a primary key) and we hope to maintain a set with 10% of bits set
|
||||
return FuzzySet.createSetBasedOnQuality(state.segmentInfo.getDocCount(), 0.10f);
|
||||
return FuzzySet.createSetBasedOnQuality(state.segmentInfo.maxDoc(), 0.10f);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -47,7 +47,7 @@ class DirectDocValuesConsumer extends DocValuesConsumer {
|
|||
final int maxDoc;
|
||||
|
||||
DirectDocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
boolean success = false;
|
||||
try {
|
||||
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||
|
|
|
@ -115,7 +115,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
DirectDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
merging = false;
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
// read in the entries from the metadata file.
|
||||
|
|
|
@ -139,8 +139,8 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
}
|
||||
private void checkFieldSummary(SegmentInfo info, IndexInput indexIn, IndexInput blockIn, TermsReader field, TermsReader previous) throws IOException {
|
||||
// #docs with field must be <= #docs
|
||||
if (field.docCount < 0 || field.docCount > info.getDocCount()) {
|
||||
throw new CorruptIndexException("invalid docCount: " + field.docCount + " maxDoc: " + info.getDocCount() + " (blockIn=" + blockIn + ")", indexIn);
|
||||
if (field.docCount < 0 || field.docCount > info.maxDoc()) {
|
||||
throw new CorruptIndexException("invalid docCount: " + field.docCount + " maxDoc: " + info.maxDoc() + " (blockIn=" + blockIn + ")", indexIn);
|
||||
}
|
||||
// #postings must be >= #docs with field
|
||||
if (field.sumDocFreq < field.docCount) {
|
||||
|
|
|
@ -166,7 +166,7 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
|
|||
|
||||
this.postingsWriter = postingsWriter;
|
||||
this.fieldInfos = state.fieldInfos;
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
this.maxDoc = state.segmentInfo.maxDoc();
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
|
|
|
@ -119,8 +119,8 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
}
|
||||
private void checkFieldSummary(SegmentInfo info, IndexInput in, TermsReader field, TermsReader previous) throws IOException {
|
||||
// #docs with field must be <= #docs
|
||||
if (field.docCount < 0 || field.docCount > info.getDocCount()) {
|
||||
throw new CorruptIndexException("invalid docCount: " + field.docCount + " maxDoc: " + info.getDocCount(), in);
|
||||
if (field.docCount < 0 || field.docCount > info.maxDoc()) {
|
||||
throw new CorruptIndexException("invalid docCount: " + field.docCount + " maxDoc: " + info.maxDoc(), in);
|
||||
}
|
||||
// #postings must be >= #docs with field
|
||||
if (field.sumDocFreq < field.docCount) {
|
||||
|
|
|
@ -137,7 +137,7 @@ public class FSTTermsWriter extends FieldsConsumer {
|
|||
this.postingsWriter = postingsWriter;
|
||||
this.fieldInfos = state.fieldInfos;
|
||||
this.out = state.directory.createOutput(termsFileName, state.context);
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
this.maxDoc = state.segmentInfo.maxDoc();
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
|
|
|
@ -69,7 +69,7 @@ class MemoryDocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
MemoryDocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension, float acceptableOverheadRatio) throws IOException {
|
||||
this.acceptableOverheadRatio = acceptableOverheadRatio;
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
boolean success = false;
|
||||
try {
|
||||
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||
|
|
|
@ -139,7 +139,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
MemoryDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
merging = false;
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
// read in the entries from the metadata file.
|
||||
|
|
|
@ -313,7 +313,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
|
|||
TermsWriter termsWriter = new TermsWriter(out, fieldInfo,
|
||||
doPackFST, acceptableOverheadRatio);
|
||||
|
||||
FixedBitSet docsSeen = new FixedBitSet(state.segmentInfo.getDocCount());
|
||||
FixedBitSet docsSeen = new FixedBitSet(state.segmentInfo.maxDoc());
|
||||
long sumTotalTermFreq = 0;
|
||||
long sumDocFreq = 0;
|
||||
PostingsEnum postingsEnum = null;
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.lucene.index.SortedSetDocValues;
|
|||
import org.apache.lucene.store.BufferedChecksumIndexInput;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
|
@ -86,7 +85,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
public SimpleTextDocValuesReader(SegmentReadState state, String ext) throws IOException {
|
||||
// System.out.println("dir=" + state.directory + " seg=" + state.segmentInfo.name + " file=" + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext));
|
||||
data = state.directory.openInput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext), state.context);
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
while(true) {
|
||||
readLine();
|
||||
//System.out.println("READ field=" + scratch.utf8ToString());
|
||||
|
|
|
@ -56,9 +56,9 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
private final Set<String> fieldsSeen = new HashSet<>(); // for asserting
|
||||
|
||||
public SimpleTextDocValuesWriter(SegmentWriteState state, String ext) throws IOException {
|
||||
// System.out.println("WRITE: " + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext) + " " + state.segmentInfo.getDocCount() + " docs");
|
||||
// System.out.println("WRITE: " + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext) + " " + state.segmentInfo.maxDoc() + " docs");
|
||||
data = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext), state.context);
|
||||
numDocs = state.segmentInfo.getDocCount();
|
||||
numDocs = state.segmentInfo.maxDoc();
|
||||
}
|
||||
|
||||
// for asserting
|
||||
|
|
|
@ -79,7 +79,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
private final int maxDoc;
|
||||
|
||||
public SimpleTextFieldsReader(SegmentReadState state) throws IOException {
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
this.maxDoc = state.segmentInfo.maxDoc();
|
||||
fieldInfos = state.fieldInfos;
|
||||
in = state.directory.openInput(SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.segmentSuffix), state.context);
|
||||
boolean success = false;
|
||||
|
|
|
@ -164,7 +164,7 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
|
|||
SimpleTextUtil.writeNewline(output);
|
||||
|
||||
SimpleTextUtil.write(output, SI_DOCCOUNT);
|
||||
SimpleTextUtil.write(output, Integer.toString(si.getDocCount()), scratch);
|
||||
SimpleTextUtil.write(output, Integer.toString(si.maxDoc()), scratch);
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
|
||||
SimpleTextUtil.write(output, SI_USECOMPOUND);
|
||||
|
|
|
@ -75,7 +75,7 @@ public class SimpleTextStoredFieldsReader extends StoredFieldsReader {
|
|||
} catch (Throwable t) {} // ensure we throw our original exception
|
||||
}
|
||||
}
|
||||
readIndex(si.getDocCount());
|
||||
readIndex(si.maxDoc());
|
||||
}
|
||||
|
||||
// used by clone
|
||||
|
|
|
@ -79,7 +79,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
|
|||
} catch (Throwable t) {} // ensure we throw our original exception
|
||||
}
|
||||
}
|
||||
readIndex(si.getDocCount());
|
||||
readIndex(si.maxDoc());
|
||||
}
|
||||
|
||||
// used by clone
|
||||
|
|
|
@ -180,8 +180,8 @@ public final class BlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
BytesRef minTerm = readBytesRef(termsIn);
|
||||
BytesRef maxTerm = readBytesRef(termsIn);
|
||||
if (docCount < 0 || docCount > state.segmentInfo.getDocCount()) { // #docs with field must be <= #docs
|
||||
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.getDocCount(), termsIn);
|
||||
if (docCount < 0 || docCount > state.segmentInfo.maxDoc()) { // #docs with field must be <= #docs
|
||||
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.maxDoc(), termsIn);
|
||||
}
|
||||
if (sumDocFreq < docCount) { // #postings must be >= #docs with field
|
||||
throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount, termsIn);
|
||||
|
|
|
@ -257,7 +257,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
|
|||
{
|
||||
validateSettings(minItemsInBlock, maxItemsInBlock);
|
||||
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
this.maxDoc = state.segmentInfo.maxDoc();
|
||||
this.fieldInfos = state.fieldInfos;
|
||||
this.minItemsInBlock = minItemsInBlock;
|
||||
this.maxItemsInBlock = maxItemsInBlock;
|
||||
|
|
|
@ -54,7 +54,7 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable, Acco
|
|||
// It is the responsibility of the caller to close fieldsIndexIn after this constructor
|
||||
// has been called
|
||||
CompressingStoredFieldsIndexReader(IndexInput fieldsIndexIn, SegmentInfo si) throws IOException {
|
||||
maxDoc = si.getDocCount();
|
||||
maxDoc = si.maxDoc();
|
||||
int[] docBases = new int[16];
|
||||
long[] startPointers = new long[16];
|
||||
int[] avgChunkDocs = new int[16];
|
||||
|
|
|
@ -119,7 +119,7 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
|
|||
final String segment = si.name;
|
||||
boolean success = false;
|
||||
fieldInfos = fn;
|
||||
numDocs = si.getDocCount();
|
||||
numDocs = si.maxDoc();
|
||||
|
||||
int version = -1;
|
||||
long maxPointer = -1;
|
||||
|
|
|
@ -110,7 +110,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
|
|||
final String segment = si.name;
|
||||
boolean success = false;
|
||||
fieldInfos = fn;
|
||||
numDocs = si.getDocCount();
|
||||
numDocs = si.maxDoc();
|
||||
int version = -1;
|
||||
CompressingStoredFieldsIndexReader indexReader = null;
|
||||
|
||||
|
|
|
@ -105,7 +105,7 @@ class Lucene50DocValuesConsumer extends DocValuesConsumer implements Closeable {
|
|||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
meta = state.directory.createOutput(metaName, state.context);
|
||||
CodecUtil.writeIndexHeader(meta, metaCodec, Lucene50DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
|
|
@ -120,7 +120,7 @@ class Lucene50DocValuesProducer extends DocValuesProducer implements Closeable {
|
|||
/** expert: instantiates a new reader */
|
||||
Lucene50DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
this.maxDoc = state.segmentInfo.maxDoc();
|
||||
merging = false;
|
||||
|
||||
int version = -1;
|
||||
|
|
|
@ -79,7 +79,7 @@ public final class Lucene50LiveDocsFormat extends LiveDocsFormat {
|
|||
public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
|
||||
long gen = info.getDelGen();
|
||||
String name = IndexFileNames.fileNameFromGeneration(info.info.name, EXTENSION, gen);
|
||||
final int length = info.info.getDocCount();
|
||||
final int length = info.info.maxDoc();
|
||||
try (ChecksumIndexInput input = dir.openChecksumInput(name, context)) {
|
||||
Throwable priorE = null;
|
||||
try {
|
||||
|
|
|
@ -87,7 +87,7 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
|
||||
Lucene50NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
merging = false;
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
|
||||
int version = -1;
|
||||
|
|
|
@ -160,7 +160,7 @@ public final class Lucene50PostingsWriter extends PushPostingsWriterBase {
|
|||
// TODO: should we try skipping every 2/4 blocks...?
|
||||
skipWriter = new Lucene50SkipWriter(MAX_SKIP_LEVELS,
|
||||
BLOCK_SIZE,
|
||||
state.segmentInfo.getDocCount(),
|
||||
state.segmentInfo.maxDoc(),
|
||||
docOut,
|
||||
posOut,
|
||||
payOut);
|
||||
|
|
|
@ -140,7 +140,7 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
|
|||
output.writeInt(version.minor);
|
||||
output.writeInt(version.bugfix);
|
||||
assert version.prerelease == 0;
|
||||
output.writeInt(si.getDocCount());
|
||||
output.writeInt(si.maxDoc());
|
||||
|
||||
output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
|
||||
output.writeMapOfStrings(si.getDiagnostics());
|
||||
|
|
|
@ -111,7 +111,7 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
|
||||
final int maxDoc = state.segmentInfo.getDocCount();
|
||||
final int maxDoc = state.segmentInfo.maxDoc();
|
||||
bytes.freeze(false);
|
||||
final PackedLongValues lengths = this.lengths.build();
|
||||
dvConsumer.addBinaryField(fieldInfo,
|
||||
|
|
|
@ -445,8 +445,8 @@ class BufferedUpdatesStream implements Accountable {
|
|||
totDelCount += segState.rld.getPendingDeleteCount() - segState.startDelCount;
|
||||
segState.reader.getSegmentInfo().setBufferedDeletesGen(gen);
|
||||
int fullDelCount = segState.rld.info.getDelCount() + segState.rld.getPendingDeleteCount();
|
||||
assert fullDelCount <= segState.rld.info.info.getDocCount();
|
||||
if (fullDelCount == segState.rld.info.info.getDocCount()) {
|
||||
assert fullDelCount <= segState.rld.info.info.maxDoc();
|
||||
if (fullDelCount == segState.rld.info.info.maxDoc()) {
|
||||
if (allDeleted == null) {
|
||||
allDeleted = new ArrayList<>();
|
||||
}
|
||||
|
|
|
@ -156,7 +156,7 @@ public class CheckIndex implements Closeable {
|
|||
public Codec codec;
|
||||
|
||||
/** Document count (does not take deletions into account). */
|
||||
public int docCount;
|
||||
public int maxDoc;
|
||||
|
||||
/** True if segment is compound file format. */
|
||||
public boolean compound;
|
||||
|
@ -586,16 +586,16 @@ public class CheckIndex implements Closeable {
|
|||
}
|
||||
Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
|
||||
result.segmentInfos.add(segInfoStat);
|
||||
msg(infoStream, " " + (1+i) + " of " + numSegments + ": name=" + info.info.name + " docCount=" + info.info.getDocCount());
|
||||
msg(infoStream, " " + (1+i) + " of " + numSegments + ": name=" + info.info.name + " maxDoc=" + info.info.maxDoc());
|
||||
segInfoStat.name = info.info.name;
|
||||
segInfoStat.docCount = info.info.getDocCount();
|
||||
segInfoStat.maxDoc = info.info.maxDoc();
|
||||
|
||||
final Version version = info.info.getVersion();
|
||||
if (info.info.getDocCount() <= 0) {
|
||||
throw new RuntimeException("illegal number of documents: maxDoc=" + info.info.getDocCount());
|
||||
if (info.info.maxDoc() <= 0) {
|
||||
throw new RuntimeException("illegal number of documents: maxDoc=" + info.info.maxDoc());
|
||||
}
|
||||
|
||||
int toLoseDocCount = info.info.getDocCount();
|
||||
int toLoseDocCount = info.info.maxDoc();
|
||||
|
||||
SegmentReader reader = null;
|
||||
|
||||
|
@ -620,8 +620,7 @@ public class CheckIndex implements Closeable {
|
|||
if (!info.hasDeletions()) {
|
||||
msg(infoStream, " no deletions");
|
||||
segInfoStat.hasDeletions = false;
|
||||
}
|
||||
else{
|
||||
} else {
|
||||
msg(infoStream, " has deletions [delGen=" + info.getDelGen() + "]");
|
||||
segInfoStat.hasDeletions = true;
|
||||
segInfoStat.deletionsGen = info.getDelGen();
|
||||
|
@ -641,26 +640,26 @@ public class CheckIndex implements Closeable {
|
|||
reader.checkIntegrity();
|
||||
msg(infoStream, String.format(Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime()-startIntegrityNS)));
|
||||
|
||||
if (reader.maxDoc() != info.info.getDocCount()) {
|
||||
throw new RuntimeException("SegmentReader.maxDoc() " + reader.maxDoc() + " != SegmentInfos.docCount " + info.info.getDocCount());
|
||||
if (reader.maxDoc() != info.info.maxDoc()) {
|
||||
throw new RuntimeException("SegmentReader.maxDoc() " + reader.maxDoc() + " != SegmentInfo.maxDoc " + info.info.maxDoc());
|
||||
}
|
||||
|
||||
final int numDocs = reader.numDocs();
|
||||
toLoseDocCount = numDocs;
|
||||
|
||||
if (reader.hasDeletions()) {
|
||||
if (reader.numDocs() != info.info.getDocCount() - info.getDelCount()) {
|
||||
throw new RuntimeException("delete count mismatch: info=" + (info.info.getDocCount() - info.getDelCount()) + " vs reader=" + reader.numDocs());
|
||||
if (reader.numDocs() != info.info.maxDoc() - info.getDelCount()) {
|
||||
throw new RuntimeException("delete count mismatch: info=" + (info.info.maxDoc() - info.getDelCount()) + " vs reader=" + reader.numDocs());
|
||||
}
|
||||
if ((info.info.getDocCount() - reader.numDocs()) > reader.maxDoc()) {
|
||||
throw new RuntimeException("too many deleted docs: maxDoc()=" + reader.maxDoc() + " vs del count=" + (info.info.getDocCount() - reader.numDocs()));
|
||||
if ((info.info.maxDoc() - reader.numDocs()) > reader.maxDoc()) {
|
||||
throw new RuntimeException("too many deleted docs: maxDoc()=" + reader.maxDoc() + " vs del count=" + (info.info.maxDoc() - reader.numDocs()));
|
||||
}
|
||||
if (info.info.getDocCount() - reader.numDocs() != info.getDelCount()) {
|
||||
throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - reader.numDocs()));
|
||||
if (info.info.maxDoc() - reader.numDocs() != info.getDelCount()) {
|
||||
throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.maxDoc() - reader.numDocs()));
|
||||
}
|
||||
} else {
|
||||
if (info.getDelCount() != 0) {
|
||||
throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - reader.numDocs()));
|
||||
throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.maxDoc() - reader.numDocs()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -88,14 +88,14 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
// NOTE: caller (DocumentsWriterPerThread) handles
|
||||
// aborting on any exception from this method
|
||||
|
||||
int numDocs = state.segmentInfo.getDocCount();
|
||||
int maxDoc = state.segmentInfo.maxDoc();
|
||||
writeNorms(state);
|
||||
writeDocValues(state);
|
||||
|
||||
// it's possible all docs hit non-aborting exceptions...
|
||||
initStoredFieldsWriter();
|
||||
fillStoredFields(numDocs);
|
||||
storedFieldsWriter.finish(state.fieldInfos, numDocs);
|
||||
fillStoredFields(maxDoc);
|
||||
storedFieldsWriter.finish(state.fieldInfos, maxDoc);
|
||||
storedFieldsWriter.close();
|
||||
|
||||
Map<String,TermsHashPerField> fieldsToFlush = new HashMap<>();
|
||||
|
@ -120,7 +120,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
|
||||
/** Writes all buffered doc values (called from {@link #flush}). */
|
||||
private void writeDocValues(SegmentWriteState state) throws IOException {
|
||||
int docCount = state.segmentInfo.getDocCount();
|
||||
int maxDoc = state.segmentInfo.maxDoc();
|
||||
DocValuesConsumer dvConsumer = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
|
@ -138,7 +138,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
dvConsumer = fmt.fieldsConsumer(state);
|
||||
}
|
||||
|
||||
perField.docValuesWriter.finish(docCount);
|
||||
perField.docValuesWriter.finish(maxDoc);
|
||||
perField.docValuesWriter.flush(state, dvConsumer);
|
||||
perField.docValuesWriter = null;
|
||||
} else if (perField.fieldInfo.getDocValuesType() != DocValuesType.NONE) {
|
||||
|
@ -200,7 +200,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
// changed for this field since the first time we added it.
|
||||
if (fi.omitsNorms() == false && fi.getIndexOptions() != IndexOptions.NONE) {
|
||||
assert perField.norms != null: "field=" + fi.name;
|
||||
perField.norms.finish(state.segmentInfo.getDocCount());
|
||||
perField.norms.finish(state.segmentInfo.maxDoc());
|
||||
perField.norms.flush(state, normsConsumer);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -388,7 +388,7 @@ class DocumentsWriterPerThread {
|
|||
FlushedSegment flush() throws IOException, AbortingException {
|
||||
assert numDocsInRAM > 0;
|
||||
assert deleteSlice.isEmpty() : "all deletes must be applied in prepareFlush";
|
||||
segmentInfo.setDocCount(numDocsInRAM);
|
||||
segmentInfo.setMaxDoc(numDocsInRAM);
|
||||
final SegmentWriteState flushState = new SegmentWriteState(infoStream, directory, segmentInfo, fieldInfos.finish(),
|
||||
pendingUpdates, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
|
||||
final double startMBUsed = bytesUsed() / 1024. / 1024.;
|
||||
|
@ -448,7 +448,7 @@ class DocumentsWriterPerThread {
|
|||
infoStream.message("DWPT", "flushed: segment=" + segmentInfo.name +
|
||||
" ramUsed=" + nf.format(startMBUsed) + " MB" +
|
||||
" newFlushedSize=" + nf.format(newSegmentSize) + " MB" +
|
||||
" docs/MB=" + nf.format(flushState.segmentInfo.getDocCount() / newSegmentSize));
|
||||
" docs/MB=" + nf.format(flushState.segmentInfo.maxDoc() / newSegmentSize));
|
||||
}
|
||||
|
||||
assert segmentInfo != null;
|
||||
|
@ -480,7 +480,7 @@ class DocumentsWriterPerThread {
|
|||
|
||||
IndexWriter.setDiagnostics(newSegment.info, IndexWriter.SOURCE_FLUSH);
|
||||
|
||||
IOContext context = new IOContext(new FlushInfo(newSegment.info.getDocCount(), newSegment.sizeInBytes()));
|
||||
IOContext context = new IOContext(new FlushInfo(newSegment.info.maxDoc(), newSegment.sizeInBytes()));
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
|
|
|
@ -63,7 +63,7 @@ final class FreqProxTermsWriter extends TermsHash {
|
|||
int doc = postingsEnum.nextDoc();
|
||||
if (doc < delDocLimit) {
|
||||
if (state.liveDocs == null) {
|
||||
state.liveDocs = state.segmentInfo.getCodec().liveDocsFormat().newLiveDocs(state.segmentInfo.getDocCount());
|
||||
state.liveDocs = state.segmentInfo.getCodec().liveDocsFormat().newLiveDocs(state.segmentInfo.maxDoc());
|
||||
}
|
||||
if (state.liveDocs.get(doc)) {
|
||||
state.delCountOnFlush++;
|
||||
|
|
|
@ -839,7 +839,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
|
||||
rollbackSegments = segmentInfos.createBackupSegmentInfos();
|
||||
pendingNumDocs.set(segmentInfos.totalDocCount());
|
||||
pendingNumDocs.set(segmentInfos.totalMaxDoc());
|
||||
|
||||
// start with previous field numbers, but new FieldInfos
|
||||
globalFieldNumberMap = getFieldNumberMap();
|
||||
|
@ -1047,7 +1047,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* @see #numDocs */
|
||||
public synchronized int maxDoc() {
|
||||
ensureOpen();
|
||||
return docWriter.getNumDocs() + segmentInfos.totalDocCount();
|
||||
return docWriter.getNumDocs() + segmentInfos.totalMaxDoc();
|
||||
}
|
||||
|
||||
/** Returns total number of docs in this index, including
|
||||
|
@ -1060,7 +1060,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
ensureOpen();
|
||||
int count = docWriter.getNumDocs();
|
||||
for (final SegmentCommitInfo info : segmentInfos) {
|
||||
count += info.info.getDocCount() - numDeletedDocs(info);
|
||||
count += info.info.maxDoc() - numDeletedDocs(info);
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
@ -1255,7 +1255,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
rld.initWritableLiveDocs();
|
||||
if (rld.delete(docID)) {
|
||||
final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount();
|
||||
if (fullDelCount == rld.info.info.getDocCount()) {
|
||||
if (fullDelCount == rld.info.info.maxDoc()) {
|
||||
// If a merge has already registered for this
|
||||
// segment, we leave it in the readerPool; the
|
||||
// merge will skip merging it and will then drop
|
||||
|
@ -1489,9 +1489,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
|
||||
// for test purpose
|
||||
final synchronized int getDocCount(int i) {
|
||||
final synchronized int maxDoc(int i) {
|
||||
if (i >= 0 && i < segmentInfos.size()) {
|
||||
return segmentInfos.info(i).info.getDocCount();
|
||||
return segmentInfos.info(i).info.maxDoc();
|
||||
} else {
|
||||
return -1;
|
||||
}
|
||||
|
@ -2065,7 +2065,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
// Abort any running merges
|
||||
abortMerges();
|
||||
// Remove all segments
|
||||
pendingNumDocs.addAndGet(-segmentInfos.totalDocCount());
|
||||
pendingNumDocs.addAndGet(-segmentInfos.totalMaxDoc());
|
||||
segmentInfos.clear();
|
||||
// Ask deleter to locate unreferenced files & remove them:
|
||||
deleter.checkpoint(segmentInfos, false);
|
||||
|
@ -2344,19 +2344,19 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
List<SegmentCommitInfo> infos = new ArrayList<>();
|
||||
|
||||
// long so we can detect int overflow:
|
||||
long totalDocCount = 0;
|
||||
long totalMaxDoc = 0;
|
||||
List<SegmentInfos> commits = new ArrayList<>(dirs.length);
|
||||
for (Directory dir : dirs) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "addIndexes: process directory " + dir);
|
||||
}
|
||||
SegmentInfos sis = SegmentInfos.readLatestCommit(dir); // read infos from dir
|
||||
totalDocCount += sis.totalDocCount();
|
||||
totalMaxDoc += sis.totalMaxDoc();
|
||||
commits.add(sis);
|
||||
}
|
||||
|
||||
// Best-effort up front check:
|
||||
testReserveDocs(totalDocCount);
|
||||
testReserveDocs(totalMaxDoc);
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
|
@ -2370,7 +2370,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
infoStream.message("IW", "addIndexes: process segment origName=" + info.info.name + " newName=" + newSegName + " info=" + info);
|
||||
}
|
||||
|
||||
IOContext context = new IOContext(new FlushInfo(info.info.getDocCount(), info.sizeInBytes()));
|
||||
IOContext context = new IOContext(new FlushInfo(info.info.maxDoc(), info.sizeInBytes()));
|
||||
|
||||
FieldInfos fis = readFieldInfos(info);
|
||||
for(FieldInfo fi : fis) {
|
||||
|
@ -2394,7 +2394,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
ensureOpen();
|
||||
|
||||
// Now reserve the docs, just before we update SIS:
|
||||
reserveDocs(totalDocCount);
|
||||
reserveDocs(totalMaxDoc);
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
|
@ -2586,7 +2586,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
//System.out.println("copy seg=" + info.info.name + " version=" + info.info.getVersion());
|
||||
// Same SI as before but we change directory and name
|
||||
SegmentInfo newInfo = new SegmentInfo(directory, info.info.getVersion(), segName, info.info.getDocCount(),
|
||||
SegmentInfo newInfo = new SegmentInfo(directory, info.info.getVersion(), segName, info.info.maxDoc(),
|
||||
info.info.getUseCompoundFile(), info.info.getCodec(),
|
||||
info.info.getDiagnostics(), info.info.getId(), info.info.getAttributes());
|
||||
SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getDelGen(),
|
||||
|
@ -3013,7 +3013,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
flushDeletesCount.incrementAndGet();
|
||||
final BufferedUpdatesStream.ApplyDeletesResult result;
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "now apply all deletes for all segments maxDoc=" + (docWriter.getNumDocs() + segmentInfos.totalDocCount()));
|
||||
infoStream.message("IW", "now apply all deletes for all segments maxDoc=" + (docWriter.getNumDocs() + segmentInfos.totalMaxDoc()));
|
||||
}
|
||||
result = bufferedUpdatesStream.applyDeletesAndUpdates(readerPool, segmentInfos.asList());
|
||||
if (result.anyDeletes) {
|
||||
|
@ -3030,7 +3030,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
// it once it's done:
|
||||
if (!mergingSegments.contains(info)) {
|
||||
segmentInfos.remove(info);
|
||||
pendingNumDocs.addAndGet(-info.info.getDocCount());
|
||||
pendingNumDocs.addAndGet(-info.info.maxDoc());
|
||||
readerPool.drop(info);
|
||||
}
|
||||
}
|
||||
|
@ -3083,7 +3083,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
if (mergedDeletesAndUpdates == null) {
|
||||
mergedDeletesAndUpdates = readerPool.get(merge.info, true);
|
||||
docMap = merge.getDocMap(mergeState);
|
||||
assert docMap.isConsistent(merge.info.info.getDocCount());
|
||||
assert docMap.isConsistent(merge.info.info.maxDoc());
|
||||
}
|
||||
if (initWritableLiveDocs && !initializedWritableLiveDocs) {
|
||||
mergedDeletesAndUpdates.initWritableLiveDocs();
|
||||
|
@ -3147,7 +3147,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
for (int i = 0; i < sourceSegments.size(); i++) {
|
||||
SegmentCommitInfo info = sourceSegments.get(i);
|
||||
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
|
||||
final int docCount = info.info.getDocCount();
|
||||
final int maxDoc = info.info.maxDoc();
|
||||
final Bits prevLiveDocs = merge.readers.get(i).getLiveDocs();
|
||||
final ReadersAndUpdates rld = readerPool.get(info, false);
|
||||
// We hold a ref so it should still be in the pool:
|
||||
|
@ -3172,7 +3172,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
mergingFields[idx] = field;
|
||||
dvFieldUpdates[idx] = mergedDVUpdates.getUpdates(field, updates.type);
|
||||
if (dvFieldUpdates[idx] == null) {
|
||||
dvFieldUpdates[idx] = mergedDVUpdates.newUpdates(field, updates.type, mergeState.segmentInfo.getDocCount());
|
||||
dvFieldUpdates[idx] = mergedDVUpdates.newUpdates(field, updates.type, mergeState.segmentInfo.maxDoc());
|
||||
}
|
||||
updatesIters[idx] = updates.iterator();
|
||||
updatesIters[idx].nextDoc(); // advance to first update doc
|
||||
|
@ -3186,8 +3186,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
// If we had deletions on starting the merge we must
|
||||
// still have deletions now:
|
||||
assert currentLiveDocs != null;
|
||||
assert prevLiveDocs.length() == docCount;
|
||||
assert currentLiveDocs.length() == docCount;
|
||||
assert prevLiveDocs.length() == maxDoc;
|
||||
assert currentLiveDocs.length() == maxDoc;
|
||||
|
||||
// There were deletes on this segment when the merge
|
||||
// started. The merge has collapsed away those
|
||||
|
@ -3205,7 +3205,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
// This means this segment received new deletes
|
||||
// since we started the merge, so we
|
||||
// must merge them:
|
||||
for (int j = 0; j < docCount; j++) {
|
||||
for (int j = 0; j < maxDoc; j++) {
|
||||
if (!prevLiveDocs.get(j)) {
|
||||
assert !currentLiveDocs.get(j);
|
||||
} else {
|
||||
|
@ -3225,7 +3225,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
} else if (mergingFields != null) {
|
||||
// need to check each non-deleted document if it has any updates
|
||||
for (int j = 0; j < docCount; j++) {
|
||||
for (int j = 0; j < maxDoc; j++) {
|
||||
if (prevLiveDocs.get(j)) {
|
||||
// document isn't deleted, check if any of the fields have an update to it
|
||||
maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
|
||||
|
@ -3237,13 +3237,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
}
|
||||
} else {
|
||||
docUpto += info.info.getDocCount() - info.getDelCount() - rld.getPendingDeleteCount();
|
||||
docUpto += info.info.maxDoc() - info.getDelCount() - rld.getPendingDeleteCount();
|
||||
}
|
||||
} else if (currentLiveDocs != null) {
|
||||
assert currentLiveDocs.length() == docCount;
|
||||
assert currentLiveDocs.length() == maxDoc;
|
||||
// This segment had no deletes before but now it
|
||||
// does:
|
||||
for (int j = 0; j < docCount; j++) {
|
||||
for (int j = 0; j < maxDoc; j++) {
|
||||
if (!currentLiveDocs.get(j)) {
|
||||
if (holder.mergedDeletesAndUpdates == null || !holder.initializedWritableLiveDocs) {
|
||||
holder.init(readerPool, merge, mergeState, true);
|
||||
|
@ -3259,18 +3259,18 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
} else if (mergingFields != null) {
|
||||
// no deletions before or after, but there were updates
|
||||
for (int j = 0; j < docCount; j++) {
|
||||
for (int j = 0; j < maxDoc; j++) {
|
||||
maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
|
||||
// advance docUpto for every non-deleted document
|
||||
docUpto++;
|
||||
}
|
||||
} else {
|
||||
// No deletes or updates before or after
|
||||
docUpto += info.info.getDocCount();
|
||||
docUpto += info.info.maxDoc();
|
||||
}
|
||||
}
|
||||
|
||||
assert docUpto == merge.info.info.getDocCount();
|
||||
assert docUpto == merge.info.info.maxDoc();
|
||||
|
||||
if (mergedDVUpdates.any()) {
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMergedDeletes: mergedDeletes.info=" + mergedDeletes.info + ", mergedFieldUpdates=" + mergedFieldUpdates);
|
||||
|
@ -3348,7 +3348,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
return false;
|
||||
}
|
||||
|
||||
final ReadersAndUpdates mergedUpdates = merge.info.info.getDocCount() == 0 ? null : commitMergedDeletesAndUpdates(merge, mergeState);
|
||||
final ReadersAndUpdates mergedUpdates = merge.info.info.maxDoc() == 0 ? null : commitMergedDeletesAndUpdates(merge, mergeState);
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMerge: mergedDeletes=" + mergedDeletes);
|
||||
|
||||
// If the doc store we are using has been closed and
|
||||
|
@ -3359,9 +3359,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
assert !segmentInfos.contains(merge.info);
|
||||
|
||||
final boolean allDeleted = merge.segments.size() == 0 ||
|
||||
merge.info.info.getDocCount() == 0 ||
|
||||
merge.info.info.maxDoc() == 0 ||
|
||||
(mergedUpdates != null &&
|
||||
mergedUpdates.getPendingDeleteCount() == merge.info.info.getDocCount());
|
||||
mergedUpdates.getPendingDeleteCount() == merge.info.info.maxDoc());
|
||||
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
if (allDeleted) {
|
||||
|
@ -3375,7 +3375,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
// the new segment:
|
||||
assert merge.segments.size() > 0 || dropSegment;
|
||||
|
||||
assert merge.info.info.getDocCount() != 0 || keepFullyDeletedSegments || dropSegment;
|
||||
assert merge.info.info.maxDoc() != 0 || keepFullyDeletedSegments || dropSegment;
|
||||
|
||||
if (mergedUpdates != null) {
|
||||
boolean success = false;
|
||||
|
@ -3404,7 +3404,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
// Now deduct the deleted docs that we just reclaimed from this
|
||||
// merge:
|
||||
int delDocCount = merge.totalDocCount - merge.info.info.getDocCount();
|
||||
int delDocCount = merge.totalMaxDoc - merge.info.info.maxDoc();
|
||||
assert delDocCount >= 0;
|
||||
pendingNumDocs.addAndGet(-delDocCount);
|
||||
|
||||
|
@ -3538,7 +3538,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
if (merge.info != null && merge.rateLimiter.getAbort() == false) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.info.getDocCount() + " docs");
|
||||
infoStream.message("IW", "merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.info.maxDoc() + " docs");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -3624,10 +3624,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
assert merge.estimatedMergeBytes == 0;
|
||||
assert merge.totalMergeBytes == 0;
|
||||
for(SegmentCommitInfo info : merge.segments) {
|
||||
if (info.info.getDocCount() > 0) {
|
||||
if (info.info.maxDoc() > 0) {
|
||||
final int delCount = numDeletedDocs(info);
|
||||
assert delCount <= info.info.getDocCount();
|
||||
final double delRatio = ((double) delCount)/info.info.getDocCount();
|
||||
assert delCount <= info.info.maxDoc();
|
||||
final double delRatio = ((double) delCount)/info.info.maxDoc();
|
||||
merge.estimatedMergeBytes += info.sizeInBytes() * (1.0 - delRatio);
|
||||
merge.totalMergeBytes += info.sizeInBytes();
|
||||
}
|
||||
|
@ -3698,7 +3698,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
for(SegmentCommitInfo info : result.allDeleted) {
|
||||
segmentInfos.remove(info);
|
||||
pendingNumDocs.addAndGet(-info.info.getDocCount());
|
||||
pendingNumDocs.addAndGet(-info.info.maxDoc());
|
||||
if (merge.segments.contains(info)) {
|
||||
mergingSegments.remove(info);
|
||||
merge.segments.remove(info);
|
||||
|
@ -3887,7 +3887,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
synchronized (this) {
|
||||
// We must also sync on IW here, because another thread could be writing
|
||||
// new DV updates / remove old gen field infos files causing FNFE:
|
||||
newReader = new SegmentReader(info, reader, liveDocs, info.info.getDocCount() - delCount);
|
||||
newReader = new SegmentReader(info, reader, liveDocs, info.info.maxDoc() - delCount);
|
||||
}
|
||||
|
||||
boolean released = false;
|
||||
|
@ -3904,7 +3904,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
|
||||
merge.readers.add(reader);
|
||||
assert delCount <= info.info.getDocCount(): "delCount=" + delCount + " info.docCount=" + info.info.getDocCount() + " rld.pendingDeleteCount=" + rld.getPendingDeleteCount() + " info.getDelCount()=" + info.getDelCount();
|
||||
assert delCount <= info.info.maxDoc(): "delCount=" + delCount + " info.maxDoc=" + info.info.maxDoc() + " rld.pendingDeleteCount=" + rld.getPendingDeleteCount() + " info.getDelCount()=" + info.getDelCount();
|
||||
segUpto++;
|
||||
}
|
||||
|
||||
|
@ -3946,7 +3946,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
double segmentMB = (merge.info.sizeInBytes()/1024./1024.);
|
||||
double stoppedSec = merge.rateLimiter.getTotalStoppedNS()/1000000000.;
|
||||
double throttleSec = merge.rateLimiter.getTotalPausedNS()/1000000000.;
|
||||
infoStream.message("IW", "merge codec=" + codec + " docCount=" + merge.info.info.getDocCount() + "; merged segment has " +
|
||||
infoStream.message("IW", "merge codec=" + codec + " maxDoc=" + merge.info.info.maxDoc() + "; merged segment has " +
|
||||
(mergeState.mergeFieldInfos.hasVectors() ? "vectors" : "no vectors") + "; " +
|
||||
(mergeState.mergeFieldInfos.hasNorms() ? "norms" : "no norms") + "; " +
|
||||
(mergeState.mergeFieldInfos.hasDocValues() ? "docValues" : "no docValues") + "; " +
|
||||
|
@ -3966,12 +3966,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
if (merger.shouldMerge() == false) {
|
||||
// Merge would produce a 0-doc segment, so we do nothing except commit the merge to remove all the 0-doc segments that we "merged":
|
||||
assert merge.info.info.getDocCount() == 0;
|
||||
assert merge.info.info.maxDoc() == 0;
|
||||
commitMerge(merge, mergeState);
|
||||
return 0;
|
||||
}
|
||||
|
||||
assert merge.info.info.getDocCount() > 0;
|
||||
assert merge.info.info.maxDoc() > 0;
|
||||
|
||||
// Very important to do this before opening the reader
|
||||
// because codec must know if prox was written for
|
||||
|
@ -4103,7 +4103,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
}
|
||||
|
||||
return merge.info.info.getDocCount();
|
||||
return merge.info.info.maxDoc();
|
||||
}
|
||||
|
||||
synchronized void addMergeException(MergePolicy.OneMerge merge) {
|
||||
|
|
|
@ -152,10 +152,10 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
protected long sizeDocs(SegmentCommitInfo info, IndexWriter writer) throws IOException {
|
||||
if (calibrateSizeByDeletes) {
|
||||
int delCount = writer.numDeletedDocs(info);
|
||||
assert delCount <= info.info.getDocCount();
|
||||
return (info.info.getDocCount() - (long)delCount);
|
||||
assert delCount <= info.info.maxDoc();
|
||||
return (info.info.maxDoc() - (long)delCount);
|
||||
} else {
|
||||
return info.info.getDocCount();
|
||||
return info.info.maxDoc();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -87,8 +87,9 @@ public abstract class MergePolicy {
|
|||
* an individual primitive merge operation, resulting in
|
||||
* a single new segment. The merge spec includes the
|
||||
* subset of segments to be merged as well as whether the
|
||||
* new segment should use the compound file format. */
|
||||
|
||||
* new segment should use the compound file format.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public static class OneMerge {
|
||||
|
||||
SegmentCommitInfo info; // used by IndexWriter
|
||||
|
@ -114,7 +115,7 @@ public abstract class MergePolicy {
|
|||
volatile long mergeStartNS = -1;
|
||||
|
||||
/** Total number of documents in segments to be merged, not accounting for deletions. */
|
||||
public final int totalDocCount;
|
||||
public final int totalMaxDoc;
|
||||
Throwable error;
|
||||
|
||||
/** Sole constructor.
|
||||
|
@ -128,9 +129,9 @@ public abstract class MergePolicy {
|
|||
this.segments = new ArrayList<>(segments);
|
||||
int count = 0;
|
||||
for(SegmentCommitInfo info : segments) {
|
||||
count += info.info.getDocCount();
|
||||
count += info.info.maxDoc();
|
||||
}
|
||||
totalDocCount = count;
|
||||
totalMaxDoc = count;
|
||||
|
||||
rateLimiter = new MergeRateLimiter(this);
|
||||
}
|
||||
|
@ -232,14 +233,14 @@ public abstract class MergePolicy {
|
|||
public int totalNumDocs() throws IOException {
|
||||
int total = 0;
|
||||
for (SegmentCommitInfo info : segments) {
|
||||
total += info.info.getDocCount();
|
||||
total += info.info.maxDoc();
|
||||
}
|
||||
return total;
|
||||
}
|
||||
|
||||
/** Return {@link MergeInfo} describing this merge. */
|
||||
public MergeInfo getMergeInfo() {
|
||||
return new MergeInfo(totalDocCount, estimatedMergeBytes, isExternal, maxNumSegments);
|
||||
return new MergeInfo(totalMaxDoc, estimatedMergeBytes, isExternal, maxNumSegments);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -440,9 +441,9 @@ public abstract class MergePolicy {
|
|||
protected long size(SegmentCommitInfo info, IndexWriter writer) throws IOException {
|
||||
long byteSize = info.sizeInBytes();
|
||||
int delCount = writer.numDeletedDocs(info);
|
||||
double delRatio = info.info.getDocCount() <= 0 ? 0.0f : (float) delCount / (float) info.info.getDocCount();
|
||||
double delRatio = info.info.maxDoc() <= 0 ? 0.0f : (float) delCount / (float) info.info.maxDoc();
|
||||
assert delRatio <= 1.0;
|
||||
return (info.info.getDocCount() <= 0 ? byteSize : (long) (byteSize * (1.0 - delRatio)));
|
||||
return (info.info.maxDoc() <= 0 ? byteSize : (long) (byteSize * (1.0 - delRatio)));
|
||||
}
|
||||
|
||||
/** Returns true if this single info is already fully merged (has no
|
||||
|
|
|
@ -139,7 +139,7 @@ public class MergeState {
|
|||
docBase += docMap.numDocs();
|
||||
}
|
||||
|
||||
segmentInfo.setDocCount(docBase);
|
||||
segmentInfo.setMaxDoc(docBase);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -66,7 +66,7 @@ class NormValuesWriter {
|
|||
|
||||
public void flush(SegmentWriteState state, NormsConsumer normsConsumer) throws IOException {
|
||||
|
||||
final int maxDoc = state.segmentInfo.getDocCount();
|
||||
final int maxDoc = state.segmentInfo.maxDoc();
|
||||
final PackedLongValues values = pending.build();
|
||||
|
||||
normsConsumer.addNormsField(fieldInfo,
|
||||
|
|
|
@ -84,7 +84,7 @@ class NumericDocValuesWriter extends DocValuesWriter {
|
|||
@Override
|
||||
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
|
||||
|
||||
final int maxDoc = state.segmentInfo.getDocCount();
|
||||
final int maxDoc = state.segmentInfo.maxDoc();
|
||||
final PackedLongValues values = pending.build();
|
||||
|
||||
dvConsumer.addNumericField(fieldInfo,
|
||||
|
|
|
@ -31,8 +31,6 @@ import org.apache.lucene.codecs.DocValuesConsumer;
|
|||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FlushInfo;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
@ -114,16 +112,16 @@ class ReadersAndUpdates {
|
|||
int count;
|
||||
if (liveDocs != null) {
|
||||
count = 0;
|
||||
for(int docID=0;docID<info.info.getDocCount();docID++) {
|
||||
for(int docID=0;docID<info.info.maxDoc();docID++) {
|
||||
if (liveDocs.get(docID)) {
|
||||
count++;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
count = info.info.getDocCount();
|
||||
count = info.info.maxDoc();
|
||||
}
|
||||
|
||||
assert info.info.getDocCount() - info.getDelCount() - pendingDeleteCount == count: "info.docCount=" + info.info.getDocCount() + " info.getDelCount()=" + info.getDelCount() + " pendingDeleteCount=" + pendingDeleteCount + " count=" + count;
|
||||
assert info.info.maxDoc() - info.getDelCount() - pendingDeleteCount == count: "info.maxDoc=" + info.info.maxDoc() + " info.getDelCount()=" + info.getDelCount() + " pendingDeleteCount=" + pendingDeleteCount + " count=" + count;
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -150,13 +148,13 @@ class ReadersAndUpdates {
|
|||
public synchronized boolean delete(int docID) {
|
||||
assert liveDocs != null;
|
||||
assert Thread.holdsLock(writer);
|
||||
assert docID >= 0 && docID < liveDocs.length() : "out of bounds: docid=" + docID + " liveDocsLength=" + liveDocs.length() + " seg=" + info.info.name + " docCount=" + info.info.getDocCount();
|
||||
assert docID >= 0 && docID < liveDocs.length() : "out of bounds: docid=" + docID + " liveDocsLength=" + liveDocs.length() + " seg=" + info.info.name + " maxDoc=" + info.info.maxDoc();
|
||||
assert !liveDocsShared;
|
||||
final boolean didDelete = liveDocs.get(docID);
|
||||
if (didDelete) {
|
||||
((MutableBits) liveDocs).clear(docID);
|
||||
pendingDeleteCount++;
|
||||
//System.out.println(" new del seg=" + info + " docID=" + docID + " pendingDelCount=" + pendingDeleteCount + " totDelCount=" + (info.docCount-liveDocs.count()));
|
||||
//System.out.println(" new del seg=" + info + " docID=" + docID + " pendingDelCount=" + pendingDeleteCount + " totDelCount=" + (info.info.maxDoc()-liveDocs.count()));
|
||||
}
|
||||
return didDelete;
|
||||
}
|
||||
|
@ -189,7 +187,7 @@ class ReadersAndUpdates {
|
|||
// force new liveDocs in initWritableLiveDocs even if it's null
|
||||
liveDocsShared = true;
|
||||
if (liveDocs != null) {
|
||||
return new SegmentReader(reader.getSegmentInfo(), reader, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount);
|
||||
return new SegmentReader(reader.getSegmentInfo(), reader, liveDocs, info.info.maxDoc() - info.getDelCount() - pendingDeleteCount);
|
||||
} else {
|
||||
// liveDocs == null and reader != null. That can only be if there are no deletes
|
||||
assert reader.getLiveDocs() == null;
|
||||
|
@ -200,7 +198,7 @@ class ReadersAndUpdates {
|
|||
|
||||
public synchronized void initWritableLiveDocs() throws IOException {
|
||||
assert Thread.holdsLock(writer);
|
||||
assert info.info.getDocCount() > 0;
|
||||
assert info.info.maxDoc() > 0;
|
||||
//System.out.println("initWritableLivedocs seg=" + info + " liveDocs=" + liveDocs + " shared=" + shared);
|
||||
if (liveDocsShared) {
|
||||
// Copy on write: this means we've cloned a
|
||||
|
@ -210,7 +208,7 @@ class ReadersAndUpdates {
|
|||
LiveDocsFormat liveDocsFormat = info.info.getCodec().liveDocsFormat();
|
||||
if (liveDocs == null) {
|
||||
//System.out.println("create BV seg=" + info);
|
||||
liveDocs = liveDocsFormat.newLiveDocs(info.info.getDocCount());
|
||||
liveDocs = liveDocsFormat.newLiveDocs(info.info.maxDoc());
|
||||
} else {
|
||||
liveDocs = liveDocsFormat.newLiveDocs(liveDocs);
|
||||
}
|
||||
|
@ -256,7 +254,7 @@ class ReadersAndUpdates {
|
|||
}
|
||||
|
||||
// We have new deletes
|
||||
assert liveDocs.length() == info.info.getDocCount();
|
||||
assert liveDocs.length() == info.info.maxDoc();
|
||||
|
||||
// Do this so we can delete any created files on
|
||||
// exception; this saves all codecs from having to do
|
||||
|
@ -303,8 +301,8 @@ class ReadersAndUpdates {
|
|||
|
||||
final long nextDocValuesGen = info.getNextDocValuesGen();
|
||||
final String segmentSuffix = Long.toString(nextDocValuesGen, Character.MAX_RADIX);
|
||||
final long estUpdatesSize = fieldUpdates.ramBytesPerDoc() * info.info.getDocCount();
|
||||
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.getDocCount(), estUpdatesSize));
|
||||
final long estUpdatesSize = fieldUpdates.ramBytesPerDoc() * info.info.maxDoc();
|
||||
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.maxDoc(), estUpdatesSize));
|
||||
final FieldInfo fieldInfo = infos.fieldInfo(field);
|
||||
assert fieldInfo != null;
|
||||
fieldInfo.setDocValuesGen(nextDocValuesGen);
|
||||
|
@ -376,8 +374,8 @@ class ReadersAndUpdates {
|
|||
|
||||
final long nextDocValuesGen = info.getNextDocValuesGen();
|
||||
final String segmentSuffix = Long.toString(nextDocValuesGen, Character.MAX_RADIX);
|
||||
final long estUpdatesSize = fieldUpdates.ramBytesPerDoc() * info.info.getDocCount();
|
||||
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.getDocCount(), estUpdatesSize));
|
||||
final long estUpdatesSize = fieldUpdates.ramBytesPerDoc() * info.info.maxDoc();
|
||||
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.maxDoc(), estUpdatesSize));
|
||||
final FieldInfo fieldInfo = infos.fieldInfo(field);
|
||||
assert fieldInfo != null;
|
||||
fieldInfo.setDocValuesGen(nextDocValuesGen);
|
||||
|
@ -448,7 +446,7 @@ class ReadersAndUpdates {
|
|||
// HEADER + FOOTER: 40
|
||||
// 90 bytes per-field (over estimating long name and attributes map)
|
||||
final long estInfosSize = 40 + 90 * fieldInfos.size();
|
||||
final IOContext infosContext = new IOContext(new FlushInfo(info.info.getDocCount(), estInfosSize));
|
||||
final IOContext infosContext = new IOContext(new FlushInfo(info.info.maxDoc(), estInfosSize));
|
||||
// separately also track which files were created for this gen
|
||||
final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
|
||||
infosFormat.write(trackingDir, info.info, segmentSuffix, fieldInfos, infosContext);
|
||||
|
@ -577,7 +575,7 @@ class ReadersAndUpdates {
|
|||
|
||||
// if there is a reader open, reopen it to reflect the updates
|
||||
if (reader != null) {
|
||||
SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount);
|
||||
SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.maxDoc() - info.getDelCount() - pendingDeleteCount);
|
||||
boolean reopened = false;
|
||||
try {
|
||||
reader.decRef();
|
||||
|
|
|
@ -26,8 +26,6 @@ import java.util.Map;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/** Embeds a [read-only] SegmentInfo and adds per-commit
|
||||
* fields.
|
||||
*
|
||||
|
@ -312,8 +310,8 @@ public class SegmentCommitInfo {
|
|||
}
|
||||
|
||||
void setDelCount(int delCount) {
|
||||
if (delCount < 0 || delCount > info.getDocCount()) {
|
||||
throw new IllegalArgumentException("invalid delCount=" + delCount + " (docCount=" + info.getDocCount() + ")");
|
||||
if (delCount < 0 || delCount > info.maxDoc()) {
|
||||
throw new IllegalArgumentException("invalid delCount=" + delCount + " (maxDoc=" + info.maxDoc() + ")");
|
||||
}
|
||||
this.delCount = delCount;
|
||||
}
|
||||
|
|
|
@ -52,7 +52,7 @@ public final class SegmentInfo {
|
|||
/** Unique segment name in the directory. */
|
||||
public final String name;
|
||||
|
||||
private int docCount; // number of docs in seg
|
||||
private int maxDoc; // number of docs in seg
|
||||
|
||||
/** Where this segment resides. */
|
||||
public final Directory dir;
|
||||
|
@ -90,14 +90,14 @@ public final class SegmentInfo {
|
|||
* <p>Note: this is public only to allow access from
|
||||
* the codecs package.</p>
|
||||
*/
|
||||
public SegmentInfo(Directory dir, Version version, String name, int docCount,
|
||||
public SegmentInfo(Directory dir, Version version, String name, int maxDoc,
|
||||
boolean isCompoundFile, Codec codec, Map<String,String> diagnostics,
|
||||
byte[] id, Map<String,String> attributes) {
|
||||
assert !(dir instanceof TrackingDirectoryWrapper);
|
||||
this.dir = Objects.requireNonNull(dir);
|
||||
this.version = Objects.requireNonNull(version);
|
||||
this.name = Objects.requireNonNull(name);
|
||||
this.docCount = docCount;
|
||||
this.maxDoc = maxDoc;
|
||||
this.isCompoundFile = isCompoundFile;
|
||||
this.codec = codec;
|
||||
this.diagnostics = Objects.requireNonNull(diagnostics);
|
||||
|
@ -142,19 +142,19 @@ public final class SegmentInfo {
|
|||
|
||||
/** Returns number of documents in this segment (deletions
|
||||
* are not taken into account). */
|
||||
public int getDocCount() {
|
||||
if (this.docCount == -1) {
|
||||
throw new IllegalStateException("docCount isn't set yet");
|
||||
public int maxDoc() {
|
||||
if (this.maxDoc == -1) {
|
||||
throw new IllegalStateException("maxDoc isn't set yet");
|
||||
}
|
||||
return docCount;
|
||||
return maxDoc;
|
||||
}
|
||||
|
||||
// NOTE: leave package private
|
||||
void setDocCount(int docCount) {
|
||||
if (this.docCount != -1) {
|
||||
throw new IllegalStateException("docCount was already set: this.docCount=" + this.docCount + " vs docCount=" + docCount);
|
||||
void setMaxDoc(int maxDoc) {
|
||||
if (this.maxDoc != -1) {
|
||||
throw new IllegalStateException("maxDoc was already set: this.maxDoc=" + this.maxDoc + " vs maxDoc=" + maxDoc);
|
||||
}
|
||||
this.docCount = docCount;
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
/** Return all files referenced by this SegmentInfo. */
|
||||
|
@ -186,7 +186,7 @@ public final class SegmentInfo {
|
|||
char cfs = getUseCompoundFile() ? 'c' : 'C';
|
||||
s.append(cfs);
|
||||
|
||||
s.append(docCount);
|
||||
s.append(maxDoc);
|
||||
|
||||
if (delCount != 0) {
|
||||
s.append('/').append(delCount);
|
||||
|
|
|
@ -301,11 +301,11 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
Codec codec = Codec.forName(input.readString());
|
||||
SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
|
||||
info.setCodec(codec);
|
||||
totalDocs += info.getDocCount();
|
||||
totalDocs += info.maxDoc();
|
||||
long delGen = input.readLong();
|
||||
int delCount = input.readInt();
|
||||
if (delCount < 0 || delCount > info.getDocCount()) {
|
||||
throw new CorruptIndexException("invalid deletion count: " + delCount + " vs docCount=" + info.getDocCount(), input);
|
||||
if (delCount < 0 || delCount > info.maxDoc()) {
|
||||
throw new CorruptIndexException("invalid deletion count: " + delCount + " vs maxDoc=" + info.maxDoc(), input);
|
||||
}
|
||||
long fieldInfosGen = input.readLong();
|
||||
long dvGen = input.readLong();
|
||||
|
@ -402,8 +402,8 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
segnOutput.writeString(si.getCodec().getName());
|
||||
segnOutput.writeLong(siPerCommit.getDelGen());
|
||||
int delCount = siPerCommit.getDelCount();
|
||||
if (delCount < 0 || delCount > si.getDocCount()) {
|
||||
throw new IllegalStateException("cannot write segment: invalid docCount segment=" + si.name + " docCount=" + si.getDocCount() + " delCount=" + delCount);
|
||||
if (delCount < 0 || delCount > si.maxDoc()) {
|
||||
throw new IllegalStateException("cannot write segment: invalid maxDoc segment=" + si.name + " maxDoc=" + si.maxDoc() + " delCount=" + delCount);
|
||||
}
|
||||
segnOutput.writeInt(delCount);
|
||||
segnOutput.writeLong(siPerCommit.getFieldInfosGen());
|
||||
|
@ -742,12 +742,12 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
lastGeneration = other.lastGeneration;
|
||||
}
|
||||
|
||||
/** Returns sum of all segment's docCounts. Note that
|
||||
/** Returns sum of all segment's maxDocs. Note that
|
||||
* this does not include deletions */
|
||||
public int totalDocCount() {
|
||||
public int totalMaxDoc() {
|
||||
long count = 0;
|
||||
for(SegmentCommitInfo info : this) {
|
||||
count += info.info.getDocCount();
|
||||
count += info.info.maxDoc();
|
||||
}
|
||||
// we should never hit this, checks should happen elsewhere...
|
||||
assert count <= IndexWriter.getActualMaxDocs();
|
||||
|
|
|
@ -62,7 +62,7 @@ final class SegmentMerger {
|
|||
|
||||
/** True if any merging should happen */
|
||||
boolean shouldMerge() {
|
||||
return mergeState.segmentInfo.getDocCount() > 0;
|
||||
return mergeState.segmentInfo.maxDoc() > 0;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -85,7 +85,7 @@ final class SegmentMerger {
|
|||
long t1 = System.nanoTime();
|
||||
mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge stored fields [" + numMerged + " docs]");
|
||||
}
|
||||
assert numMerged == mergeState.segmentInfo.getDocCount(): "numMerged=" + numMerged + " vs mergeState.segmentInfo.getDocCount()=" + mergeState.segmentInfo.getDocCount();
|
||||
assert numMerged == mergeState.segmentInfo.maxDoc(): "numMerged=" + numMerged + " vs mergeState.segmentInfo.maxDoc()=" + mergeState.segmentInfo.maxDoc();
|
||||
|
||||
final SegmentWriteState segmentWriteState = new SegmentWriteState(mergeState.infoStream, directory, mergeState.segmentInfo,
|
||||
mergeState.mergeFieldInfos, null, context);
|
||||
|
@ -129,7 +129,7 @@ final class SegmentMerger {
|
|||
long t1 = System.nanoTime();
|
||||
mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge vectors [" + numMerged + " docs]");
|
||||
}
|
||||
assert numMerged == mergeState.segmentInfo.getDocCount();
|
||||
assert numMerged == mergeState.segmentInfo.maxDoc();
|
||||
}
|
||||
|
||||
// write the merged infos
|
||||
|
|
|
@ -43,9 +43,9 @@ public final class SegmentReader extends CodecReader {
|
|||
private final SegmentCommitInfo si;
|
||||
private final Bits liveDocs;
|
||||
|
||||
// Normally set to si.docCount - si.delDocCount, unless we
|
||||
// Normally set to si.maxDoc - si.delDocCount, unless we
|
||||
// were created as an NRT reader from IW, in which case IW
|
||||
// tells us the docCount:
|
||||
// tells us the number of live docs:
|
||||
private final int numDocs;
|
||||
|
||||
final SegmentCoreReaders core;
|
||||
|
@ -75,7 +75,7 @@ public final class SegmentReader extends CodecReader {
|
|||
assert si.getDelCount() == 0;
|
||||
liveDocs = null;
|
||||
}
|
||||
numDocs = si.info.getDocCount() - si.getDelCount();
|
||||
numDocs = si.info.maxDoc() - si.getDelCount();
|
||||
|
||||
fieldInfos = initFieldInfos();
|
||||
docValuesProducer = initDocValuesProducer();
|
||||
|
@ -99,7 +99,7 @@ public final class SegmentReader extends CodecReader {
|
|||
SegmentReader(SegmentCommitInfo si, SegmentReader sr) throws IOException {
|
||||
this(si, sr,
|
||||
si.info.getCodec().liveDocsFormat().readLiveDocs(si.info.dir, si, IOContext.READONCE),
|
||||
si.info.getDocCount() - si.getDelCount());
|
||||
si.info.maxDoc() - si.getDelCount());
|
||||
}
|
||||
|
||||
/** Create new SegmentReader sharing core from a previous
|
||||
|
@ -195,7 +195,7 @@ public final class SegmentReader extends CodecReader {
|
|||
@Override
|
||||
public int maxDoc() {
|
||||
// Don't call ensureOpen() here (it could affect performance)
|
||||
return si.info.getDocCount();
|
||||
return si.info.maxDoc();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -232,7 +232,7 @@ public final class SegmentReader extends CodecReader {
|
|||
public String toString() {
|
||||
// SegmentInfo.toString takes dir and number of
|
||||
// *pending* deletions; so we reverse compute that here:
|
||||
return si.toString(si.info.getDocCount() - numDocs - si.getDelCount());
|
||||
return si.toString(si.info.maxDoc() - numDocs - si.getDelCount());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -108,7 +108,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
|
||||
final int maxDoc = state.segmentInfo.getDocCount();
|
||||
final int maxDoc = state.segmentInfo.maxDoc();
|
||||
|
||||
assert pending.size() == maxDoc;
|
||||
final int valueCount = hash.size();
|
||||
|
|
|
@ -103,7 +103,7 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
|
|||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
|
||||
final int maxDoc = state.segmentInfo.getDocCount();
|
||||
final int maxDoc = state.segmentInfo.maxDoc();
|
||||
assert pendingCounts.size() == maxDoc;
|
||||
final PackedLongValues values = pending.build();
|
||||
final PackedLongValues valueCounts = pendingCounts.build();
|
||||
|
|
|
@ -147,7 +147,7 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
|
|||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
|
||||
final int maxDoc = state.segmentInfo.getDocCount();
|
||||
final int maxDoc = state.segmentInfo.maxDoc();
|
||||
final int maxCountPerDoc = maxCount;
|
||||
assert pendingCounts.size() == maxDoc;
|
||||
final int valueCount = hash.size();
|
||||
|
|
|
@ -184,7 +184,7 @@ final class StandardDirectoryReader extends DirectoryReader {
|
|||
|
||||
// Make a best effort to detect when the app illegally "rm -rf" their
|
||||
// index while a reader was open, and then called openIfChanged:
|
||||
boolean illegalDocCountChange = commitInfo.info.getDocCount() != oldReader.getSegmentInfo().info.getDocCount();
|
||||
boolean illegalDocCountChange = commitInfo.info.maxDoc() != oldReader.getSegmentInfo().info.maxDoc();
|
||||
|
||||
boolean hasNeitherDeletionsNorUpdates = commitInfo.hasDeletions()== false && commitInfo.hasFieldUpdates() == false;
|
||||
|
||||
|
|
|
@ -56,7 +56,7 @@ final class TermVectorsConsumer extends TermsHash {
|
|||
@Override
|
||||
void flush(Map<String, TermsHashPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
|
||||
if (writer != null) {
|
||||
int numDocs = state.segmentInfo.getDocCount();
|
||||
int numDocs = state.segmentInfo.maxDoc();
|
||||
assert numDocs > 0;
|
||||
// At least one doc in this run had term vectors enabled
|
||||
try {
|
||||
|
|
|
@ -585,7 +585,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
final List<SegmentCommitInfo> eligible = new ArrayList<>();
|
||||
final Collection<SegmentCommitInfo> merging = writer.getMergingSegments();
|
||||
for(SegmentCommitInfo info : infos) {
|
||||
double pctDeletes = 100.*((double) writer.numDeletedDocs(info))/info.info.getDocCount();
|
||||
double pctDeletes = 100.*((double) writer.numDeletedDocs(info))/info.info.maxDoc();
|
||||
if (pctDeletes > forceMergeDeletesPctAllowed && !merging.contains(info)) {
|
||||
eligible.add(info);
|
||||
}
|
||||
|
|
|
@ -23,7 +23,7 @@ package org.apache.lucene.store;
|
|||
|
||||
public class MergeInfo {
|
||||
|
||||
public final int totalDocCount;
|
||||
public final int totalMaxDoc;
|
||||
|
||||
public final long estimatedMergeBytes;
|
||||
|
||||
|
@ -40,8 +40,8 @@ public class MergeInfo {
|
|||
*
|
||||
*/
|
||||
|
||||
public MergeInfo(int totalDocCount, long estimatedMergeBytes, boolean isExternal, int mergeMaxNumSegments) {
|
||||
this.totalDocCount = totalDocCount;
|
||||
public MergeInfo(int totalMaxDoc, long estimatedMergeBytes, boolean isExternal, int mergeMaxNumSegments) {
|
||||
this.totalMaxDoc = totalMaxDoc;
|
||||
this.estimatedMergeBytes = estimatedMergeBytes;
|
||||
this.isExternal = isExternal;
|
||||
this.mergeMaxNumSegments = mergeMaxNumSegments;
|
||||
|
@ -56,7 +56,7 @@ public class MergeInfo {
|
|||
+ (int) (estimatedMergeBytes ^ (estimatedMergeBytes >>> 32));
|
||||
result = prime * result + (isExternal ? 1231 : 1237);
|
||||
result = prime * result + mergeMaxNumSegments;
|
||||
result = prime * result + totalDocCount;
|
||||
result = prime * result + totalMaxDoc;
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -75,14 +75,14 @@ public class MergeInfo {
|
|||
return false;
|
||||
if (mergeMaxNumSegments != other.mergeMaxNumSegments)
|
||||
return false;
|
||||
if (totalDocCount != other.totalDocCount)
|
||||
if (totalMaxDoc != other.totalMaxDoc)
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "MergeInfo [totalDocCount=" + totalDocCount
|
||||
return "MergeInfo [totalMaxDoc=" + totalMaxDoc
|
||||
+ ", estimatedMergeBytes=" + estimatedMergeBytes + ", isExternal="
|
||||
+ isExternal + ", mergeMaxNumSegments=" + mergeMaxNumSegments + "]";
|
||||
}
|
||||
|
|
|
@ -339,7 +339,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
|
||||
writer.addIndexes(aux);
|
||||
assertEquals(1040, writer.maxDoc());
|
||||
assertEquals(1000, writer.getDocCount(0));
|
||||
assertEquals(1000, writer.maxDoc(0));
|
||||
writer.close();
|
||||
|
||||
// make sure the index is correct
|
||||
|
@ -368,7 +368,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
|
||||
writer.addIndexes(aux);
|
||||
assertEquals(1032, writer.maxDoc());
|
||||
assertEquals(1000, writer.getDocCount(0));
|
||||
assertEquals(1000, writer.maxDoc(0));
|
||||
writer.close();
|
||||
|
||||
// make sure the index is correct
|
||||
|
@ -396,7 +396,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
|
||||
writer.addIndexes(aux, new MockDirectoryWrapper(random(), TestUtil.ramCopyOf(aux)));
|
||||
assertEquals(1060, writer.maxDoc());
|
||||
assertEquals(1000, writer.getDocCount(0));
|
||||
assertEquals(1000, writer.maxDoc(0));
|
||||
writer.close();
|
||||
|
||||
// make sure the index is correct
|
||||
|
@ -438,7 +438,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
}
|
||||
writer.addIndexes(aux, new MockDirectoryWrapper(random(), TestUtil.ramCopyOf(aux)));
|
||||
assertEquals(1020, writer.maxDoc());
|
||||
assertEquals(1000, writer.getDocCount(0));
|
||||
assertEquals(1000, writer.maxDoc(0));
|
||||
writer.close();
|
||||
dir.close();
|
||||
aux.close();
|
||||
|
@ -498,7 +498,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
|
||||
writer.addIndexes(aux, aux2);
|
||||
assertEquals(1040, writer.maxDoc());
|
||||
assertEquals(1000, writer.getDocCount(0));
|
||||
assertEquals(1000, writer.maxDoc(0));
|
||||
writer.close();
|
||||
dir.close();
|
||||
aux.close();
|
||||
|
|
|
@ -248,7 +248,7 @@ public class TestIndexWriterMergePolicy extends LuceneTestCase {
|
|||
|
||||
int segmentCount = writer.getSegmentCount();
|
||||
for (int i = segmentCount - 1; i >= 0; i--) {
|
||||
int docCount = writer.getDocCount(i);
|
||||
int docCount = writer.maxDoc(i);
|
||||
assertTrue("docCount=" + docCount + " lowerBound=" + lowerBound + " upperBound=" + upperBound + " i=" + i + " segmentCount=" + segmentCount + " index=" + writer.segString() + " config=" + writer.getConfig(), docCount > lowerBound);
|
||||
|
||||
if (docCount <= upperBound) {
|
||||
|
|
|
@ -320,7 +320,7 @@ public class TestIndexWriterMerging extends LuceneTestCase
|
|||
break;
|
||||
}
|
||||
for(int i=0;i<merge.segments.size();i++) {
|
||||
assert merge.segments.get(i).info.getDocCount() < 20;
|
||||
assert merge.segments.get(i).info.maxDoc() < 20;
|
||||
}
|
||||
writer.merge(merge);
|
||||
}
|
||||
|
|
|
@ -90,7 +90,7 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
new FieldInfos.FieldNumbers(),
|
||||
newIOContext(random(), new IOContext(new MergeInfo(-1, -1, false, -1))));
|
||||
MergeState mergeState = merger.merge();
|
||||
int docsMerged = mergeState.segmentInfo.getDocCount();
|
||||
int docsMerged = mergeState.segmentInfo.maxDoc();
|
||||
assertTrue(docsMerged == 2);
|
||||
//Should be able to open a new SegmentReader against the new directory
|
||||
SegmentReader mergedReader = new SegmentReader(new SegmentCommitInfo(
|
||||
|
|
|
@ -614,7 +614,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
q.add(new TermQuery(new Term("field", "c")), Occur.FILTER);
|
||||
|
||||
final Weight weight = searcher.createNormalizedWeight(q, random().nextBoolean());
|
||||
final Scorer scorer = weight.scorer(reader.leaves().get(0), null);
|
||||
final Scorer scorer = weight.scorer(searcher.getIndexReader().leaves().get(0), null);
|
||||
assertTrue(scorer instanceof ConjunctionScorer);
|
||||
assertNotNull(scorer.asTwoPhaseIterator());
|
||||
|
||||
|
@ -676,7 +676,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
q.add(new TermQuery(new Term("field", "d")), Occur.SHOULD);
|
||||
|
||||
final Weight weight = searcher.createNormalizedWeight(q, random().nextBoolean());
|
||||
final Scorer scorer = weight.scorer(reader.leaves().get(0), null);
|
||||
final Scorer scorer = weight.scorer(searcher.getIndexReader().leaves().get(0), null);
|
||||
assertTrue(scorer instanceof BoostedScorer || scorer instanceof ExactPhraseScorer);
|
||||
assertNotNull(scorer.asTwoPhaseIterator());
|
||||
|
||||
|
|
|
@ -224,7 +224,7 @@ public class TestConstantScoreQuery extends LuceneTestCase {
|
|||
ConstantScoreQuery q = new ConstantScoreQuery(pq);
|
||||
|
||||
final Weight weight = searcher.createNormalizedWeight(q, true);
|
||||
final Scorer scorer = weight.scorer(reader.leaves().get(0), null);
|
||||
final Scorer scorer = weight.scorer(searcher.getIndexReader().leaves().get(0), null);
|
||||
assertNotNull(scorer.asTwoPhaseIterator());
|
||||
|
||||
reader.close();
|
||||
|
|
|
@ -35,16 +35,16 @@ reChapterOnly = re.compile('^<CHAPTER ID=.*?>$')
|
|||
reTagOnly = re.compile('^<.*?>$')
|
||||
reNumberOnly = re.compile(r'^\d+\.?$')
|
||||
|
||||
docCount = 0
|
||||
maxDoc = 0
|
||||
didEnglish = False
|
||||
|
||||
def write(date, title, pending, fOut):
|
||||
global docCount
|
||||
global maxDoc
|
||||
body = ' '.join(pending).replace('\t', ' ').strip()
|
||||
if len(body) > 0:
|
||||
line = '%s\t%s\t%s\n' % (title, date, body)
|
||||
fOut.write(line)
|
||||
docCount += 1
|
||||
maxDoc += 1
|
||||
del pending[:]
|
||||
if VERBOSE:
|
||||
print len(body)
|
||||
|
@ -109,10 +109,10 @@ fOut = open(fileOut, 'wb')
|
|||
|
||||
for fileName in glob.glob('%s/??-??.tgz' % dirIn):
|
||||
if fileName.endswith('.tgz'):
|
||||
print 'process %s; %d docs so far...' % (fileName, docCount)
|
||||
print 'process %s; %d docs so far...' % (fileName, maxDoc)
|
||||
processTar(fileName, fOut)
|
||||
|
||||
print 'TOTAL: %s' % docCount
|
||||
print 'TOTAL: %s' % maxDoc
|
||||
|
||||
#run something like this:
|
||||
"""
|
||||
|
|
|
@ -29,7 +29,6 @@ import java.util.List;
|
|||
import java.util.Locale;
|
||||
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/**
|
||||
* Command-line tool that enables listing segments in an
|
||||
|
@ -137,7 +136,7 @@ public class IndexSplitter {
|
|||
SegmentCommitInfo infoPerCommit = getInfo(n);
|
||||
SegmentInfo info = infoPerCommit.info;
|
||||
// Same info just changing the dir:
|
||||
SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.getDocCount(),
|
||||
SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.maxDoc(),
|
||||
info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId(), new HashMap<>());
|
||||
destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(),
|
||||
infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen(),
|
||||
|
|
|
@ -132,8 +132,8 @@ public final class VersionBlockTreeTermsReader extends FieldsProducer {
|
|||
|
||||
BytesRef minTerm = readBytesRef(in);
|
||||
BytesRef maxTerm = readBytesRef(in);
|
||||
if (docCount < 0 || docCount > state.segmentInfo.getDocCount()) { // #docs with field must be <= #docs
|
||||
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.getDocCount(), in);
|
||||
if (docCount < 0 || docCount > state.segmentInfo.maxDoc()) { // #docs with field must be <= #docs
|
||||
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.maxDoc(), in);
|
||||
}
|
||||
if (sumDocFreq < docCount) { // #postings must be >= #docs with field
|
||||
throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount, in);
|
||||
|
|
|
@ -178,7 +178,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
|
|||
{
|
||||
BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock);
|
||||
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
|
||||
out = state.directory.createOutput(termsFileName, state.context);
|
||||
|
|
|
@ -58,7 +58,7 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
DocValuesConsumer consumer = in.fieldsConsumer(state);
|
||||
assert consumer != null;
|
||||
return new AssertingDocValuesConsumer(consumer, state.segmentInfo.getDocCount());
|
||||
return new AssertingDocValuesConsumer(consumer, state.segmentInfo.maxDoc());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -66,7 +66,7 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
assert state.fieldInfos.hasDocValues();
|
||||
DocValuesProducer producer = in.fieldsProducer(state);
|
||||
assert producer != null;
|
||||
return new AssertingDocValuesProducer(producer, state.segmentInfo.getDocCount());
|
||||
return new AssertingDocValuesProducer(producer, state.segmentInfo.maxDoc());
|
||||
}
|
||||
|
||||
static class AssertingDocValuesConsumer extends DocValuesConsumer {
|
||||
|
|
|
@ -63,7 +63,7 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
|
|||
public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
|
||||
Bits raw = in.readLiveDocs(dir, info, context);
|
||||
assert raw != null;
|
||||
check(raw, info.info.getDocCount(), info.getDelCount());
|
||||
check(raw, info.info.maxDoc(), info.getDelCount());
|
||||
return new AssertingBits(raw);
|
||||
}
|
||||
|
||||
|
@ -71,7 +71,7 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
|
|||
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
|
||||
assert bits instanceof AssertingMutableBits;
|
||||
MutableBits raw = (MutableBits) ((AssertingMutableBits)bits).in;
|
||||
check(raw, info.info.getDocCount(), info.getDelCount() + newDelCount);
|
||||
check(raw, info.info.maxDoc(), info.getDelCount() + newDelCount);
|
||||
in.writeLiveDocs(raw, dir, info, newDelCount, context);
|
||||
}
|
||||
|
||||
|
|
|
@ -41,7 +41,7 @@ public class AssertingNormsFormat extends NormsFormat {
|
|||
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
|
||||
NormsConsumer consumer = in.normsConsumer(state);
|
||||
assert consumer != null;
|
||||
return new AssertingNormsConsumer(consumer, state.segmentInfo.getDocCount());
|
||||
return new AssertingNormsConsumer(consumer, state.segmentInfo.maxDoc());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -49,7 +49,7 @@ public class AssertingNormsFormat extends NormsFormat {
|
|||
assert state.fieldInfos.hasNorms();
|
||||
NormsProducer producer = in.normsProducer(state);
|
||||
assert producer != null;
|
||||
return new AssertingNormsProducer(producer, state.segmentInfo.getDocCount());
|
||||
return new AssertingNormsProducer(producer, state.segmentInfo.maxDoc());
|
||||
}
|
||||
|
||||
static class AssertingNormsConsumer extends NormsConsumer {
|
||||
|
|
|
@ -41,7 +41,7 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
|
|||
|
||||
@Override
|
||||
public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
|
||||
return new AssertingStoredFieldsReader(in.fieldsReader(directory, si, fn, context), si.getDocCount());
|
||||
return new AssertingStoredFieldsReader(in.fieldsReader(directory, si, fn, context), si.maxDoc());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -84,7 +84,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
|
|||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
int minSkipInterval;
|
||||
if (state.segmentInfo.getDocCount() > 1000000) {
|
||||
if (state.segmentInfo.maxDoc() > 1000000) {
|
||||
// Test2BPostings can OOME otherwise:
|
||||
minSkipInterval = 3;
|
||||
} else {
|
||||
|
|
|
@ -248,7 +248,7 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
|
|||
postings.fieldToTerms.put(field, ramField);
|
||||
termsConsumer.reset(ramField);
|
||||
|
||||
FixedBitSet docsSeen = new FixedBitSet(state.segmentInfo.getDocCount());
|
||||
FixedBitSet docsSeen = new FixedBitSet(state.segmentInfo.maxDoc());
|
||||
long sumTotalTermFreq = 0;
|
||||
long sumDocFreq = 0;
|
||||
PostingsEnum postingsEnum = null;
|
||||
|
|
|
@ -212,7 +212,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
|
|||
// we don't assert this, because SI format has nothing to do with it... set by SIS
|
||||
// assertSame(expected.getCodec(), actual.getCodec());
|
||||
assertEquals(expected.getDiagnostics(), actual.getDiagnostics());
|
||||
assertEquals(expected.getDocCount(), actual.getDocCount());
|
||||
assertEquals(expected.maxDoc(), actual.maxDoc());
|
||||
assertIDEquals(expected.getId(), actual.getId());
|
||||
assertEquals(expected.getUseCompoundFile(), actual.getUseCompoundFile());
|
||||
assertEquals(expected.getVersion(), actual.getVersion());
|
||||
|
|
|
@ -172,6 +172,9 @@ Bug Fixes
|
|||
* SOLR-7178: OverseerAutoReplicaFailoverThread compares Integer objects using ==
|
||||
(shalin)
|
||||
|
||||
* SOLR-7171: BaseDistributedSearchTestCase now clones getSolrHome() for each subclass,
|
||||
and consistently uses getSolrXml(). (hossman)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -0,0 +1,10 @@
|
|||
# Logging level
|
||||
log4j.rootLogger=INFO, CONSOLE
|
||||
|
||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||
log4j.appender.CONSOLE.Target=System.err
|
||||
log4j.appender.CONSOLE.layout=org.apache.solr.util.SolrLogLayout
|
||||
log4j.appender.CONSOLE.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %C; %m\n
|
||||
|
||||
log4j.logger.org.apache.zookeeper=WARN
|
||||
log4j.logger.org.apache.hadoop=WARN
|
|
@ -0,0 +1,58 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.solr;
|
||||
|
||||
import org.apache.solr.client.solrj.response.QueryResponse;
|
||||
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
|
||||
import org.apache.solr.handler.component.TrackingShardHandlerFactory;
|
||||
import org.apache.solr.handler.component.TrackingShardHandlerFactory.ShardRequestAndParams;
|
||||
import org.apache.solr.handler.component.TrackingShardHandlerFactory.RequestTrackingQueue;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* super simple sanity check that SimpleTrackingShardHandler can be used in a
|
||||
* {@link BaseDistributedSearchTestCase} subclass
|
||||
*/
|
||||
public class TestSimpleTrackingShardHandler extends BaseDistributedSearchTestCase {
|
||||
|
||||
@Override
|
||||
protected String getSolrXml() {
|
||||
return "solr-trackingshardhandler.xml";
|
||||
}
|
||||
|
||||
public void testSolrXmlOverrideAndCorrectShardHandler() throws Exception {
|
||||
RequestTrackingQueue trackingQueue = new RequestTrackingQueue();
|
||||
|
||||
TrackingShardHandlerFactory.setTrackingQueue(jettys, trackingQueue);
|
||||
// sanity check that our control jetty has the correct configs as well
|
||||
TrackingShardHandlerFactory.setTrackingQueue(Collections.singletonList(controlJetty), trackingQueue);
|
||||
|
||||
QueryResponse ignored = query("q","*:*", "fl", "id", "sort", "id asc");
|
||||
|
||||
int numShardRequests = 0;
|
||||
for (List<ShardRequestAndParams> shard : trackingQueue.getAllRequests().values()) {
|
||||
for (ShardRequestAndParams shardReq : shard) {
|
||||
numShardRequests++;
|
||||
}
|
||||
}
|
||||
TrackingShardHandlerFactory.setTrackingQueue(jettys, null);
|
||||
TrackingShardHandlerFactory.setTrackingQueue(Collections.singletonList(controlJetty), null);
|
||||
}
|
||||
}
|
|
@ -54,6 +54,7 @@ import java.lang.annotation.ElementType;
|
|||
import java.lang.annotation.Retention;
|
||||
import java.lang.annotation.RetentionPolicy;
|
||||
import java.lang.annotation.Target;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
@ -306,9 +307,12 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
protected JettySolrRunner createControlJetty() throws Exception {
|
||||
writeCoreProperties(testDir.toPath().resolve("control/cores"), DEFAULT_TEST_CORENAME);
|
||||
System.setProperty("coreRootDirectory", testDir.toPath().resolve("control").toString());
|
||||
JettySolrRunner jetty = createJetty(new File(getSolrHome()), testDir + "/control/data", null, getSolrConfigFile(), getSchemaFile());
|
||||
Path jettyHome = testDir.toPath().resolve("control");
|
||||
File jettyHomeFile = jettyHome.toFile();
|
||||
seedSolrHome(jettyHomeFile);
|
||||
seedCoreRootDirWithDefaultTestCore(jettyHome);
|
||||
System.setProperty("coreRootDirectory", jettyHome.toString());
|
||||
JettySolrRunner jetty = createJetty(jettyHomeFile, null, null, getSolrConfigFile(), getSchemaFile());
|
||||
return jetty;
|
||||
}
|
||||
|
||||
|
@ -323,13 +327,14 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
|
|||
StringBuilder sb = new StringBuilder();
|
||||
for (int i = 0; i < numShards; i++) {
|
||||
if (sb.length() > 0) sb.append(',');
|
||||
String shardname = "shard" + i;
|
||||
Path coresPath = testDir.toPath().resolve(shardname).resolve("cores");
|
||||
writeCoreProperties(coresPath, DEFAULT_TEST_CORENAME);
|
||||
System.setProperty("coreRootDirectory", testDir.toPath().resolve(shardname).toString());
|
||||
JettySolrRunner j = createJetty(new File(getSolrHome()),
|
||||
testDir + "/shard" + i + "/data", null, getSolrConfigFile(),
|
||||
getSchemaFile());
|
||||
final String shardname = "shard" + i;
|
||||
Path jettyHome = testDir.toPath().resolve(shardname);
|
||||
File jettyHomeFile = jettyHome.toFile();
|
||||
seedSolrHome(jettyHomeFile);
|
||||
seedCoreRootDirWithDefaultTestCore(jettyHome);
|
||||
System.setProperty("coreRootDirectory", jettyHome.toString());
|
||||
JettySolrRunner j = createJetty(jettyHomeFile, null, null, getSolrConfigFile(), getSchemaFile());
|
||||
|
||||
jettys.add(j);
|
||||
clients.add(createNewSolrClient(j.getLocalPort()));
|
||||
String shardStr = buildUrl(j.getLocalPort()) + "/" + DEFAULT_TEST_CORENAME;
|
||||
|
@ -1041,12 +1046,38 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
|
|||
return null;
|
||||
}
|
||||
|
||||
protected void setupJettySolrHome(File jettyHome) throws IOException {
|
||||
/**
|
||||
* Given a directory that will be used as the SOLR_HOME for a jetty instance, seeds that
|
||||
* directory with the contents of {@link #getSolrHome} and ensures that the proper {@link #getSolrXml}
|
||||
* file is in place.
|
||||
*/
|
||||
private void seedSolrHome(File jettyHome) throws IOException {
|
||||
FileUtils.copyDirectory(new File(getSolrHome()), jettyHome);
|
||||
String solrxml = getSolrXml();
|
||||
if (solrxml != null) {
|
||||
FileUtils.copyFile(new File(getSolrHome(), solrxml), new File(jettyHome, "solr.xml"));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a directory that will be used as the <code>coreRootDirectory</code> for a jetty instance,
|
||||
* Creates a core directory named {@link #DEFAULT_TEST_CORENAME} using a trivial
|
||||
* <code>core.properties</code> if this file does not already exist.
|
||||
*
|
||||
* @see #writeCoreProperties(Path,String)
|
||||
* @see #CORE_PROPERTIES_FILENAME
|
||||
*/
|
||||
private void seedCoreRootDirWithDefaultTestCore(Path coreRootDirectory) throws IOException {
|
||||
// Kludgy and brittle with assumptions about writeCoreProperties, but i don't want to
|
||||
// try to change the semantics of that method to ignore existing files
|
||||
Path coreDir = coreRootDirectory.resolve(DEFAULT_TEST_CORENAME);
|
||||
if (Files.notExists(coreDir.resolve(CORE_PROPERTIES_FILENAME))) {
|
||||
writeCoreProperties(coreDir, DEFAULT_TEST_CORENAME);
|
||||
} // else nothing to do, DEFAULT_TEST_CORENAME already exists
|
||||
}
|
||||
|
||||
protected void setupJettySolrHome(File jettyHome) throws IOException {
|
||||
seedSolrHome(jettyHome);
|
||||
|
||||
Properties coreProperties = new Properties();
|
||||
coreProperties.setProperty("name", "collection1");
|
||||
|
|
|
@ -133,6 +133,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
|
|||
public abstract class SolrTestCaseJ4 extends LuceneTestCase {
|
||||
|
||||
public static final String DEFAULT_TEST_CORENAME = "collection1";
|
||||
protected static final String CORE_PROPERTIES_FILENAME = "core.properties";
|
||||
|
||||
private static String coreName = DEFAULT_TEST_CORENAME;
|
||||
|
||||
|
@ -152,7 +153,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
|
|||
log.info("Writing core.properties file to {}", coreDirectory);
|
||||
Files.createDirectories(coreDirectory);
|
||||
try (Writer writer =
|
||||
new OutputStreamWriter(Files.newOutputStream(coreDirectory.resolve("core.properties")), Charset.forName("UTF-8"))) {
|
||||
new OutputStreamWriter(Files.newOutputStream(coreDirectory.resolve(CORE_PROPERTIES_FILENAME)), Charset.forName("UTF-8"))) {
|
||||
properties.store(writer, testname);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -155,7 +155,7 @@ public class TrackingShardHandlerFactory extends HttpShardHandlerFactory {
|
|||
for (JettySolrRunner runner : runners) {
|
||||
CoreContainer container = ((SolrDispatchFilter) runner.getDispatchFilter().getFilter()).getCores();
|
||||
ShardHandlerFactory factory = container.getShardHandlerFactory();
|
||||
assert factory instanceof TrackingShardHandlerFactory;
|
||||
assert factory instanceof TrackingShardHandlerFactory : "not a TrackingShardHandlerFactory: " + factory.getClass();
|
||||
TrackingShardHandlerFactory trackingShardHandlerFactory = (TrackingShardHandlerFactory) factory;
|
||||
trackingShardHandlerFactory.setTrackingQueue(queue);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue