mirror of https://github.com/apache/lucene.git
Remove over-counting of deleted terms (#12586)
This commit is contained in:
parent
da032535d6
commit
8961dda2a4
|
@ -174,6 +174,8 @@ Improvements
|
|||
* GITHUB#12634, GITHUB#12632, GITHUB#12680, GITHUB#12681: Speed up Panama vector support
|
||||
and test improvements. (Uwe Schindler, Robert Muir)
|
||||
|
||||
* GITHUB#12586: Remove over-counting of deleted terms. (Guo Feng)
|
||||
|
||||
Optimizations
|
||||
---------------------
|
||||
* GITHUB#12183: Make TermStates#build concurrent. (Shubham Chaudhary)
|
||||
|
|
|
@ -58,7 +58,6 @@ class BufferedUpdates implements Accountable {
|
|||
+ 2 * RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
|
||||
+ 2 * Integer.BYTES
|
||||
+ 24;
|
||||
final AtomicInteger numTermDeletes = new AtomicInteger();
|
||||
final AtomicInteger numFieldUpdates = new AtomicInteger();
|
||||
|
||||
final DeletedTerms deleteTerms = new DeletedTerms();
|
||||
|
@ -85,16 +84,14 @@ class BufferedUpdates implements Accountable {
|
|||
public String toString() {
|
||||
if (VERBOSE_DELETES) {
|
||||
return ("gen=" + gen)
|
||||
+ (" numTerms=" + numTermDeletes)
|
||||
+ (", deleteTerms=" + deleteTerms)
|
||||
+ (", deleteQueries=" + deleteQueries)
|
||||
+ (", fieldUpdates=" + fieldUpdates)
|
||||
+ (", bytesUsed=" + bytesUsed);
|
||||
} else {
|
||||
String s = "gen=" + gen;
|
||||
if (numTermDeletes.get() != 0) {
|
||||
s +=
|
||||
" " + numTermDeletes.get() + " deleted terms (unique count=" + deleteTerms.size() + ")";
|
||||
if (!deleteTerms.isEmpty()) {
|
||||
s += " " + deleteTerms.size() + " unique deleted terms ";
|
||||
}
|
||||
if (deleteQueries.size() != 0) {
|
||||
s += " " + deleteQueries.size() + " deleted queries";
|
||||
|
@ -132,10 +129,6 @@ class BufferedUpdates implements Accountable {
|
|||
}
|
||||
|
||||
deleteTerms.put(term, docIDUpto);
|
||||
// note that if current != -1 then it means there's already a buffered
|
||||
// delete on that term, therefore we seem to over-count. this over-counting
|
||||
// is done to respect IndexWriterConfig.setMaxBufferedDeleteTerms.
|
||||
numTermDeletes.incrementAndGet();
|
||||
}
|
||||
|
||||
void addNumericUpdate(NumericDocValuesUpdate update, int docIDUpto) {
|
||||
|
@ -163,14 +156,12 @@ class BufferedUpdates implements Accountable {
|
|||
}
|
||||
|
||||
void clearDeleteTerms() {
|
||||
numTermDeletes.set(0);
|
||||
deleteTerms.clear();
|
||||
}
|
||||
|
||||
void clear() {
|
||||
deleteTerms.clear();
|
||||
deleteQueries.clear();
|
||||
numTermDeletes.set(0);
|
||||
numFieldUpdates.set(0);
|
||||
fieldUpdates.clear();
|
||||
bytesUsed.addAndGet(-bytesUsed.get());
|
||||
|
|
|
@ -25,7 +25,6 @@ import java.util.List;
|
|||
import java.util.Locale;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
|
@ -56,7 +55,6 @@ final class BufferedUpdatesStream implements Accountable {
|
|||
private final FinishedSegments finishedSegments;
|
||||
private final InfoStream infoStream;
|
||||
private final AtomicLong bytesUsed = new AtomicLong();
|
||||
private final AtomicInteger numTerms = new AtomicInteger();
|
||||
|
||||
BufferedUpdatesStream(InfoStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
|
@ -78,7 +76,6 @@ final class BufferedUpdatesStream implements Accountable {
|
|||
assert checkDeleteStats();
|
||||
|
||||
updates.add(packet);
|
||||
numTerms.addAndGet(packet.numTermDeletes);
|
||||
bytesUsed.addAndGet(packet.bytesUsed);
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message(
|
||||
|
@ -104,7 +101,6 @@ final class BufferedUpdatesStream implements Accountable {
|
|||
updates.clear();
|
||||
nextGen = 1;
|
||||
finishedSegments.clear();
|
||||
numTerms.set(0);
|
||||
bytesUsed.set(0);
|
||||
}
|
||||
|
||||
|
@ -112,10 +108,6 @@ final class BufferedUpdatesStream implements Accountable {
|
|||
return bytesUsed.get() != 0;
|
||||
}
|
||||
|
||||
int numTerms() {
|
||||
return numTerms.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return bytesUsed.get();
|
||||
|
@ -175,8 +167,6 @@ final class BufferedUpdatesStream implements Accountable {
|
|||
packet.applied.countDown();
|
||||
|
||||
updates.remove(packet);
|
||||
numTerms.addAndGet(-packet.numTermDeletes);
|
||||
assert numTerms.get() >= 0 : "numTerms=" + numTerms + " packet=" + packet;
|
||||
|
||||
bytesUsed.addAndGet(-packet.bytesUsed);
|
||||
|
||||
|
@ -311,13 +301,10 @@ final class BufferedUpdatesStream implements Accountable {
|
|||
|
||||
// only for assert
|
||||
private boolean checkDeleteStats() {
|
||||
int numTerms2 = 0;
|
||||
long bytesUsed2 = 0;
|
||||
for (FrozenBufferedUpdates packet : updates) {
|
||||
numTerms2 += packet.numTermDeletes;
|
||||
bytesUsed2 += packet.bytesUsed;
|
||||
}
|
||||
assert numTerms2 == numTerms.get() : "numTerms2=" + numTerms2 + " vs " + numTerms.get();
|
||||
assert bytesUsed2 == bytesUsed.get() : "bytesUsed2=" + bytesUsed2 + " vs " + bytesUsed;
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -366,11 +366,6 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
return deleteQueue.getBufferedUpdatesTermsSize();
|
||||
}
|
||||
|
||||
// for testing
|
||||
int getNumBufferedDeleteTerms() {
|
||||
return deleteQueue.numGlobalTermDeletes();
|
||||
}
|
||||
|
||||
boolean anyDeletions() {
|
||||
return deleteQueue.anyChanges();
|
||||
}
|
||||
|
|
|
@ -392,8 +392,9 @@ final class DocumentsWriterDeleteQueue implements Accountable, Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
public int numGlobalTermDeletes() {
|
||||
return globalBufferedUpdates.numTermDeletes.get();
|
||||
/** For test purposes. */
|
||||
int numGlobalTermDeletes() {
|
||||
return globalBufferedUpdates.deleteTerms.size();
|
||||
}
|
||||
|
||||
void clear() {
|
||||
|
|
|
@ -70,7 +70,6 @@ final class FrozenBufferedUpdates {
|
|||
private final int fieldUpdatesCount;
|
||||
|
||||
final int bytesUsed;
|
||||
final int numTermDeletes;
|
||||
|
||||
private long delGen = -1; // assigned by BufferedUpdatesStream once pushed
|
||||
|
||||
|
@ -111,7 +110,6 @@ final class FrozenBufferedUpdates {
|
|||
((deleteTerms.ramBytesUsed() + deleteQueries.length * (long) BYTES_PER_DEL_QUERY)
|
||||
+ updates.fieldUpdatesBytesUsed.get());
|
||||
|
||||
numTermDeletes = updates.numTermDeletes.get();
|
||||
if (infoStream != null && infoStream.isEnabled("BD")) {
|
||||
infoStream.message(
|
||||
"BD",
|
||||
|
@ -513,11 +511,8 @@ final class FrozenBufferedUpdates {
|
|||
@Override
|
||||
public String toString() {
|
||||
String s = "delGen=" + delGen;
|
||||
if (numTermDeletes != 0) {
|
||||
s += " numDeleteTerms=" + numTermDeletes;
|
||||
if (numTermDeletes != deleteTerms.size()) {
|
||||
s += " (" + deleteTerms.size() + " unique)";
|
||||
}
|
||||
if (deleteTerms.size() != 0) {
|
||||
s += " unique deleteTerms=" + deleteTerms.size();
|
||||
}
|
||||
if (deleteQueries.length != 0) {
|
||||
s += " numDeleteQueries=" + deleteQueries.length;
|
||||
|
|
|
@ -5367,11 +5367,6 @@ public class IndexWriter
|
|||
return docWriter.getBufferedDeleteTermsSize();
|
||||
}
|
||||
|
||||
// For test purposes.
|
||||
final int getNumBufferedDeleteTerms() {
|
||||
return docWriter.getNumBufferedDeleteTerms();
|
||||
}
|
||||
|
||||
// utility routines for tests
|
||||
synchronized SegmentCommitInfo newestSegment() {
|
||||
return segmentInfos.size() > 0 ? segmentInfos.info(segmentInfos.size() - 1) : null;
|
||||
|
|
|
@ -156,7 +156,6 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
addDoc(modifier, ++id, value);
|
||||
if (0 == t) {
|
||||
modifier.deleteDocuments(new Term("value", String.valueOf(value)));
|
||||
assertEquals(2, modifier.getNumBufferedDeleteTerms());
|
||||
assertEquals(1, modifier.getBufferedDeleteTermsSize());
|
||||
} else modifier.deleteDocuments(new TermQuery(new Term("value", String.valueOf(value))));
|
||||
|
||||
|
|
Loading…
Reference in New Issue