package org.apache.lucene.index;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.RamUsageEstimator;
class BufferedUpdates implements Accountable {
final static int BYTES_PER_DEL_TERM = 9*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 7*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 10*Integer.BYTES;
final static int BYTES_PER_DEL_DOCID = 2*RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + Integer.BYTES;
final static int BYTES_PER_DEL_QUERY = 5*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 2*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 2*Integer.BYTES + 24;
final AtomicInteger numTermDeletes = new AtomicInteger();
final AtomicInteger numFieldUpdates = new AtomicInteger();
final Map<Term,Integer> deleteTerms = new HashMap<>();
final Map<Query,Integer> deleteQueries = new HashMap<>();
final List<Integer> deleteDocIDs = new ArrayList<>();
final Map<String, FieldUpdatesBuffer> fieldUpdates = new HashMap<>();
public static final Integer MAX_INT = Integer.valueOf(Integer.MAX_VALUE);
private final Counter bytesUsed = Counter.newCounter(true);
final Counter fieldUpdatesBytesUsed = Counter.newCounter(true);
private final static boolean VERBOSE_DELETES = false;
long gen;
final String segmentName;
public BufferedUpdates(String segmentName) {
this.segmentName = segmentName;
}
@Override
public String toString() {
if (VERBOSE_DELETES) {
return "gen=" + gen + " numTerms=" + numTermDeletes + ", deleteTerms=" + deleteTerms
+ ", deleteQueries=" + deleteQueries + ", deleteDocIDs=" + deleteDocIDs + ", fieldUpdates=" + fieldUpdates
+ ", bytesUsed=" + bytesUsed;
} else {
String s = "gen=" + gen;
if (numTermDeletes.get() != 0) {
s += " " + numTermDeletes.get() + " deleted terms (unique count=" + deleteTerms.size() + ")";
}
if (deleteQueries.size() != 0) {
s += " " + deleteQueries.size() + " deleted queries";
}
if (deleteDocIDs.size() != 0) {
s += " " + deleteDocIDs.size() + " deleted docIDs";
}
if (numFieldUpdates.get() != 0) {
s += " " + numFieldUpdates.get() + " field updates";
}
if (bytesUsed.get() != 0) {
s += " bytesUsed=" + bytesUsed.get();
}
return s;
}
}
public void addQuery(Query query, int docIDUpto) {
Integer current = deleteQueries.put(query, docIDUpto);
if (current == null) {
bytesUsed.addAndGet(BYTES_PER_DEL_QUERY);
}
}
public void addDocID(int docID) {
deleteDocIDs.add(Integer.valueOf(docID));
bytesUsed.addAndGet(BYTES_PER_DEL_DOCID);
}
public void addTerm(Term term, int docIDUpto) {
Integer current = deleteTerms.get(term);
if (current != null && docIDUpto < current) {
return;
}
deleteTerms.put(term, Integer.valueOf(docIDUpto));
numTermDeletes.incrementAndGet();
if (current == null) {
bytesUsed.addAndGet(BYTES_PER_DEL_TERM + term.bytes.length + (Character.BYTES * term.field().length()));
}
}
void addNumericUpdate(NumericDocValuesUpdate update, int docIDUpto) {
FieldUpdatesBuffer buffer = fieldUpdates.computeIfAbsent(update.field, k -> new FieldUpdatesBuffer(fieldUpdatesBytesUsed, update, docIDUpto));
if (update.hasValue) {
buffer.addUpdate(update.term, update.getValue(), docIDUpto);
} else {
buffer.addNoValue(update.term, docIDUpto);
}
numFieldUpdates.incrementAndGet();
}
void addBinaryUpdate(BinaryDocValuesUpdate update, int docIDUpto) {
FieldUpdatesBuffer buffer = fieldUpdates.computeIfAbsent(update.field, k -> new FieldUpdatesBuffer(fieldUpdatesBytesUsed, update, docIDUpto));
if (update.hasValue) {
buffer.addUpdate(update.term, update.getValue(), docIDUpto);
} else {
buffer.addNoValue(update.term, docIDUpto);
}
numFieldUpdates.incrementAndGet();
}
void clearDeleteTerms() {
deleteTerms.clear();
numTermDeletes.set(0);
}
void clear() {
deleteTerms.clear();
deleteQueries.clear();
deleteDocIDs.clear();
numTermDeletes.set(0);
numFieldUpdates.set(0);
fieldUpdates.clear();
bytesUsed.addAndGet(-bytesUsed.get());
fieldUpdatesBytesUsed.addAndGet(-fieldUpdatesBytesUsed.get());
}
boolean any() {
return deleteTerms.size() > 0 || deleteDocIDs.size() > 0 || deleteQueries.size() > 0 || numFieldUpdates.get() > 0;
}
@Override
public long ramBytesUsed() {
return bytesUsed.get() + fieldUpdatesBytesUsed.get();
}
void clearDeletedDocIds() {
deleteDocIDs.clear();
bytesUsed.addAndGet(-deleteDocIDs.size() * BufferedUpdates.BYTES_PER_DEL_DOCID);
}
}