mirror of
https://github.com/apache/lucene.git
synced 2025-03-07 00:39:21 +00:00
LUCENE-7868: use multiple threads to concurrently resolve deletes and DV udpates
This commit is contained in:
parent
1737fce5df
commit
58105a203a
@ -24,6 +24,10 @@ New Features
|
||||
the term frequency to this value. (Uwe Schindler, Robert Muir, Mike
|
||||
McCandless)
|
||||
|
||||
* LUCENE-7868: Multiple threads can now resolve deletes and doc values
|
||||
updates concurrently, giving sizable speedups in update-heavy
|
||||
indexing use cases (Simon Willnauer, Mike McCandless)
|
||||
|
||||
API Changes
|
||||
|
||||
* LUCENE-2605: Classic QueryParser no longer splits on whitespace by default.
|
||||
@ -76,6 +80,9 @@ API Changes
|
||||
|
||||
* LUCENE-7872: TopDocs.totalHits is now a long. (Adrien Grand, hossman)
|
||||
|
||||
* LUCENE-7868: IndexWriterConfig.setMaxBufferedDeleteTerms is
|
||||
removed. (Simon Willnauer, Mike McCandless)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-7626: IndexWriter will no longer accept broken token offsets
|
||||
|
@ -22,6 +22,7 @@ import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.InPlaceMergeSorter;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.apache.lucene.util.packed.PagedGrowableWriter;
|
||||
import org.apache.lucene.util.packed.PagedMutable;
|
||||
@ -35,22 +36,24 @@ import org.apache.lucene.util.packed.PagedMutable;
|
||||
class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
|
||||
final static class Iterator extends DocValuesFieldUpdates.Iterator {
|
||||
private final PagedGrowableWriter offsets;
|
||||
private final int size;
|
||||
private final PagedGrowableWriter offsets;
|
||||
private final PagedGrowableWriter lengths;
|
||||
private final PagedMutable docs;
|
||||
private long idx = 0; // long so we don't overflow if size == Integer.MAX_VALUE
|
||||
private int doc = -1;
|
||||
private final BytesRef value;
|
||||
private int offset, length;
|
||||
private final long delGen;
|
||||
|
||||
Iterator(int size, PagedGrowableWriter offsets, PagedGrowableWriter lengths,
|
||||
PagedMutable docs, BytesRef values) {
|
||||
PagedMutable docs, BytesRef values, long delGen) {
|
||||
this.offsets = offsets;
|
||||
this.size = size;
|
||||
this.lengths = lengths;
|
||||
this.docs = docs;
|
||||
value = values.clone();
|
||||
this.delGen = delGen;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -69,6 +72,7 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
doc = (int) docs.get(idx);
|
||||
++idx;
|
||||
while (idx < size && docs.get(idx) == doc) {
|
||||
// scan forward to last update to this doc
|
||||
++idx;
|
||||
}
|
||||
// idx points to the "next" element
|
||||
@ -87,10 +91,8 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
}
|
||||
|
||||
@Override
|
||||
void reset() {
|
||||
doc = -1;
|
||||
offset = -1;
|
||||
idx = 0;
|
||||
long delGen() {
|
||||
return delGen;
|
||||
}
|
||||
}
|
||||
|
||||
@ -100,18 +102,29 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
private int size;
|
||||
private final int bitsPerValue;
|
||||
|
||||
public BinaryDocValuesFieldUpdates(String field, int maxDoc) {
|
||||
super(field, DocValuesType.BINARY);
|
||||
public BinaryDocValuesFieldUpdates(long delGen, String field, int maxDoc) {
|
||||
super(maxDoc, delGen, field, DocValuesType.BINARY);
|
||||
bitsPerValue = PackedInts.bitsRequired(maxDoc - 1);
|
||||
docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
|
||||
offsets = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
|
||||
lengths = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
|
||||
values = new BytesRefBuilder();
|
||||
size = 0;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void add(int doc, Object value) {
|
||||
public int size() {
|
||||
return size;
|
||||
}
|
||||
|
||||
// NOTE: we fully consume the incoming BytesRef so caller is free to reuse it after we return:
|
||||
@Override
|
||||
synchronized public void add(int doc, Object value) {
|
||||
if (finished) {
|
||||
throw new IllegalStateException("already finished");
|
||||
}
|
||||
|
||||
assert doc < maxDoc: "doc=" + doc + " maxDoc=" + maxDoc;
|
||||
|
||||
// TODO: if the Sorter interface changes to take long indexes, we can remove that limitation
|
||||
if (size == Integer.MAX_VALUE) {
|
||||
throw new IllegalStateException("cannot support more than Integer.MAX_VALUE doc/value entries");
|
||||
@ -134,11 +147,19 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator iterator() {
|
||||
final PagedMutable docs = this.docs;
|
||||
final PagedGrowableWriter offsets = this.offsets;
|
||||
final PagedGrowableWriter lengths = this.lengths;
|
||||
final BytesRef values = this.values.get();
|
||||
public void finish() {
|
||||
if (finished) {
|
||||
throw new IllegalStateException("already finished");
|
||||
}
|
||||
finished = true;
|
||||
|
||||
// shrink wrap
|
||||
if (size < docs.size()) {
|
||||
docs = docs.resize(size);
|
||||
offsets = offsets.resize(size);
|
||||
lengths = lengths.resize(size);
|
||||
}
|
||||
|
||||
new InPlaceMergeSorter() {
|
||||
@Override
|
||||
protected void swap(int i, int j) {
|
||||
@ -157,36 +178,20 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
|
||||
@Override
|
||||
protected int compare(int i, int j) {
|
||||
int x = (int) docs.get(i);
|
||||
int y = (int) docs.get(j);
|
||||
return (x < y) ? -1 : ((x == y) ? 0 : 1);
|
||||
// increasing docID order:
|
||||
// NOTE: we can have ties here, when the same docID was updated in the same segment, in which case we rely on sort being
|
||||
// stable and preserving original order so the last update to that docID wins
|
||||
return Integer.compare((int) docs.get(i), (int) docs.get(j));
|
||||
}
|
||||
}.sort(0, size);
|
||||
|
||||
return new Iterator(size, offsets, lengths, docs, values);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void merge(DocValuesFieldUpdates other) {
|
||||
BinaryDocValuesFieldUpdates otherUpdates = (BinaryDocValuesFieldUpdates) other;
|
||||
if (otherUpdates.size > Integer.MAX_VALUE - size) {
|
||||
throw new IllegalStateException(
|
||||
"cannot support more than Integer.MAX_VALUE doc/value entries; size="
|
||||
+ size + " other.size=" + otherUpdates.size);
|
||||
public Iterator iterator() {
|
||||
if (finished == false) {
|
||||
throw new IllegalStateException("call finish first");
|
||||
}
|
||||
final int newSize = size + otherUpdates.size;
|
||||
docs = docs.grow(newSize);
|
||||
offsets = offsets.grow(newSize);
|
||||
lengths = lengths.grow(newSize);
|
||||
for (int i = 0; i < otherUpdates.size; i++) {
|
||||
int doc = (int) otherUpdates.docs.get(i);
|
||||
docs.set(size, doc);
|
||||
offsets.set(size, values.length() + otherUpdates.offsets.get(i)); // correct relative offset
|
||||
lengths.set(size, otherUpdates.lengths.get(i));
|
||||
++size;
|
||||
}
|
||||
|
||||
values.append(otherUpdates.values);
|
||||
return new Iterator(size, offsets, lengths, docs, values.get(), delGen);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -195,13 +200,13 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesPerDoc() {
|
||||
long bytesPerDoc = (long) Math.ceil((double) (bitsPerValue) / 8); // docs
|
||||
final int capacity = estimateCapacity(size);
|
||||
bytesPerDoc += (long) Math.ceil((double) offsets.ramBytesUsed() / capacity); // offsets
|
||||
bytesPerDoc += (long) Math.ceil((double) lengths.ramBytesUsed() / capacity); // lengths
|
||||
bytesPerDoc += (long) Math.ceil((double) values.length() / size); // values
|
||||
return bytesPerDoc;
|
||||
public long ramBytesUsed() {
|
||||
return offsets.ramBytesUsed()
|
||||
+ lengths.ramBytesUsed()
|
||||
+ docs.ramBytesUsed()
|
||||
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
|
||||
+ 4 * RamUsageEstimator.NUM_BYTES_INT
|
||||
+ 5 * RamUsageEstimator.NUM_BYTES_OBJECT_REF
|
||||
+ values.bytes().length;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -30,11 +30,12 @@ import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/* Holds buffered deletes and updates, by docID, term or query for a
|
||||
* single segment. This is used to hold buffered pending
|
||||
* deletes and updates against the to-be-flushed segment. Once the
|
||||
* deletes and updates are pushed (on flush in DocumentsWriter), they
|
||||
* are converted to a FrozenBufferedUpdates instance. */
|
||||
/** Holds buffered deletes and updates, by docID, term or query for a
|
||||
* single segment. This is used to hold buffered pending
|
||||
* deletes and updates against the to-be-flushed segment. Once the
|
||||
* deletes and updates are pushed (on flush in DocumentsWriter), they
|
||||
* are converted to a {@link FrozenBufferedUpdates} instance and
|
||||
* pushed to the {@link BufferedUpdatesStream}. */
|
||||
|
||||
// NOTE: instances of this class are accessed either via a private
|
||||
// instance on DocumentWriterPerThread, or via sync'd code by
|
||||
@ -128,10 +129,9 @@ class BufferedUpdates {
|
||||
final AtomicInteger numNumericUpdates = new AtomicInteger();
|
||||
final AtomicInteger numBinaryUpdates = new AtomicInteger();
|
||||
|
||||
// TODO: rename thes three: put "deleted" prefix in front:
|
||||
final Map<Term,Integer> terms = new HashMap<>();
|
||||
final Map<Query,Integer> queries = new HashMap<>();
|
||||
final List<Integer> docIDs = new ArrayList<>();
|
||||
final Map<Term,Integer> deleteTerms = new HashMap<>();
|
||||
final Map<Query,Integer> deleteQueries = new HashMap<>();
|
||||
final List<Integer> deleteDocIDs = new ArrayList<>();
|
||||
|
||||
// Map<dvField,Map<updateTerm,NumericUpdate>>
|
||||
// For each field we keep an ordered list of NumericUpdates, key'd by the
|
||||
@ -169,19 +169,19 @@ class BufferedUpdates {
|
||||
@Override
|
||||
public String toString() {
|
||||
if (VERBOSE_DELETES) {
|
||||
return "gen=" + gen + " numTerms=" + numTermDeletes + ", terms=" + terms
|
||||
+ ", queries=" + queries + ", docIDs=" + docIDs + ", numericUpdates=" + numericUpdates
|
||||
return "gen=" + gen + " numTerms=" + numTermDeletes + ", deleteTerms=" + deleteTerms
|
||||
+ ", deleteQueries=" + deleteQueries + ", deleteDocIDs=" + deleteDocIDs + ", numericUpdates=" + numericUpdates
|
||||
+ ", binaryUpdates=" + binaryUpdates + ", bytesUsed=" + bytesUsed;
|
||||
} else {
|
||||
String s = "gen=" + gen;
|
||||
if (numTermDeletes.get() != 0) {
|
||||
s += " " + numTermDeletes.get() + " deleted terms (unique count=" + terms.size() + ")";
|
||||
s += " " + numTermDeletes.get() + " deleted terms (unique count=" + deleteTerms.size() + ")";
|
||||
}
|
||||
if (queries.size() != 0) {
|
||||
s += " " + queries.size() + " deleted queries";
|
||||
if (deleteQueries.size() != 0) {
|
||||
s += " " + deleteQueries.size() + " deleted queries";
|
||||
}
|
||||
if (docIDs.size() != 0) {
|
||||
s += " " + docIDs.size() + " deleted docIDs";
|
||||
if (deleteDocIDs.size() != 0) {
|
||||
s += " " + deleteDocIDs.size() + " deleted docIDs";
|
||||
}
|
||||
if (numNumericUpdates.get() != 0) {
|
||||
s += " " + numNumericUpdates.get() + " numeric updates (unique count=" + numericUpdates.size() + ")";
|
||||
@ -198,7 +198,7 @@ class BufferedUpdates {
|
||||
}
|
||||
|
||||
public void addQuery(Query query, int docIDUpto) {
|
||||
Integer current = queries.put(query, docIDUpto);
|
||||
Integer current = deleteQueries.put(query, docIDUpto);
|
||||
// increment bytes used only if the query wasn't added so far.
|
||||
if (current == null) {
|
||||
bytesUsed.addAndGet(BYTES_PER_DEL_QUERY);
|
||||
@ -206,12 +206,12 @@ class BufferedUpdates {
|
||||
}
|
||||
|
||||
public void addDocID(int docID) {
|
||||
docIDs.add(Integer.valueOf(docID));
|
||||
deleteDocIDs.add(Integer.valueOf(docID));
|
||||
bytesUsed.addAndGet(BYTES_PER_DEL_DOCID);
|
||||
}
|
||||
|
||||
public void addTerm(Term term, int docIDUpto) {
|
||||
Integer current = terms.get(term);
|
||||
Integer current = deleteTerms.get(term);
|
||||
if (current != null && docIDUpto < current) {
|
||||
// Only record the new number if it's greater than the
|
||||
// current one. This is important because if multiple
|
||||
@ -223,7 +223,7 @@ class BufferedUpdates {
|
||||
return;
|
||||
}
|
||||
|
||||
terms.put(term, Integer.valueOf(docIDUpto));
|
||||
deleteTerms.put(term, Integer.valueOf(docIDUpto));
|
||||
// note that if current != null 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.
|
||||
@ -290,11 +290,16 @@ class BufferedUpdates {
|
||||
bytesUsed.addAndGet(BYTES_PER_BINARY_UPDATE_ENTRY + update.sizeInBytes());
|
||||
}
|
||||
}
|
||||
|
||||
void clearDeleteTerms() {
|
||||
deleteTerms.clear();
|
||||
numTermDeletes.set(0);
|
||||
}
|
||||
|
||||
void clear() {
|
||||
terms.clear();
|
||||
queries.clear();
|
||||
docIDs.clear();
|
||||
deleteTerms.clear();
|
||||
deleteQueries.clear();
|
||||
deleteDocIDs.clear();
|
||||
numericUpdates.clear();
|
||||
binaryUpdates.clear();
|
||||
numTermDeletes.set(0);
|
||||
@ -304,6 +309,6 @@ class BufferedUpdates {
|
||||
}
|
||||
|
||||
boolean any() {
|
||||
return terms.size() > 0 || docIDs.size() > 0 || queries.size() > 0 || numericUpdates.size() > 0 || binaryUpdates.size() > 0;
|
||||
return deleteTerms.size() > 0 || deleteDocIDs.size() > 0 || deleteQueries.size() > 0 || numericUpdates.size() > 0 || binaryUpdates.size() > 0;
|
||||
}
|
||||
}
|
||||
|
@ -14,40 +14,33 @@
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
/* Tracks the stream of {@link BufferedDeletes}.
|
||||
/** Tracks the stream of {@link FrozenBufferedUpdates}.
|
||||
* When DocumentsWriterPerThread flushes, its buffered
|
||||
* deletes and updates are appended to this stream. We later
|
||||
* apply them (resolve them to the actual
|
||||
* docIDs, per segment) when a merge is started
|
||||
* (only to the to-be-merged segments). We
|
||||
* also apply to all segments when NRT reader is pulled,
|
||||
* commit/close is called, or when too many deletes or updates are
|
||||
* deletes and updates are appended to this stream and immediately
|
||||
* resolved (to actual docIDs, per segment) using the indexing
|
||||
* thread that triggered the flush for concurrency. When a
|
||||
* merge kicks off, we sync to ensure all resolving packets
|
||||
* complete. We also apply to all segments when NRT reader is pulled,
|
||||
* commit/close is called, or when too many deletes or updates are
|
||||
* buffered and must be flushed (by RAM usage or by count).
|
||||
*
|
||||
* Each packet is assigned a generation, and each flushed or
|
||||
@ -57,23 +50,24 @@ import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
class BufferedUpdatesStream implements Accountable {
|
||||
|
||||
// TODO: maybe linked list?
|
||||
private final List<FrozenBufferedUpdates> updates = new ArrayList<>();
|
||||
private final Set<FrozenBufferedUpdates> updates = new HashSet<>();
|
||||
|
||||
// Starts at 1 so that SegmentInfos that have never had
|
||||
// deletes applied (whose bufferedDelGen defaults to 0)
|
||||
// will be correct:
|
||||
private long nextGen = 1;
|
||||
|
||||
// used only by assert
|
||||
private BytesRef lastDeleteTerm;
|
||||
|
||||
private final FinishedSegments finishedSegments;
|
||||
private final InfoStream infoStream;
|
||||
private final AtomicLong bytesUsed = new AtomicLong();
|
||||
private final AtomicInteger numTerms = new AtomicInteger();
|
||||
private final IndexWriter writer;
|
||||
private boolean closed;
|
||||
|
||||
public BufferedUpdatesStream(InfoStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
public BufferedUpdatesStream(IndexWriter writer) {
|
||||
this.writer = writer;
|
||||
this.infoStream = writer.infoStream;
|
||||
this.finishedSegments = new FinishedSegments(infoStream);
|
||||
}
|
||||
|
||||
// Appends a new packet of buffered deletes to the stream,
|
||||
@ -89,21 +83,27 @@ class BufferedUpdatesStream implements Accountable {
|
||||
packet.setDelGen(nextGen++);
|
||||
assert packet.any();
|
||||
assert checkDeleteStats();
|
||||
assert packet.delGen() < nextGen;
|
||||
assert updates.isEmpty() || updates.get(updates.size()-1).delGen() < packet.delGen() : "Delete packets must be in order";
|
||||
|
||||
updates.add(packet);
|
||||
numTerms.addAndGet(packet.numTermDeletes);
|
||||
bytesUsed.addAndGet(packet.bytesUsed);
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "push deletes " + packet + " segmentPrivate?=" + packet.isSegmentPrivate + " delGen=" + packet.delGen() + " packetCount=" + updates.size() + " totBytesUsed=" + bytesUsed.get());
|
||||
infoStream.message("BD", String.format(Locale.ROOT, "push new packet (%s), packetCount=%d, bytesUsed=%.3f MB", packet, updates.size(), bytesUsed.get()/1024./1024.));
|
||||
}
|
||||
assert checkDeleteStats();
|
||||
|
||||
return packet.delGen();
|
||||
}
|
||||
|
||||
public synchronized int getPendingUpdatesCount() {
|
||||
return updates.size();
|
||||
}
|
||||
|
||||
/** Only used by IW.rollback */
|
||||
public synchronized void clear() {
|
||||
updates.clear();
|
||||
nextGen = 1;
|
||||
finishedSegments.clear();
|
||||
numTerms.set(0);
|
||||
bytesUsed.set(0);
|
||||
}
|
||||
@ -121,253 +121,148 @@ class BufferedUpdatesStream implements Accountable {
|
||||
return bytesUsed.get();
|
||||
}
|
||||
|
||||
private synchronized void ensureOpen() {
|
||||
if (closed) {
|
||||
throw new AlreadyClosedException("already closed");
|
||||
}
|
||||
}
|
||||
|
||||
public static class ApplyDeletesResult {
|
||||
|
||||
// True if any actual deletes took place:
|
||||
public final boolean anyDeletes;
|
||||
|
||||
// Current gen, for the merged segment:
|
||||
public final long gen;
|
||||
|
||||
// If non-null, contains segments that are 100% deleted
|
||||
public final List<SegmentCommitInfo> allDeleted;
|
||||
|
||||
ApplyDeletesResult(boolean anyDeletes, long gen, List<SegmentCommitInfo> allDeleted) {
|
||||
ApplyDeletesResult(boolean anyDeletes, List<SegmentCommitInfo> allDeleted) {
|
||||
this.anyDeletes = anyDeletes;
|
||||
this.gen = gen;
|
||||
this.allDeleted = allDeleted;
|
||||
}
|
||||
}
|
||||
|
||||
// Sorts SegmentInfos from smallest to biggest bufferedDelGen:
|
||||
private static final Comparator<SegmentCommitInfo> sortSegInfoByDelGen = new Comparator<SegmentCommitInfo>() {
|
||||
@Override
|
||||
public int compare(SegmentCommitInfo si1, SegmentCommitInfo si2) {
|
||||
return Long.compare(si1.getBufferedDeletesGen(), si2.getBufferedDeletesGen());
|
||||
/** Waits for all in-flight packets, which are already being resolved concurrently
|
||||
* by indexing threads, to finish. Returns true if there were any
|
||||
* new deletes or updates. This is called for refresh, commit. */
|
||||
public void waitApplyAll() throws IOException {
|
||||
|
||||
assert Thread.holdsLock(writer) == false;
|
||||
|
||||
final long t0 = System.nanoTime();
|
||||
|
||||
Set<FrozenBufferedUpdates> waitFor;
|
||||
synchronized (this) {
|
||||
waitFor = new HashSet<>(updates);
|
||||
}
|
||||
};
|
||||
|
||||
waitApply(waitFor);
|
||||
}
|
||||
|
||||
/** Returns true if this delGen is still running. */
|
||||
public boolean stillRunning(long delGen) {
|
||||
return finishedSegments.stillRunning(delGen);
|
||||
}
|
||||
|
||||
public void finishedSegment(long delGen) {
|
||||
finishedSegments.finishedSegment(delGen);
|
||||
}
|
||||
|
||||
/** Resolves the buffered deleted Term/Query/docIDs, into
|
||||
* actual deleted docIDs in the liveDocs MutableBits for
|
||||
* each SegmentReader. */
|
||||
public synchronized ApplyDeletesResult applyDeletesAndUpdates(IndexWriter.ReaderPool pool, List<SegmentCommitInfo> infos) throws IOException {
|
||||
final long t0 = System.currentTimeMillis();
|
||||
/** Called by indexing threads once they are fully done resolving all deletes for the provided
|
||||
* delGen. We track the completed delGens and record the maximum delGen for which all prior
|
||||
* delGens, inclusive, are completed, so that it's safe for doc values updates to apply and write. */
|
||||
|
||||
final long gen = nextGen++;
|
||||
public synchronized void finished(FrozenBufferedUpdates packet) {
|
||||
// TODO: would be a bit more memory efficient to track this per-segment, so when each segment writes it writes all packets finished for
|
||||
// it, rather than only recording here, across all segments. But, more complex code, and more CPU, and maybe not so much impact in
|
||||
// practice?
|
||||
|
||||
if (infos.size() == 0) {
|
||||
return new ApplyDeletesResult(false, gen, null);
|
||||
packet.applied.countDown();
|
||||
|
||||
updates.remove(packet);
|
||||
numTerms.addAndGet(-packet.numTermDeletes);
|
||||
bytesUsed.addAndGet(-packet.bytesUsed);
|
||||
|
||||
finishedSegment(packet.delGen());
|
||||
}
|
||||
|
||||
/** All frozen packets up to and including this del gen are guaranteed to be finished. */
|
||||
public long getCompletedDelGen() {
|
||||
return finishedSegments.getCompletedDelGen();
|
||||
}
|
||||
|
||||
/** Waits only for those in-flight packets that apply to these merge segments. This is
|
||||
* called when a merge needs to finish and must ensure all deletes to the merging
|
||||
* segments are resolved. */
|
||||
public void waitApplyForMerge(List<SegmentCommitInfo> mergeInfos) throws IOException {
|
||||
assert Thread.holdsLock(writer) == false;
|
||||
|
||||
final long t0 = System.nanoTime();
|
||||
|
||||
long maxDelGen = Long.MIN_VALUE;
|
||||
for (SegmentCommitInfo info : mergeInfos) {
|
||||
maxDelGen = Math.max(maxDelGen, info.getBufferedDeletesGen());
|
||||
}
|
||||
|
||||
// We only init these on demand, when we find our first deletes that need to be applied:
|
||||
SegmentState[] segStates = null;
|
||||
|
||||
long totDelCount = 0;
|
||||
long totTermVisitedCount = 0;
|
||||
|
||||
boolean success = false;
|
||||
|
||||
ApplyDeletesResult result = null;
|
||||
|
||||
try {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT, "applyDeletes: open segment readers took %d msec", System.currentTimeMillis()-t0));
|
||||
}
|
||||
|
||||
assert checkDeleteStats();
|
||||
|
||||
if (!any()) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "applyDeletes: no segments; skipping");
|
||||
Set<FrozenBufferedUpdates> waitFor = new HashSet<>();
|
||||
synchronized (this) {
|
||||
for (FrozenBufferedUpdates packet : updates) {
|
||||
if (packet.delGen() <= maxDelGen) {
|
||||
// We must wait for this packet before finishing the merge because its
|
||||
// deletes apply to a subset of the segments being merged:
|
||||
waitFor.add(packet);
|
||||
}
|
||||
return new ApplyDeletesResult(false, gen, null);
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "applyDeletes: infos=" + infos + " packetCount=" + updates.size());
|
||||
}
|
||||
|
||||
infos = sortByDelGen(infos);
|
||||
|
||||
CoalescedUpdates coalescedUpdates = null;
|
||||
int infosIDX = infos.size()-1;
|
||||
int delIDX = updates.size()-1;
|
||||
|
||||
// Backwards merge sort the segment delGens with the packet delGens in the buffered stream:
|
||||
while (infosIDX >= 0) {
|
||||
final FrozenBufferedUpdates packet = delIDX >= 0 ? updates.get(delIDX) : null;
|
||||
final SegmentCommitInfo info = infos.get(infosIDX);
|
||||
final long segGen = info.getBufferedDeletesGen();
|
||||
|
||||
if (packet != null && segGen < packet.delGen()) {
|
||||
if (!packet.isSegmentPrivate && packet.any()) {
|
||||
/*
|
||||
* Only coalesce if we are NOT on a segment private del packet: the segment private del packet
|
||||
* must only apply to segments with the same delGen. Yet, if a segment is already deleted
|
||||
* from the SI since it had no more documents remaining after some del packets younger than
|
||||
* its segPrivate packet (higher delGen) have been applied, the segPrivate packet has not been
|
||||
* removed.
|
||||
*/
|
||||
if (coalescedUpdates == null) {
|
||||
coalescedUpdates = new CoalescedUpdates();
|
||||
}
|
||||
coalescedUpdates.update(packet);
|
||||
}
|
||||
|
||||
delIDX--;
|
||||
} else if (packet != null && segGen == packet.delGen()) {
|
||||
assert packet.isSegmentPrivate : "Packet and Segments deletegen can only match on a segment private del packet gen=" + segGen;
|
||||
|
||||
if (segStates == null) {
|
||||
segStates = openSegmentStates(pool, infos);
|
||||
}
|
||||
|
||||
SegmentState segState = segStates[infosIDX];
|
||||
|
||||
// Lock order: IW -> BD -> RP
|
||||
assert pool.infoIsLive(info);
|
||||
int delCount = 0;
|
||||
final DocValuesFieldUpdates.Container dvUpdates = new DocValuesFieldUpdates.Container();
|
||||
|
||||
// first apply segment-private deletes/updates
|
||||
delCount += applyQueryDeletes(packet.queriesIterable(), segState);
|
||||
applyDocValuesUpdates(Arrays.asList(packet.numericDVUpdates), segState, dvUpdates);
|
||||
applyDocValuesUpdates(Arrays.asList(packet.binaryDVUpdates), segState, dvUpdates);
|
||||
|
||||
// ... then coalesced deletes/updates, so that if there is an update that appears in both, the coalesced updates (carried from
|
||||
// updates ahead of the segment-privates ones) win:
|
||||
if (coalescedUpdates != null) {
|
||||
delCount += applyQueryDeletes(coalescedUpdates.queriesIterable(), segState);
|
||||
applyDocValuesUpdatesList(coalescedUpdates.numericDVUpdates, segState, dvUpdates);
|
||||
applyDocValuesUpdatesList(coalescedUpdates.binaryDVUpdates, segState, dvUpdates);
|
||||
}
|
||||
if (dvUpdates.any()) {
|
||||
segState.rld.writeFieldUpdates(info.info.dir, dvUpdates);
|
||||
}
|
||||
|
||||
totDelCount += delCount;
|
||||
|
||||
/*
|
||||
* Since we are on a segment private del packet we must not
|
||||
* update the coalescedUpdates here! We can simply advance to the
|
||||
* next packet and seginfo.
|
||||
*/
|
||||
delIDX--;
|
||||
infosIDX--;
|
||||
|
||||
} else {
|
||||
if (coalescedUpdates != null) {
|
||||
if (segStates == null) {
|
||||
segStates = openSegmentStates(pool, infos);
|
||||
}
|
||||
SegmentState segState = segStates[infosIDX];
|
||||
// Lock order: IW -> BD -> RP
|
||||
assert pool.infoIsLive(info);
|
||||
int delCount = 0;
|
||||
delCount += applyQueryDeletes(coalescedUpdates.queriesIterable(), segState);
|
||||
DocValuesFieldUpdates.Container dvUpdates = new DocValuesFieldUpdates.Container();
|
||||
applyDocValuesUpdatesList(coalescedUpdates.numericDVUpdates, segState, dvUpdates);
|
||||
applyDocValuesUpdatesList(coalescedUpdates.binaryDVUpdates, segState, dvUpdates);
|
||||
if (dvUpdates.any()) {
|
||||
segState.rld.writeFieldUpdates(info.info.dir, dvUpdates);
|
||||
}
|
||||
|
||||
totDelCount += delCount;
|
||||
}
|
||||
|
||||
infosIDX--;
|
||||
}
|
||||
}
|
||||
|
||||
// Now apply all term deletes:
|
||||
if (coalescedUpdates != null && coalescedUpdates.totalTermCount != 0) {
|
||||
if (segStates == null) {
|
||||
segStates = openSegmentStates(pool, infos);
|
||||
}
|
||||
totTermVisitedCount += applyTermDeletes(coalescedUpdates, segStates);
|
||||
}
|
||||
|
||||
assert checkDeleteStats();
|
||||
|
||||
success = true;
|
||||
|
||||
} finally {
|
||||
if (segStates != null) {
|
||||
result = closeSegmentStates(pool, segStates, success, gen);
|
||||
}
|
||||
}
|
||||
|
||||
if (result == null) {
|
||||
result = new ApplyDeletesResult(false, gen, null);
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "waitApplyForMerge: " + waitFor.size() + " packets, " + mergeInfos.size() + " merging segments");
|
||||
}
|
||||
|
||||
waitApply(waitFor);
|
||||
}
|
||||
|
||||
private void waitApply(Set<FrozenBufferedUpdates> waitFor) throws IOException {
|
||||
|
||||
long startNS = System.nanoTime();
|
||||
|
||||
int packetCount = waitFor.size();
|
||||
|
||||
if (waitFor.isEmpty()) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "waitApply: no deletes to apply");
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "waitApply: " + waitFor.size() + " packets: " + waitFor);
|
||||
}
|
||||
|
||||
long totalDelCount = 0;
|
||||
for (FrozenBufferedUpdates packet : waitFor) {
|
||||
// Frozen packets are now resolved, concurrently, by the indexing threads that
|
||||
// create them, by adding a DocumentsWriter.ResolveUpdatesEvent to the events queue,
|
||||
// but if we get here and the packet is not yet resolved, we resolve it now ourselves:
|
||||
packet.apply(writer);
|
||||
totalDelCount += packet.totalDelCount;
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD",
|
||||
String.format(Locale.ROOT,
|
||||
"applyDeletes took %d msec for %d segments, %d newly deleted docs (query deletes), %d visited terms, allDeleted=%s",
|
||||
System.currentTimeMillis()-t0, infos.size(), totDelCount, totTermVisitedCount, result.allDeleted));
|
||||
String.format(Locale.ROOT, "waitApply: done %d packets; totalDelCount=%d; totBytesUsed=%d; took %.2f msec",
|
||||
packetCount,
|
||||
totalDelCount,
|
||||
bytesUsed.get(),
|
||||
(System.nanoTime() - startNS) / 1000000.));
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
private List<SegmentCommitInfo> sortByDelGen(List<SegmentCommitInfo> infos) {
|
||||
infos = new ArrayList<>(infos);
|
||||
// Smaller delGens come first:
|
||||
Collections.sort(infos, sortSegInfoByDelGen);
|
||||
return infos;
|
||||
}
|
||||
|
||||
synchronized long getNextGen() {
|
||||
return nextGen++;
|
||||
}
|
||||
|
||||
// Lock order IW -> BD
|
||||
/* Removes any BufferedDeletes that we no longer need to
|
||||
* store because all segments in the index have had the
|
||||
* deletes applied. */
|
||||
public synchronized void prune(SegmentInfos segmentInfos) {
|
||||
assert checkDeleteStats();
|
||||
long minGen = Long.MAX_VALUE;
|
||||
for(SegmentCommitInfo info : segmentInfos) {
|
||||
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + updates.size());
|
||||
}
|
||||
final int limit = updates.size();
|
||||
for(int delIDX=0;delIDX<limit;delIDX++) {
|
||||
if (updates.get(delIDX).delGen() >= minGen) {
|
||||
prune(delIDX);
|
||||
assert checkDeleteStats();
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
// All deletes pruned
|
||||
prune(limit);
|
||||
assert !any();
|
||||
assert checkDeleteStats();
|
||||
}
|
||||
|
||||
private synchronized void prune(int count) {
|
||||
if (count > 0) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "pruneDeletes: prune " + count + " packets; " + (updates.size() - count) + " packets remain");
|
||||
}
|
||||
for(int delIDX=0;delIDX<count;delIDX++) {
|
||||
final FrozenBufferedUpdates packet = updates.get(delIDX);
|
||||
numTerms.addAndGet(-packet.numTermDeletes);
|
||||
assert numTerms.get() >= 0;
|
||||
bytesUsed.addAndGet(-packet.bytesUsed);
|
||||
assert bytesUsed.get() >= 0;
|
||||
}
|
||||
updates.subList(0, count).clear();
|
||||
}
|
||||
}
|
||||
|
||||
static class SegmentState {
|
||||
/** Holds all per-segment internal state used while resolving deletions. */
|
||||
public static final class SegmentState {
|
||||
final long delGen;
|
||||
final ReadersAndUpdates rld;
|
||||
final SegmentReader reader;
|
||||
@ -376,7 +271,6 @@ class BufferedUpdatesStream implements Accountable {
|
||||
TermsEnum termsEnum;
|
||||
PostingsEnum postingsEnum;
|
||||
BytesRef term;
|
||||
boolean any;
|
||||
|
||||
public SegmentState(IndexWriter.ReaderPool pool, SegmentCommitInfo info) throws IOException {
|
||||
rld = pool.get(info, true);
|
||||
@ -392,58 +286,54 @@ class BufferedUpdatesStream implements Accountable {
|
||||
pool.release(rld);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Does a merge sort by current term across all segments. */
|
||||
static class SegmentQueue extends PriorityQueue<SegmentState> {
|
||||
public SegmentQueue(int size) {
|
||||
super(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean lessThan(SegmentState a, SegmentState b) {
|
||||
return a.term.compareTo(b.term) < 0;
|
||||
public String toString() {
|
||||
return "SegmentState(" + rld.info + ")";
|
||||
}
|
||||
}
|
||||
|
||||
/** Opens SegmentReader and inits SegmentState for each segment. */
|
||||
private SegmentState[] openSegmentStates(IndexWriter.ReaderPool pool, List<SegmentCommitInfo> infos) throws IOException {
|
||||
int numReaders = infos.size();
|
||||
SegmentState[] segStates = new SegmentState[numReaders];
|
||||
public SegmentState[] openSegmentStates(IndexWriter.ReaderPool pool, List<SegmentCommitInfo> infos,
|
||||
Set<SegmentCommitInfo> alreadySeenSegments, long delGen) throws IOException {
|
||||
ensureOpen();
|
||||
|
||||
List<SegmentState> segStates = new ArrayList<>();
|
||||
boolean success = false;
|
||||
try {
|
||||
for(int i=0;i<numReaders;i++) {
|
||||
segStates[i] = new SegmentState(pool, infos.get(i));
|
||||
for (SegmentCommitInfo info : infos) {
|
||||
if (info.getBufferedDeletesGen() <= delGen && alreadySeenSegments.contains(info) == false) {
|
||||
segStates.add(new SegmentState(pool, info));
|
||||
alreadySeenSegments.add(info);
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success == false) {
|
||||
for(int j=0;j<numReaders;j++) {
|
||||
if (segStates[j] != null) {
|
||||
for(SegmentState segState : segStates) {
|
||||
try {
|
||||
segStates[j].finish(pool);
|
||||
segState.finish(pool);
|
||||
} catch (Throwable th) {
|
||||
// suppress so we keep throwing original exc
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return segStates;
|
||||
|
||||
return segStates.toArray(new SegmentState[0]);
|
||||
}
|
||||
|
||||
/** Close segment states previously opened with openSegmentStates. */
|
||||
private ApplyDeletesResult closeSegmentStates(IndexWriter.ReaderPool pool, SegmentState[] segStates, boolean success, long gen) throws IOException {
|
||||
int numReaders = segStates.length;
|
||||
public ApplyDeletesResult closeSegmentStates(IndexWriter.ReaderPool pool, SegmentState[] segStates, boolean success) throws IOException {
|
||||
int count = segStates.length;
|
||||
Throwable firstExc = null;
|
||||
List<SegmentCommitInfo> allDeleted = null;
|
||||
long totDelCount = 0;
|
||||
for (int j=0;j<numReaders;j++) {
|
||||
|
||||
for (int j=0;j<count;j++) {
|
||||
SegmentState segState = segStates[j];
|
||||
if (success) {
|
||||
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.maxDoc();
|
||||
if (fullDelCount == segState.rld.info.info.maxDoc()) {
|
||||
@ -469,276 +359,10 @@ class BufferedUpdatesStream implements Accountable {
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "applyDeletes: " + totDelCount + " new deleted documents");
|
||||
infoStream.message("BD", "closeSegmentStates: " + totDelCount + " new deleted documents; pool " + updates.size() + " packets; bytesUsed=" + pool.ramBytesUsed());
|
||||
}
|
||||
|
||||
return new ApplyDeletesResult(totDelCount > 0, gen, allDeleted);
|
||||
}
|
||||
|
||||
/** Merge sorts the deleted terms and all segments to resolve terms to docIDs for deletion. */
|
||||
private synchronized long applyTermDeletes(CoalescedUpdates updates, SegmentState[] segStates) throws IOException {
|
||||
|
||||
long startNS = System.nanoTime();
|
||||
|
||||
int numReaders = segStates.length;
|
||||
|
||||
long delTermVisitedCount = 0;
|
||||
long segTermVisitedCount = 0;
|
||||
|
||||
FieldTermIterator iter = updates.termIterator();
|
||||
|
||||
String field = null;
|
||||
SegmentQueue queue = null;
|
||||
|
||||
BytesRef term;
|
||||
|
||||
while ((term = iter.next()) != null) {
|
||||
|
||||
if (iter.field() != field) {
|
||||
// field changed
|
||||
field = iter.field();
|
||||
|
||||
queue = new SegmentQueue(numReaders);
|
||||
|
||||
long segTermCount = 0;
|
||||
for (SegmentState state : segStates) {
|
||||
Terms terms = state.reader.terms(field);
|
||||
if (terms != null) {
|
||||
segTermCount += terms.size();
|
||||
state.termsEnum = terms.iterator();
|
||||
state.term = state.termsEnum.next();
|
||||
if (state.term != null) {
|
||||
queue.add(state);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
assert checkDeleteTerm(null);
|
||||
}
|
||||
|
||||
assert checkDeleteTerm(term);
|
||||
|
||||
delTermVisitedCount++;
|
||||
|
||||
long delGen = iter.delGen();
|
||||
|
||||
while (queue.size() != 0) {
|
||||
|
||||
// Get next term merged across all segments
|
||||
SegmentState state = queue.top();
|
||||
segTermVisitedCount++;
|
||||
|
||||
int cmp = term.compareTo(state.term);
|
||||
|
||||
if (cmp < 0) {
|
||||
break;
|
||||
} else if (cmp == 0) {
|
||||
// fall through
|
||||
} else {
|
||||
TermsEnum.SeekStatus status = state.termsEnum.seekCeil(term);
|
||||
if (status == TermsEnum.SeekStatus.FOUND) {
|
||||
// fallthrough
|
||||
} else {
|
||||
if (status == TermsEnum.SeekStatus.NOT_FOUND) {
|
||||
state.term = state.termsEnum.term();
|
||||
queue.updateTop();
|
||||
} else {
|
||||
// No more terms in this segment
|
||||
queue.pop();
|
||||
}
|
||||
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
assert state.delGen != delGen;
|
||||
|
||||
if (state.delGen < delGen) {
|
||||
|
||||
// we don't need term frequencies for this
|
||||
final Bits acceptDocs = state.rld.getLiveDocs();
|
||||
state.postingsEnum = state.termsEnum.postings(state.postingsEnum, PostingsEnum.NONE);
|
||||
|
||||
assert state.postingsEnum != null;
|
||||
|
||||
while (true) {
|
||||
final int docID = state.postingsEnum.nextDoc();
|
||||
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
|
||||
break;
|
||||
}
|
||||
if (acceptDocs != null && acceptDocs.get(docID) == false) {
|
||||
continue;
|
||||
}
|
||||
if (!state.any) {
|
||||
state.rld.initWritableLiveDocs();
|
||||
state.any = true;
|
||||
}
|
||||
|
||||
// NOTE: there is no limit check on the docID
|
||||
// when deleting by Term (unlike by Query)
|
||||
// because on flush we apply all Term deletes to
|
||||
// each segment. So all Term deleting here is
|
||||
// against prior segments:
|
||||
state.rld.delete(docID);
|
||||
}
|
||||
}
|
||||
|
||||
state.term = state.termsEnum.next();
|
||||
if (state.term == null) {
|
||||
queue.pop();
|
||||
} else {
|
||||
queue.updateTop();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD",
|
||||
String.format(Locale.ROOT, "applyTermDeletes took %.1f msec for %d segments and %d packets; %d del terms visited; %d seg terms visited",
|
||||
(System.nanoTime()-startNS)/1000000.,
|
||||
numReaders,
|
||||
updates.terms.size(),
|
||||
delTermVisitedCount, segTermVisitedCount));
|
||||
}
|
||||
|
||||
return delTermVisitedCount;
|
||||
}
|
||||
|
||||
private synchronized void applyDocValuesUpdatesList(List<List<DocValuesUpdate>> updates,
|
||||
SegmentState segState, DocValuesFieldUpdates.Container dvUpdatesContainer) throws IOException {
|
||||
// we walk backwards through the segments, appending deletion packets to the coalesced updates, so we must apply the packets in reverse
|
||||
// so that newer packets override older ones:
|
||||
for(int idx=updates.size()-1;idx>=0;idx--) {
|
||||
applyDocValuesUpdates(updates.get(idx), segState, dvUpdatesContainer);
|
||||
}
|
||||
}
|
||||
|
||||
// DocValues updates
|
||||
private synchronized void applyDocValuesUpdates(List<DocValuesUpdate> updates,
|
||||
SegmentState segState, DocValuesFieldUpdates.Container dvUpdatesContainer) throws IOException {
|
||||
|
||||
// TODO: we can process the updates per DV field, from last to first so that
|
||||
// if multiple terms affect same document for the same field, we add an update
|
||||
// only once (that of the last term). To do that, we can keep a bitset which
|
||||
// marks which documents have already been updated. So e.g. if term T1
|
||||
// updates doc 7, and then we process term T2 and it updates doc 7 as well,
|
||||
// we don't apply the update since we know T1 came last and therefore wins
|
||||
// the update.
|
||||
// We can also use that bitset as 'liveDocs' to pass to TermEnum.docs(), so
|
||||
// that these documents aren't even returned.
|
||||
|
||||
String currentField = null;
|
||||
TermsEnum termsEnum = null;
|
||||
PostingsEnum postingsEnum = null;
|
||||
|
||||
for (DocValuesUpdate update : updates) {
|
||||
Term term = update.term;
|
||||
int limit = update.docIDUpto;
|
||||
|
||||
// TODO: we traverse the terms in update order (not term order) so that we
|
||||
// apply the updates in the correct order, i.e. if two terms udpate the
|
||||
// same document, the last one that came in wins, irrespective of the
|
||||
// terms lexical order.
|
||||
// we can apply the updates in terms order if we keep an updatesGen (and
|
||||
// increment it with every update) and attach it to each NumericUpdate. Note
|
||||
// that we cannot rely only on docIDUpto because an app may send two updates
|
||||
// which will get same docIDUpto, yet will still need to respect the order
|
||||
// those updates arrived.
|
||||
|
||||
if (!term.field().equals(currentField)) {
|
||||
// if we change the code to process updates in terms order, enable this assert
|
||||
// assert currentField == null || currentField.compareTo(term.field()) < 0;
|
||||
currentField = term.field();
|
||||
Terms terms = segState.reader.terms(currentField);
|
||||
if (terms != null) {
|
||||
termsEnum = terms.iterator();
|
||||
} else {
|
||||
termsEnum = null;
|
||||
}
|
||||
}
|
||||
|
||||
if (termsEnum == null) {
|
||||
// no terms in this field
|
||||
continue;
|
||||
}
|
||||
|
||||
if (termsEnum.seekExact(term.bytes())) {
|
||||
// we don't need term frequencies for this
|
||||
final Bits acceptDocs = segState.rld.getLiveDocs();
|
||||
postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
|
||||
|
||||
DocValuesFieldUpdates dvUpdates = dvUpdatesContainer.getUpdates(update.field, update.type);
|
||||
if (dvUpdates == null) {
|
||||
dvUpdates = dvUpdatesContainer.newUpdates(update.field, update.type, segState.reader.maxDoc());
|
||||
}
|
||||
int doc;
|
||||
while ((doc = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
if (doc >= limit) {
|
||||
break; // no more docs that can be updated for this term
|
||||
}
|
||||
if (acceptDocs != null && acceptDocs.get(doc) == false) {
|
||||
continue;
|
||||
}
|
||||
dvUpdates.add(doc, update.value);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class QueryAndLimit {
|
||||
public final Query query;
|
||||
public final int limit;
|
||||
public QueryAndLimit(Query query, int limit) {
|
||||
this.query = query;
|
||||
this.limit = limit;
|
||||
}
|
||||
}
|
||||
|
||||
// Delete by query
|
||||
private static long applyQueryDeletes(Iterable<QueryAndLimit> queriesIter, SegmentState segState) throws IOException {
|
||||
long delCount = 0;
|
||||
final LeafReaderContext readerContext = segState.reader.getContext();
|
||||
for (QueryAndLimit ent : queriesIter) {
|
||||
Query query = ent.query;
|
||||
int limit = ent.limit;
|
||||
final IndexSearcher searcher = new IndexSearcher(readerContext.reader());
|
||||
searcher.setQueryCache(null);
|
||||
final Weight weight = searcher.createNormalizedWeight(query, false);
|
||||
final Scorer scorer = weight.scorer(readerContext);
|
||||
if (scorer != null) {
|
||||
final DocIdSetIterator it = scorer.iterator();
|
||||
final Bits liveDocs = readerContext.reader().getLiveDocs();
|
||||
while (true) {
|
||||
int doc = it.nextDoc();
|
||||
if (doc >= limit) {
|
||||
break;
|
||||
}
|
||||
if (liveDocs != null && liveDocs.get(doc) == false) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!segState.any) {
|
||||
segState.rld.initWritableLiveDocs();
|
||||
segState.any = true;
|
||||
}
|
||||
if (segState.rld.delete(doc)) {
|
||||
delCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return delCount;
|
||||
}
|
||||
|
||||
// used only by assert
|
||||
private boolean checkDeleteTerm(BytesRef term) {
|
||||
if (term != null) {
|
||||
assert lastDeleteTerm == null || term.compareTo(lastDeleteTerm) >= 0: "lastTerm=" + lastDeleteTerm + " vs term=" + term;
|
||||
}
|
||||
// TODO: we re-use term now in our merged iterable, but we shouldn't clone, instead copy for this assert
|
||||
lastDeleteTerm = term == null ? null : BytesRef.deepCopyOf(term);
|
||||
return true;
|
||||
return new ApplyDeletesResult(totDelCount > 0, allDeleted);
|
||||
}
|
||||
|
||||
// only for assert
|
||||
@ -753,4 +377,52 @@ class BufferedUpdatesStream implements Accountable {
|
||||
assert bytesUsed2 == bytesUsed.get(): "bytesUsed2=" + bytesUsed2 + " vs " + bytesUsed;
|
||||
return true;
|
||||
}
|
||||
|
||||
/** Tracks the contiguous range of packets that have finished resolving. We need this because the packets
|
||||
* are concurrently resolved, and we can only write to disk the contiguous completed
|
||||
* packets. */
|
||||
private static class FinishedSegments {
|
||||
|
||||
/** Largest del gen, inclusive, for which all prior packets have finished applying. */
|
||||
private long completedDelGen;
|
||||
|
||||
/** This lets us track the "holes" in the current frontier of applying del
|
||||
* gens; once the holes are filled in we can advance completedDelGen. */
|
||||
private final Set<Long> finishedDelGens = new HashSet<>();
|
||||
|
||||
private final InfoStream infoStream;
|
||||
|
||||
public FinishedSegments(InfoStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
}
|
||||
|
||||
public synchronized void clear() {
|
||||
finishedDelGens.clear();
|
||||
completedDelGen = 0;
|
||||
}
|
||||
|
||||
public synchronized boolean stillRunning(long delGen) {
|
||||
return delGen > completedDelGen && finishedDelGens.contains(delGen) == false;
|
||||
}
|
||||
|
||||
public synchronized long getCompletedDelGen() {
|
||||
return completedDelGen;
|
||||
}
|
||||
|
||||
public synchronized void finishedSegment(long delGen) {
|
||||
finishedDelGens.add(delGen);
|
||||
while (true) {
|
||||
if (finishedDelGens.contains(completedDelGen + 1)) {
|
||||
finishedDelGens.remove(completedDelGen + 1);
|
||||
completedDelGen++;
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "finished packet delGen=" + delGen + " now completedDelGen=" + completedDelGen);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1,109 +0,0 @@
|
||||
/*
|
||||
* 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.lucene.index;
|
||||
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.BufferedUpdatesStream.QueryAndLimit;
|
||||
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.BytesRef;
|
||||
|
||||
class CoalescedUpdates {
|
||||
final Map<Query,Integer> queries = new HashMap<>();
|
||||
final List<PrefixCodedTerms> terms = new ArrayList<>();
|
||||
final List<List<DocValuesUpdate>> numericDVUpdates = new ArrayList<>();
|
||||
final List<List<DocValuesUpdate>> binaryDVUpdates = new ArrayList<>();
|
||||
long totalTermCount;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
// note: we could add/collect more debugging information
|
||||
return "CoalescedUpdates(termSets=" + terms.size()
|
||||
+ ",totalTermCount=" + totalTermCount
|
||||
+ ",queries=" + queries.size() + ",numericDVUpdates=" + numericDVUpdates.size()
|
||||
+ ",binaryDVUpdates=" + binaryDVUpdates.size() + ")";
|
||||
}
|
||||
|
||||
void update(FrozenBufferedUpdates in) {
|
||||
totalTermCount += in.terms.size();
|
||||
terms.add(in.terms);
|
||||
|
||||
for (int queryIdx = 0; queryIdx < in.queries.length; queryIdx++) {
|
||||
final Query query = in.queries[queryIdx];
|
||||
queries.put(query, BufferedUpdates.MAX_INT);
|
||||
}
|
||||
|
||||
List<DocValuesUpdate> numericPacket = new ArrayList<>();
|
||||
numericDVUpdates.add(numericPacket);
|
||||
for (NumericDocValuesUpdate nu : in.numericDVUpdates) {
|
||||
NumericDocValuesUpdate clone = new NumericDocValuesUpdate(nu.term, nu.field, (Long) nu.value);
|
||||
clone.docIDUpto = Integer.MAX_VALUE;
|
||||
numericPacket.add(clone);
|
||||
}
|
||||
|
||||
List<DocValuesUpdate> binaryPacket = new ArrayList<>();
|
||||
binaryDVUpdates.add(binaryPacket);
|
||||
for (BinaryDocValuesUpdate bu : in.binaryDVUpdates) {
|
||||
BinaryDocValuesUpdate clone = new BinaryDocValuesUpdate(bu.term, bu.field, (BytesRef) bu.value);
|
||||
clone.docIDUpto = Integer.MAX_VALUE;
|
||||
binaryPacket.add(clone);
|
||||
}
|
||||
}
|
||||
|
||||
public FieldTermIterator termIterator() {
|
||||
if (terms.size() == 1) {
|
||||
return terms.get(0).iterator();
|
||||
} else {
|
||||
return new MergedPrefixCodedTermsIterator(terms);
|
||||
}
|
||||
}
|
||||
|
||||
public Iterable<QueryAndLimit> queriesIterable() {
|
||||
return new Iterable<QueryAndLimit>() {
|
||||
|
||||
@Override
|
||||
public Iterator<QueryAndLimit> iterator() {
|
||||
return new Iterator<QueryAndLimit>() {
|
||||
private final Iterator<Map.Entry<Query,Integer>> iter = queries.entrySet().iterator();
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return iter.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryAndLimit next() {
|
||||
final Map.Entry<Query,Integer> ent = iter.next();
|
||||
return new QueryAndLimit(ent.getKey(), ent.getValue());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
@ -16,15 +16,13 @@
|
||||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.util.packed.PagedGrowableWriter;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
||||
|
||||
/**
|
||||
* Holds updates of a single DocValues field, for a set of documents.
|
||||
* Holds updates of a single DocValues field, for a set of documents within one segment.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
@ -54,100 +52,114 @@ abstract class DocValuesFieldUpdates {
|
||||
* {@code null} value means that it was unset for this document.
|
||||
*/
|
||||
abstract Object value();
|
||||
|
||||
/**
|
||||
* Reset the iterator's state. Should be called before {@link #nextDoc()}
|
||||
* and {@link #value()}.
|
||||
*/
|
||||
abstract void reset();
|
||||
|
||||
|
||||
/** Returns delGen for this packet. */
|
||||
abstract long delGen();
|
||||
}
|
||||
|
||||
static class Container {
|
||||
|
||||
final Map<String,NumericDocValuesFieldUpdates> numericDVUpdates = new HashMap<>();
|
||||
final Map<String,BinaryDocValuesFieldUpdates> binaryDVUpdates = new HashMap<>();
|
||||
|
||||
boolean any() {
|
||||
for (NumericDocValuesFieldUpdates updates : numericDVUpdates.values()) {
|
||||
if (updates.any()) {
|
||||
return true;
|
||||
/** Merge-sorts multiple iterators, one per delGen, favoring the largest delGen that has updates for a given docID. */
|
||||
public static Iterator mergedIterator(Iterator[] subs) {
|
||||
|
||||
if (subs.length == 1) {
|
||||
return subs[0];
|
||||
}
|
||||
|
||||
PriorityQueue<Iterator> queue = new PriorityQueue<Iterator>(subs.length) {
|
||||
@Override
|
||||
protected boolean lessThan(Iterator a, Iterator b) {
|
||||
// sort by smaller docID
|
||||
int cmp = Integer.compare(a.doc(), b.doc());
|
||||
if (cmp == 0) {
|
||||
// then by larger delGen
|
||||
cmp = Long.compare(b.delGen(), a.delGen());
|
||||
|
||||
// delGens are unique across our subs:
|
||||
assert cmp != 0;
|
||||
}
|
||||
|
||||
return cmp < 0;
|
||||
}
|
||||
};
|
||||
|
||||
for (Iterator sub : subs) {
|
||||
if (sub.nextDoc() != NO_MORE_DOCS) {
|
||||
queue.add(sub);
|
||||
}
|
||||
for (BinaryDocValuesFieldUpdates updates : binaryDVUpdates.values()) {
|
||||
if (updates.any()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (queue.size() == 0) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return new Iterator() {
|
||||
private int doc;
|
||||
|
||||
private boolean first = true;
|
||||
|
||||
@Override
|
||||
public int nextDoc() {
|
||||
// TODO: can we do away with this first boolean?
|
||||
if (first == false) {
|
||||
// Advance all sub iterators past current doc
|
||||
while (true) {
|
||||
if (queue.size() == 0) {
|
||||
doc = NO_MORE_DOCS;
|
||||
break;
|
||||
}
|
||||
int newDoc = queue.top().doc();
|
||||
if (newDoc != doc) {
|
||||
assert newDoc > doc: "doc=" + doc + " newDoc=" + newDoc;
|
||||
doc = newDoc;
|
||||
break;
|
||||
}
|
||||
if (queue.top().nextDoc() == NO_MORE_DOCS) {
|
||||
queue.pop();
|
||||
} else {
|
||||
queue.updateTop();
|
||||
}
|
||||
}
|
||||
} else {
|
||||
doc = queue.top().doc();
|
||||
first = false;
|
||||
}
|
||||
return doc;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
int size() {
|
||||
return numericDVUpdates.size() + binaryDVUpdates.size();
|
||||
}
|
||||
|
||||
long ramBytesPerDoc() {
|
||||
long ramBytesPerDoc = 0;
|
||||
for (NumericDocValuesFieldUpdates updates : numericDVUpdates.values()) {
|
||||
ramBytesPerDoc += updates.ramBytesPerDoc();
|
||||
|
||||
@Override
|
||||
public int doc() {
|
||||
return doc;
|
||||
}
|
||||
for (BinaryDocValuesFieldUpdates updates : binaryDVUpdates.values()) {
|
||||
ramBytesPerDoc += updates.ramBytesPerDoc();
|
||||
|
||||
@Override
|
||||
public Object value() {
|
||||
return queue.top().value();
|
||||
}
|
||||
return ramBytesPerDoc;
|
||||
}
|
||||
|
||||
DocValuesFieldUpdates getUpdates(String field, DocValuesType type) {
|
||||
switch (type) {
|
||||
case NUMERIC:
|
||||
return numericDVUpdates.get(field);
|
||||
case BINARY:
|
||||
return binaryDVUpdates.get(field);
|
||||
default:
|
||||
throw new IllegalArgumentException("unsupported type: " + type);
|
||||
|
||||
@Override
|
||||
public long delGen() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
DocValuesFieldUpdates newUpdates(String field, DocValuesType type, int maxDoc) {
|
||||
switch (type) {
|
||||
case NUMERIC:
|
||||
assert numericDVUpdates.get(field) == null;
|
||||
NumericDocValuesFieldUpdates numericUpdates = new NumericDocValuesFieldUpdates(field, maxDoc);
|
||||
numericDVUpdates.put(field, numericUpdates);
|
||||
return numericUpdates;
|
||||
case BINARY:
|
||||
assert binaryDVUpdates.get(field) == null;
|
||||
BinaryDocValuesFieldUpdates binaryUpdates = new BinaryDocValuesFieldUpdates(field, maxDoc);
|
||||
binaryDVUpdates.put(field, binaryUpdates);
|
||||
return binaryUpdates;
|
||||
default:
|
||||
throw new IllegalArgumentException("unsupported type: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "numericDVUpdates=" + numericDVUpdates + " binaryDVUpdates=" + binaryDVUpdates;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
final String field;
|
||||
final DocValuesType type;
|
||||
|
||||
protected DocValuesFieldUpdates(String field, DocValuesType type) {
|
||||
final long delGen;
|
||||
protected boolean finished;
|
||||
protected final int maxDoc;
|
||||
|
||||
protected DocValuesFieldUpdates(int maxDoc, long delGen, String field, DocValuesType type) {
|
||||
this.maxDoc = maxDoc;
|
||||
this.delGen = delGen;
|
||||
this.field = field;
|
||||
if (type == null) {
|
||||
throw new NullPointerException("DocValuesType must not be null");
|
||||
}
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the estimated capacity of a {@link PagedGrowableWriter} given the
|
||||
* actual number of stored elements.
|
||||
*/
|
||||
protected static int estimateCapacity(int size) {
|
||||
return (int) Math.ceil((double) size / PAGE_SIZE) * PAGE_SIZE;
|
||||
|
||||
public boolean getFinished() {
|
||||
return finished;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -160,19 +172,17 @@ abstract class DocValuesFieldUpdates {
|
||||
* Returns an {@link Iterator} over the updated documents and their
|
||||
* values.
|
||||
*/
|
||||
// TODO: also use this for merging, instead of having to write through to disk first
|
||||
public abstract Iterator iterator();
|
||||
|
||||
/**
|
||||
* Merge with another {@link DocValuesFieldUpdates}. This is called for a
|
||||
* segment which received updates while it was being merged. The given updates
|
||||
* should override whatever updates are in that instance.
|
||||
*/
|
||||
public abstract void merge(DocValuesFieldUpdates other);
|
||||
|
||||
/** Freezes internal data structures and sorts updates by docID for efficient iteration. */
|
||||
public abstract void finish();
|
||||
|
||||
/** Returns true if this instance contains any updates. */
|
||||
public abstract boolean any();
|
||||
|
||||
/** Returns approximate RAM bytes used per document. */
|
||||
public abstract long ramBytesPerDoc();
|
||||
/** Returns approximate RAM bytes used. */
|
||||
public abstract long ramBytesUsed();
|
||||
|
||||
public abstract int size();
|
||||
}
|
||||
|
@ -85,7 +85,6 @@ abstract class DocValuesUpdate {
|
||||
long valueSizeInBytes() {
|
||||
return RAW_VALUE_SIZE_IN_BYTES + ((BytesRef) value).bytes.length;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/** An in-place update to a numeric DocValues field */
|
||||
@ -99,7 +98,5 @@ abstract class DocValuesUpdate {
|
||||
long valueSizeInBytes() {
|
||||
return Long.BYTES;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -107,7 +107,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
private final AtomicInteger numDocsInRAM = new AtomicInteger(0);
|
||||
|
||||
// TODO: cut over to BytesRefHash in BufferedDeletes
|
||||
volatile DocumentsWriterDeleteQueue deleteQueue = new DocumentsWriterDeleteQueue();
|
||||
volatile DocumentsWriterDeleteQueue deleteQueue;
|
||||
private final DocumentsWriterFlushQueue ticketQueue = new DocumentsWriterFlushQueue();
|
||||
/*
|
||||
* we preserve changes during a full flush since IW might not checkout before
|
||||
@ -129,6 +129,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
this.directory = directory;
|
||||
this.config = config;
|
||||
this.infoStream = config.getInfoStream();
|
||||
this.deleteQueue = new DocumentsWriterDeleteQueue(infoStream);
|
||||
this.perThreadPool = config.getIndexerThreadPool();
|
||||
flushPolicy = config.getFlushPolicy();
|
||||
this.writer = writer;
|
||||
@ -141,10 +142,10 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
|
||||
long seqNo = deleteQueue.addDelete(queries);
|
||||
flushControl.doOnDelete();
|
||||
lastSeqNo = Math.max(lastSeqNo, seqNo);
|
||||
if (applyAllDeletes(deleteQueue)) {
|
||||
seqNo = -seqNo;
|
||||
}
|
||||
lastSeqNo = Math.max(lastSeqNo, seqNo);
|
||||
return seqNo;
|
||||
}
|
||||
|
||||
@ -160,10 +161,10 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
|
||||
long seqNo = deleteQueue.addDelete(terms);
|
||||
flushControl.doOnDelete();
|
||||
lastSeqNo = Math.max(lastSeqNo, seqNo);
|
||||
if (applyAllDeletes(deleteQueue)) {
|
||||
seqNo = -seqNo;
|
||||
}
|
||||
lastSeqNo = Math.max(lastSeqNo, seqNo);
|
||||
return seqNo;
|
||||
}
|
||||
|
||||
@ -171,20 +172,21 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
|
||||
long seqNo = deleteQueue.addDocValuesUpdates(updates);
|
||||
flushControl.doOnDelete();
|
||||
lastSeqNo = Math.max(lastSeqNo, seqNo);
|
||||
if (applyAllDeletes(deleteQueue)) {
|
||||
seqNo = -seqNo;
|
||||
}
|
||||
lastSeqNo = Math.max(lastSeqNo, seqNo);
|
||||
return seqNo;
|
||||
}
|
||||
|
||||
DocumentsWriterDeleteQueue currentDeleteSession() {
|
||||
return deleteQueue;
|
||||
}
|
||||
|
||||
|
||||
/** If buffered deletes are using too much heap, resolve them and write disk and return true. */
|
||||
private boolean applyAllDeletes(DocumentsWriterDeleteQueue deleteQueue) throws IOException {
|
||||
if (flushControl.getAndResetApplyAllDeletes()) {
|
||||
if (deleteQueue != null && !flushControl.isFullFlush()) {
|
||||
if (deleteQueue != null) {
|
||||
ticketQueue.addDeletes(deleteQueue);
|
||||
}
|
||||
putEvent(ApplyDeletesEvent.INSTANCE); // apply deletes event forces a purge
|
||||
@ -200,7 +202,6 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
return ticketQueue.tryPurge(writer);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/** Returns how many docs are currently buffered in RAM. */
|
||||
int getNumDocs() {
|
||||
@ -246,11 +247,13 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
}
|
||||
|
||||
/** Returns how many documents were aborted. */
|
||||
synchronized long lockAndAbortAll(IndexWriter indexWriter) {
|
||||
synchronized long lockAndAbortAll(IndexWriter indexWriter) throws IOException {
|
||||
assert indexWriter.holdsFullFlushLock();
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", "lockAndAbortAll");
|
||||
}
|
||||
// Make sure we move all pending tickets into the flush queue:
|
||||
ticketQueue.forcePurge(indexWriter);
|
||||
long abortedDocCount = 0;
|
||||
boolean success = false;
|
||||
try {
|
||||
@ -578,9 +581,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
|
||||
flushingDWPT = flushControl.nextPendingFlush();
|
||||
}
|
||||
if (hasEvents) {
|
||||
putEvent(MergePendingEvent.INSTANCE);
|
||||
}
|
||||
|
||||
// If deletes alone are consuming > 1/2 our RAM
|
||||
// buffer, force them all to apply now. This is to
|
||||
// prevent too-frequent flushing of a long tail of
|
||||
@ -589,9 +590,9 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
|
||||
flushControl.getDeleteBytesUsed() > (1024*1024*ramBufferSizeMB/2)) {
|
||||
hasEvents = true;
|
||||
if (!this.applyAllDeletes(deleteQueue)) {
|
||||
if (applyAllDeletes(deleteQueue) == false) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", String.format(Locale.ROOT, "force apply deletes bytesUsed=%.1f MB vs ramBuffer=%.1f MB",
|
||||
infoStream.message("DW", String.format(Locale.ROOT, "force apply deletes after flush bytesUsed=%.1f MB vs ramBuffer=%.1f MB",
|
||||
flushControl.getDeleteBytesUsed()/(1024.*1024.),
|
||||
ramBufferSizeMB));
|
||||
}
|
||||
@ -654,7 +655,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
}
|
||||
// If a concurrent flush is still in flight wait for it
|
||||
flushControl.waitForFlush();
|
||||
if (!anythingFlushed && flushingDeleteQueue.anyChanges()) { // apply deletes if we did not flush any document
|
||||
if (anythingFlushed == false && flushingDeleteQueue.anyChanges()) { // apply deletes if we did not flush any document
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", Thread.currentThread().getName() + ": flush naked frozen global deletes");
|
||||
}
|
||||
@ -695,7 +696,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
return config;
|
||||
}
|
||||
|
||||
private void putEvent(Event event) {
|
||||
void putEvent(Event event) {
|
||||
events.add(event);
|
||||
}
|
||||
|
||||
@ -704,6 +705,25 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
return flushControl.ramBytesUsed();
|
||||
}
|
||||
|
||||
static final class ResolveUpdatesEvent implements Event {
|
||||
|
||||
private final FrozenBufferedUpdates packet;
|
||||
|
||||
ResolveUpdatesEvent(FrozenBufferedUpdates packet) {
|
||||
this.packet = packet;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(IndexWriter writer, boolean triggerMerge, boolean forcePurge) throws IOException {
|
||||
try {
|
||||
packet.apply(writer);
|
||||
} catch (Throwable t) {
|
||||
writer.tragicEvent(t, "applyUpdatesPacket");
|
||||
}
|
||||
writer.flushDeletesCount.incrementAndGet();
|
||||
}
|
||||
}
|
||||
|
||||
static final class ApplyDeletesEvent implements Event {
|
||||
static final Event INSTANCE = new ApplyDeletesEvent();
|
||||
private int instCount = 0;
|
||||
@ -717,21 +737,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||
writer.applyDeletesAndPurge(true); // we always purge!
|
||||
}
|
||||
}
|
||||
|
||||
static final class MergePendingEvent implements Event {
|
||||
static final Event INSTANCE = new MergePendingEvent();
|
||||
private int instCount = 0;
|
||||
private MergePendingEvent() {
|
||||
assert instCount == 0;
|
||||
instCount++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(IndexWriter writer, boolean triggerMerge, boolean forcePurge) throws IOException {
|
||||
writer.doAfterSegmentFlushed(triggerMerge, forcePurge);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
static final class ForcedPurgeEvent implements Event {
|
||||
static final Event INSTANCE = new ForcedPurgeEvent();
|
||||
private int instCount = 0;
|
||||
|
@ -16,6 +16,7 @@
|
||||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
@ -25,6 +26,7 @@ import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/**
|
||||
* {@link DocumentsWriterDeleteQueue} is a non-blocking linked pending deletes
|
||||
@ -85,19 +87,22 @@ final class DocumentsWriterDeleteQueue implements Accountable {
|
||||
/** Generates the sequence number that IW returns to callers changing the index, showing the effective serialization of all operations. */
|
||||
private final AtomicLong nextSeqNo;
|
||||
|
||||
private final InfoStream infoStream;
|
||||
|
||||
// for asserts
|
||||
long maxSeqNo = Long.MAX_VALUE;
|
||||
|
||||
DocumentsWriterDeleteQueue() {
|
||||
DocumentsWriterDeleteQueue(InfoStream infoStream) {
|
||||
// seqNo must start at 1 because some APIs negate this to also return a boolean
|
||||
this(0, 1);
|
||||
this(infoStream, 0, 1);
|
||||
}
|
||||
|
||||
DocumentsWriterDeleteQueue(long generation, long startSeqNo) {
|
||||
this(new BufferedUpdates("global"), generation, startSeqNo);
|
||||
DocumentsWriterDeleteQueue(InfoStream infoStream, long generation, long startSeqNo) {
|
||||
this(infoStream, new BufferedUpdates("global"), generation, startSeqNo);
|
||||
}
|
||||
|
||||
DocumentsWriterDeleteQueue(BufferedUpdates globalBufferedUpdates, long generation, long startSeqNo) {
|
||||
DocumentsWriterDeleteQueue(InfoStream infoStream, BufferedUpdates globalBufferedUpdates, long generation, long startSeqNo) {
|
||||
this.infoStream = infoStream;
|
||||
this.globalBufferedUpdates = globalBufferedUpdates;
|
||||
this.generation = generation;
|
||||
this.nextSeqNo = new AtomicLong(startSeqNo);
|
||||
@ -189,7 +194,7 @@ final class DocumentsWriterDeleteQueue implements Accountable {
|
||||
}
|
||||
}
|
||||
|
||||
FrozenBufferedUpdates freezeGlobalBuffer(DeleteSlice callerSlice) {
|
||||
FrozenBufferedUpdates freezeGlobalBuffer(DeleteSlice callerSlice) throws IOException {
|
||||
globalBufferLock.lock();
|
||||
/*
|
||||
* Here we freeze the global buffer so we need to lock it, apply all
|
||||
@ -209,9 +214,13 @@ final class DocumentsWriterDeleteQueue implements Accountable {
|
||||
globalSlice.apply(globalBufferedUpdates, BufferedUpdates.MAX_INT);
|
||||
}
|
||||
|
||||
final FrozenBufferedUpdates packet = new FrozenBufferedUpdates(globalBufferedUpdates, false);
|
||||
globalBufferedUpdates.clear();
|
||||
return packet;
|
||||
if (globalBufferedUpdates.any()) {
|
||||
final FrozenBufferedUpdates packet = new FrozenBufferedUpdates(infoStream, globalBufferedUpdates, null);
|
||||
globalBufferedUpdates.clear();
|
||||
return packet;
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
} finally {
|
||||
globalBufferLock.unlock();
|
||||
}
|
||||
@ -426,7 +435,7 @@ final class DocumentsWriterDeleteQueue implements Accountable {
|
||||
globalBufferLock.lock();
|
||||
try {
|
||||
forceApplyGlobalSlice();
|
||||
return globalBufferedUpdates.terms.size();
|
||||
return globalBufferedUpdates.deleteTerms.size();
|
||||
} finally {
|
||||
globalBufferLock.unlock();
|
||||
}
|
||||
|
@ -424,22 +424,16 @@ final class DocumentsWriterFlushControl implements Accountable {
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
|
||||
synchronized void doOnDelete() {
|
||||
// pass null this is a global delete no update
|
||||
flushPolicy.onDelete(this, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of delete terms in the global pool
|
||||
*/
|
||||
public int getNumGlobalTermDeletes() {
|
||||
return documentsWriter.deleteQueue.numGlobalTermDeletes() + bufferedUpdatesStream.numTerms();
|
||||
}
|
||||
|
||||
/** Returns heap bytes currently consumed by buffered deletes/updates that would be
|
||||
* freed if we pushed all deletes. This does not include bytes consumed by
|
||||
* already pushed delete/update packets. */
|
||||
public long getDeleteBytesUsed() {
|
||||
return documentsWriter.deleteQueue.ramBytesUsed() + bufferedUpdatesStream.ramBytesUsed();
|
||||
return documentsWriter.deleteQueue.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -501,7 +495,7 @@ final class DocumentsWriterFlushControl implements Accountable {
|
||||
seqNo = documentsWriter.deleteQueue.getLastSequenceNumber() + perThreadPool.getActiveThreadStateCount() + 2;
|
||||
flushingQueue.maxSeqNo = seqNo+1;
|
||||
|
||||
DocumentsWriterDeleteQueue newQueue = new DocumentsWriterDeleteQueue(flushingQueue.generation+1, seqNo+1);
|
||||
DocumentsWriterDeleteQueue newQueue = new DocumentsWriterDeleteQueue(infoStream, flushingQueue.generation+1, seqNo+1);
|
||||
|
||||
documentsWriter.deleteQueue = newQueue;
|
||||
}
|
||||
@ -648,8 +642,7 @@ final class DocumentsWriterFlushControl implements Accountable {
|
||||
}
|
||||
for (BlockedFlush blockedFlush : blockedFlushes) {
|
||||
try {
|
||||
flushingWriters
|
||||
.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
|
||||
flushingWriters.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
|
||||
documentsWriter.subtractFlushedNumDocs(blockedFlush.dwpt.getNumDocsInRAM());
|
||||
blockedFlush.dwpt.abort();
|
||||
} catch (Throwable ex) {
|
||||
@ -720,6 +713,4 @@ final class DocumentsWriterFlushControl implements Accountable {
|
||||
public InfoStream getInfoStream() {
|
||||
return infoStream;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -60,7 +60,7 @@ class DocumentsWriterFlushQueue {
|
||||
assert numTickets >= 0;
|
||||
}
|
||||
|
||||
synchronized SegmentFlushTicket addFlushTicket(DocumentsWriterPerThread dwpt) {
|
||||
synchronized SegmentFlushTicket addFlushTicket(DocumentsWriterPerThread dwpt) throws IOException {
|
||||
// Each flush is assigned a ticket in the order they acquire the ticketQueue
|
||||
// lock
|
||||
incTickets();
|
||||
@ -168,11 +168,11 @@ class DocumentsWriterFlushQueue {
|
||||
protected boolean published = false;
|
||||
|
||||
protected FlushTicket(FrozenBufferedUpdates frozenUpdates) {
|
||||
assert frozenUpdates != null;
|
||||
this.frozenUpdates = frozenUpdates;
|
||||
}
|
||||
|
||||
protected abstract void publish(IndexWriter writer) throws IOException;
|
||||
|
||||
protected abstract boolean canPublish();
|
||||
|
||||
/**
|
||||
@ -186,33 +186,31 @@ class DocumentsWriterFlushQueue {
|
||||
assert newSegment != null;
|
||||
assert newSegment.segmentInfo != null;
|
||||
final FrozenBufferedUpdates segmentUpdates = newSegment.segmentUpdates;
|
||||
//System.out.println("FLUSH: " + newSegment.segmentInfo.info.name);
|
||||
if (indexWriter.infoStream.isEnabled("DW")) {
|
||||
indexWriter.infoStream.message("DW", "publishFlushedSegment seg-private updates=" + segmentUpdates);
|
||||
indexWriter.infoStream.message("DW", "publishFlushedSegment seg-private updates=" + segmentUpdates);
|
||||
}
|
||||
|
||||
if (segmentUpdates != null && indexWriter.infoStream.isEnabled("DW")) {
|
||||
indexWriter.infoStream.message("DW", "flush: push buffered seg private updates: " + segmentUpdates);
|
||||
indexWriter.infoStream.message("DW", "flush: push buffered seg private updates: " + segmentUpdates);
|
||||
}
|
||||
// now publish!
|
||||
indexWriter.publishFlushedSegment(newSegment.segmentInfo, segmentUpdates, globalPacket);
|
||||
indexWriter.publishFlushedSegment(newSegment.segmentInfo, segmentUpdates, globalPacket, newSegment.sortMap);
|
||||
}
|
||||
|
||||
protected final void finishFlush(IndexWriter indexWriter, FlushedSegment newSegment, FrozenBufferedUpdates bufferedUpdates)
|
||||
throws IOException {
|
||||
// Finish the flushed segment and publish it to IndexWriter
|
||||
if (newSegment == null) {
|
||||
assert bufferedUpdates != null;
|
||||
if (bufferedUpdates != null && bufferedUpdates.any()) {
|
||||
indexWriter.publishFrozenUpdates(bufferedUpdates);
|
||||
if (indexWriter.infoStream.isEnabled("DW")) {
|
||||
indexWriter.infoStream.message("DW", "flush: push buffered updates: " + bufferedUpdates);
|
||||
}
|
||||
// Finish the flushed segment and publish it to IndexWriter
|
||||
if (newSegment == null) {
|
||||
if (bufferedUpdates != null && bufferedUpdates.any()) {
|
||||
indexWriter.publishFrozenUpdates(bufferedUpdates);
|
||||
if (indexWriter.infoStream.isEnabled("DW")) {
|
||||
indexWriter.infoStream.message("DW", "flush: push buffered updates: " + bufferedUpdates);
|
||||
}
|
||||
} else {
|
||||
publishFlushedSegment(indexWriter, newSegment, bufferedUpdates);
|
||||
}
|
||||
} else {
|
||||
publishFlushedSegment(indexWriter, newSegment, bufferedUpdates);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static final class GlobalDeletesTicket extends FlushTicket {
|
||||
@ -220,6 +218,7 @@ class DocumentsWriterFlushQueue {
|
||||
protected GlobalDeletesTicket(FrozenBufferedUpdates frozenUpdates) {
|
||||
super(frozenUpdates);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void publish(IndexWriter writer) throws IOException {
|
||||
assert !published : "ticket was already publised - can not publish twice";
|
||||
@ -264,4 +263,4 @@ class DocumentsWriterFlushQueue {
|
||||
return segment != null || failed;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -96,15 +96,18 @@ class DocumentsWriterPerThread {
|
||||
final FieldInfos fieldInfos;
|
||||
final FrozenBufferedUpdates segmentUpdates;
|
||||
final MutableBits liveDocs;
|
||||
final Sorter.DocMap sortMap;
|
||||
final int delCount;
|
||||
|
||||
private FlushedSegment(SegmentCommitInfo segmentInfo, FieldInfos fieldInfos,
|
||||
BufferedUpdates segmentUpdates, MutableBits liveDocs, int delCount) {
|
||||
private FlushedSegment(InfoStream infoStream, SegmentCommitInfo segmentInfo, FieldInfos fieldInfos,
|
||||
BufferedUpdates segmentUpdates, MutableBits liveDocs, int delCount, Sorter.DocMap sortMap)
|
||||
throws IOException {
|
||||
this.segmentInfo = segmentInfo;
|
||||
this.fieldInfos = fieldInfos;
|
||||
this.segmentUpdates = segmentUpdates != null && segmentUpdates.any() ? new FrozenBufferedUpdates(segmentUpdates, true) : null;
|
||||
this.segmentUpdates = segmentUpdates != null && segmentUpdates.any() ? new FrozenBufferedUpdates(infoStream, segmentUpdates, segmentInfo) : null;
|
||||
this.liveDocs = liveDocs;
|
||||
this.delCount = delCount;
|
||||
this.sortMap = sortMap;
|
||||
}
|
||||
}
|
||||
|
||||
@ -142,7 +145,7 @@ class DocumentsWriterPerThread {
|
||||
SegmentWriteState flushState;
|
||||
// Updates for our still-in-RAM (to be flushed next) segment
|
||||
final BufferedUpdates pendingUpdates;
|
||||
private final SegmentInfo segmentInfo; // Current segment we are working on
|
||||
final SegmentInfo segmentInfo; // Current segment we are working on
|
||||
boolean aborted = false; // True if we aborted
|
||||
|
||||
private final FieldInfos.Builder fieldInfos;
|
||||
@ -395,7 +398,7 @@ class DocumentsWriterPerThread {
|
||||
* {@link DocumentsWriterDeleteQueue}s global buffer and apply all pending
|
||||
* deletes to this DWPT.
|
||||
*/
|
||||
FrozenBufferedUpdates prepareFlush() {
|
||||
FrozenBufferedUpdates prepareFlush() throws IOException {
|
||||
assert numDocsInRAM > 0;
|
||||
final FrozenBufferedUpdates globalUpdates = deleteQueue.freezeGlobalBuffer(deleteSlice);
|
||||
/* deleteSlice can possibly be null if we have hit non-aborting exceptions during indexing and never succeeded
|
||||
@ -421,14 +424,14 @@ class DocumentsWriterPerThread {
|
||||
// Apply delete-by-docID now (delete-byDocID only
|
||||
// happens when an exception is hit processing that
|
||||
// doc, eg if analyzer has some problem w/ the text):
|
||||
if (pendingUpdates.docIDs.size() > 0) {
|
||||
if (pendingUpdates.deleteDocIDs.size() > 0) {
|
||||
flushState.liveDocs = codec.liveDocsFormat().newLiveDocs(numDocsInRAM);
|
||||
for(int delDocID : pendingUpdates.docIDs) {
|
||||
for(int delDocID : pendingUpdates.deleteDocIDs) {
|
||||
flushState.liveDocs.clear(delDocID);
|
||||
}
|
||||
flushState.delCountOnFlush = pendingUpdates.docIDs.size();
|
||||
pendingUpdates.bytesUsed.addAndGet(-pendingUpdates.docIDs.size() * BufferedUpdates.BYTES_PER_DEL_DOCID);
|
||||
pendingUpdates.docIDs.clear();
|
||||
flushState.delCountOnFlush = pendingUpdates.deleteDocIDs.size();
|
||||
pendingUpdates.bytesUsed.addAndGet(-pendingUpdates.deleteDocIDs.size() * BufferedUpdates.BYTES_PER_DEL_DOCID);
|
||||
pendingUpdates.deleteDocIDs.clear();
|
||||
}
|
||||
|
||||
if (aborted) {
|
||||
@ -446,7 +449,8 @@ class DocumentsWriterPerThread {
|
||||
final Sorter.DocMap sortMap;
|
||||
try {
|
||||
sortMap = consumer.flush(flushState);
|
||||
pendingUpdates.terms.clear();
|
||||
// We clear this here because we already resolved them (private to this segment) when writing postings:
|
||||
pendingUpdates.clearDeleteTerms();
|
||||
segmentInfo.setFiles(new HashSet<>(directory.getCreatedFiles()));
|
||||
|
||||
final SegmentCommitInfo segmentInfoPerCommit = new SegmentCommitInfo(segmentInfo, 0, -1L, -1L, -1L);
|
||||
@ -463,7 +467,7 @@ class DocumentsWriterPerThread {
|
||||
}
|
||||
|
||||
final BufferedUpdates segmentDeletes;
|
||||
if (pendingUpdates.queries.isEmpty() && pendingUpdates.numericUpdates.isEmpty() && pendingUpdates.binaryUpdates.isEmpty()) {
|
||||
if (pendingUpdates.deleteQueries.isEmpty() && pendingUpdates.numericUpdates.isEmpty() && pendingUpdates.binaryUpdates.isEmpty()) {
|
||||
pendingUpdates.clear();
|
||||
segmentDeletes = null;
|
||||
} else {
|
||||
@ -480,13 +484,14 @@ class DocumentsWriterPerThread {
|
||||
|
||||
assert segmentInfo != null;
|
||||
|
||||
FlushedSegment fs = new FlushedSegment(segmentInfoPerCommit, flushState.fieldInfos,
|
||||
segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush);
|
||||
FlushedSegment fs = new FlushedSegment(infoStream, segmentInfoPerCommit, flushState.fieldInfos,
|
||||
segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush,
|
||||
sortMap);
|
||||
sealFlushedSegment(fs, sortMap);
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "flush time " + ((System.nanoTime() - t0)/1000000.0) + " msec");
|
||||
}
|
||||
|
||||
|
||||
return fs;
|
||||
} catch (Throwable th) {
|
||||
abort();
|
||||
|
@ -16,11 +16,12 @@
|
||||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
|
||||
/**
|
||||
* {@link DocumentsWriterPerThreadPool} controls {@link ThreadState} instances
|
||||
* and their thread assignments during indexing. Each {@link ThreadState} holds
|
||||
@ -163,6 +164,9 @@ final class DocumentsWriterPerThreadPool {
|
||||
// don't recycle DWPT by default
|
||||
}
|
||||
|
||||
// TODO: maybe we should try to do load leveling here: we want roughly even numbers
|
||||
// of items (docs, deletes, DV updates) to most take advantage of concurrency while flushing
|
||||
|
||||
/** This method is used by DocumentsWriter/FlushControl to obtain a ThreadState to do an indexing operation (add/updateDocument). */
|
||||
ThreadState getAndLock(Thread requestingThread, DocumentsWriter documentsWriter) {
|
||||
ThreadState threadState = null;
|
||||
@ -208,9 +212,8 @@ final class DocumentsWriterPerThreadPool {
|
||||
state.unlock();
|
||||
synchronized (this) {
|
||||
freeList.add(state);
|
||||
// In case any thread is waiting, wake one of them up since we just released a thread state; notify() should be sufficient but we do
|
||||
// notifyAll defensively:
|
||||
notifyAll();
|
||||
// In case any thread is waiting, wake one of them up since we just released a thread state:
|
||||
notify();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -29,8 +29,7 @@ import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
|
||||
* <li>
|
||||
* {@link #onDelete(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
|
||||
* - applies pending delete operations based on the global number of buffered
|
||||
* delete terms iff {@link IndexWriterConfig#getMaxBufferedDeleteTerms()} is
|
||||
* enabled</li>
|
||||
* delete terms if the consumed memory is greater than {@link IndexWriterConfig#getRAMBufferSizeMB()}</li>.
|
||||
* <li>
|
||||
* {@link #onInsert(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
|
||||
* - flushes either on the number of documents per
|
||||
@ -60,21 +59,12 @@ class FlushByRamOrCountsPolicy extends FlushPolicy {
|
||||
|
||||
@Override
|
||||
public void onDelete(DocumentsWriterFlushControl control, ThreadState state) {
|
||||
if (flushOnDeleteTerms()) {
|
||||
// Flush this state by num del terms
|
||||
final int maxBufferedDeleteTerms = indexWriterConfig
|
||||
.getMaxBufferedDeleteTerms();
|
||||
if (control.getNumGlobalTermDeletes() >= maxBufferedDeleteTerms) {
|
||||
control.setApplyAllDeletes();
|
||||
if ((flushOnRAM() && control.getDeleteBytesUsed() > 1024*1024*indexWriterConfig.getRAMBufferSizeMB())) {
|
||||
control.setApplyAllDeletes();
|
||||
if (infoStream.isEnabled("FP")) {
|
||||
infoStream.message("FP", "force apply deletes bytesUsed=" + control.getDeleteBytesUsed() + " vs ramBufferMB=" + indexWriterConfig.getRAMBufferSizeMB());
|
||||
}
|
||||
}
|
||||
if ((flushOnRAM() &&
|
||||
control.getDeleteBytesUsed() > (1024*1024*indexWriterConfig.getRAMBufferSizeMB()))) {
|
||||
control.setApplyAllDeletes();
|
||||
if (infoStream.isEnabled("FP")) {
|
||||
infoStream.message("FP", "force apply deletes bytesUsed=" + control.getDeleteBytesUsed() + " vs ramBufferMB=" + indexWriterConfig.getRAMBufferSizeMB());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -114,15 +104,6 @@ class FlushByRamOrCountsPolicy extends FlushPolicy {
|
||||
return indexWriterConfig.getMaxBufferedDocs() != IndexWriterConfig.DISABLE_AUTO_FLUSH;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if this {@link FlushPolicy} flushes on
|
||||
* {@link IndexWriterConfig#getMaxBufferedDeleteTerms()}, otherwise
|
||||
* <code>false</code>.
|
||||
*/
|
||||
protected boolean flushOnDeleteTerms() {
|
||||
return indexWriterConfig.getMaxBufferedDeleteTerms() != IndexWriterConfig.DISABLE_AUTO_FLUSH;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if this {@link FlushPolicy} flushes on
|
||||
* {@link IndexWriterConfig#getRAMBufferSizeMB()}, otherwise
|
||||
|
@ -33,9 +33,6 @@ import org.apache.lucene.util.InfoStream;
|
||||
* <li>Number of RAM resident documents - configured via
|
||||
* {@link IndexWriterConfig#setMaxBufferedDocs(int)}</li>
|
||||
* </ul>
|
||||
* The policy also applies pending delete operations (by term and/or query),
|
||||
* given the threshold set in
|
||||
* {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)}.
|
||||
* <p>
|
||||
* {@link IndexWriter} consults the provided {@link FlushPolicy} to control the
|
||||
* flushing process. The policy is informed for each added or updated document
|
||||
|
@ -34,11 +34,10 @@ final class FreqProxTermsWriter extends TermsHash {
|
||||
}
|
||||
|
||||
private void applyDeletes(SegmentWriteState state, Fields fields) throws IOException {
|
||||
|
||||
// Process any pending Term deletes for this newly
|
||||
// flushed segment:
|
||||
if (state.segUpdates != null && state.segUpdates.terms.size() > 0) {
|
||||
Map<Term,Integer> segDeletes = state.segUpdates.terms;
|
||||
if (state.segUpdates != null && state.segUpdates.deleteTerms.size() > 0) {
|
||||
Map<Term,Integer> segDeletes = state.segUpdates.deleteTerms;
|
||||
List<Term> deleteTerms = new ArrayList<>(segDeletes.keySet());
|
||||
Collections.sort(deleteTerms);
|
||||
String lastField = null;
|
||||
|
@ -16,167 +16,846 @@
|
||||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import org.apache.lucene.index.BufferedUpdatesStream.QueryAndLimit;
|
||||
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
||||
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
||||
import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* Holds buffered deletes and updates by term or query, once pushed. Pushed
|
||||
* deletes/updates are write-once, so we shift to more memory efficient data
|
||||
* structure to hold them. We don't hold docIDs because these are applied on
|
||||
* structure to hold them. We don't hold docIDs because these are applied on
|
||||
* flush.
|
||||
*/
|
||||
class FrozenBufferedUpdates {
|
||||
|
||||
/* NOTE: we now apply this frozen packet immediately on creation, yet this process is heavy, and runs
|
||||
* in multiple threads, and this compression is sizable (~8.3% of the original size), so it's important
|
||||
* we run this before applying the deletes/updates. */
|
||||
|
||||
/* Query we often undercount (say 24 bytes), plus int. */
|
||||
final static int BYTES_PER_DEL_QUERY = RamUsageEstimator.NUM_BYTES_OBJECT_REF + Integer.BYTES + 24;
|
||||
|
||||
// Terms, in sorted order:
|
||||
final PrefixCodedTerms terms;
|
||||
final PrefixCodedTerms deleteTerms;
|
||||
|
||||
// Parallel array of deleted query, and the docIDUpto for each
|
||||
final Query[] queries;
|
||||
final int[] queryLimits;
|
||||
final Query[] deleteQueries;
|
||||
final int[] deleteQueryLimits;
|
||||
|
||||
// numeric DV update term and their updates
|
||||
final NumericDocValuesUpdate[] numericDVUpdates;
|
||||
final byte[] numericDVUpdates;
|
||||
|
||||
// binary DV update term and their updates
|
||||
final BinaryDocValuesUpdate[] binaryDVUpdates;
|
||||
final byte[] binaryDVUpdates;
|
||||
|
||||
private int numericDVUpdateCount;
|
||||
private int binaryDVUpdateCount;
|
||||
|
||||
/** Counts down once all deletes/updates have been applied */
|
||||
public final CountDownLatch applied = new CountDownLatch(1);
|
||||
|
||||
/** How many total documents were deleted/updated. */
|
||||
public long totalDelCount;
|
||||
|
||||
final int bytesUsed;
|
||||
final int numTermDeletes;
|
||||
private long gen = -1; // assigned by BufferedUpdatesStream once pushed
|
||||
final boolean isSegmentPrivate; // set to true iff this frozen packet represents
|
||||
|
||||
private long delGen = -1; // assigned by BufferedUpdatesStream once pushed
|
||||
|
||||
final SegmentCommitInfo privateSegment; // non-null iff this frozen packet represents
|
||||
// a segment private deletes. in that case is should
|
||||
// only have Queries
|
||||
// only have Queries and doc values updates
|
||||
private final InfoStream infoStream;
|
||||
|
||||
|
||||
public FrozenBufferedUpdates(BufferedUpdates deletes, boolean isSegmentPrivate) {
|
||||
this.isSegmentPrivate = isSegmentPrivate;
|
||||
assert !isSegmentPrivate || deletes.terms.size() == 0 : "segment private package should only have del queries";
|
||||
Term termsArray[] = deletes.terms.keySet().toArray(new Term[deletes.terms.size()]);
|
||||
public FrozenBufferedUpdates(InfoStream infoStream, BufferedUpdates updates, SegmentCommitInfo privateSegment) throws IOException {
|
||||
this.infoStream = infoStream;
|
||||
this.privateSegment = privateSegment;
|
||||
assert updates.deleteDocIDs.isEmpty();
|
||||
assert privateSegment == null || updates.deleteTerms.isEmpty() : "segment private packet should only have del queries";
|
||||
Term termsArray[] = updates.deleteTerms.keySet().toArray(new Term[updates.deleteTerms.size()]);
|
||||
ArrayUtil.timSort(termsArray);
|
||||
PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
|
||||
for (Term term : termsArray) {
|
||||
builder.add(term);
|
||||
}
|
||||
terms = builder.finish();
|
||||
deleteTerms = builder.finish();
|
||||
|
||||
queries = new Query[deletes.queries.size()];
|
||||
queryLimits = new int[deletes.queries.size()];
|
||||
deleteQueries = new Query[updates.deleteQueries.size()];
|
||||
deleteQueryLimits = new int[updates.deleteQueries.size()];
|
||||
int upto = 0;
|
||||
for(Map.Entry<Query,Integer> ent : deletes.queries.entrySet()) {
|
||||
queries[upto] = ent.getKey();
|
||||
queryLimits[upto] = ent.getValue();
|
||||
for(Map.Entry<Query,Integer> ent : updates.deleteQueries.entrySet()) {
|
||||
deleteQueries[upto] = ent.getKey();
|
||||
deleteQueryLimits[upto] = ent.getValue();
|
||||
upto++;
|
||||
}
|
||||
|
||||
// TODO if a Term affects multiple fields, we could keep the updates key'd by Term
|
||||
// so that it maps to all fields it affects, sorted by their docUpto, and traverse
|
||||
// that Term only once, applying the update to all fields that still need to be
|
||||
// updated.
|
||||
List<NumericDocValuesUpdate> allNumericUpdates = new ArrayList<>();
|
||||
int numericUpdatesSize = 0;
|
||||
for (LinkedHashMap<Term,NumericDocValuesUpdate> numericUpdates : deletes.numericUpdates.values()) {
|
||||
for (NumericDocValuesUpdate update : numericUpdates.values()) {
|
||||
allNumericUpdates.add(update);
|
||||
numericUpdatesSize += update.sizeInBytes();
|
||||
}
|
||||
}
|
||||
numericDVUpdates = allNumericUpdates.toArray(new NumericDocValuesUpdate[allNumericUpdates.size()]);
|
||||
// updated.
|
||||
numericDVUpdates = freezeNumericDVUpdates(updates.numericUpdates);
|
||||
|
||||
// TODO if a Term affects multiple fields, we could keep the updates key'd by Term
|
||||
// so that it maps to all fields it affects, sorted by their docUpto, and traverse
|
||||
// that Term only once, applying the update to all fields that still need to be
|
||||
// updated.
|
||||
List<BinaryDocValuesUpdate> allBinaryUpdates = new ArrayList<>();
|
||||
int binaryUpdatesSize = 0;
|
||||
for (LinkedHashMap<Term,BinaryDocValuesUpdate> binaryUpdates : deletes.binaryUpdates.values()) {
|
||||
for (BinaryDocValuesUpdate update : binaryUpdates.values()) {
|
||||
allBinaryUpdates.add(update);
|
||||
binaryUpdatesSize += update.sizeInBytes();
|
||||
binaryDVUpdates = freezeBinaryDVUpdates(updates.binaryUpdates);
|
||||
|
||||
bytesUsed = (int) (deleteTerms.ramBytesUsed() + deleteQueries.length * BYTES_PER_DEL_QUERY
|
||||
+ numericDVUpdates.length + binaryDVUpdates.length);
|
||||
|
||||
numTermDeletes = updates.numTermDeletes.get();
|
||||
if (infoStream != null && infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT,
|
||||
"compressed %d to %d bytes (%.2f%%) for deletes/updates; private segment %s",
|
||||
updates.bytesUsed.get(), bytesUsed, 100.*bytesUsed/updates.bytesUsed.get(),
|
||||
privateSegment));
|
||||
}
|
||||
}
|
||||
|
||||
private byte[] freezeNumericDVUpdates(Map<String,LinkedHashMap<Term,NumericDocValuesUpdate>> numericDVUpdates)
|
||||
throws IOException {
|
||||
// TODO: we could do better here, e.g. collate the updates by field
|
||||
// so if you are updating 2 fields interleaved we don't keep writing the field strings
|
||||
|
||||
RAMOutputStream out = new RAMOutputStream();
|
||||
String lastTermField = null;
|
||||
String lastUpdateField = null;
|
||||
for (LinkedHashMap<Term,NumericDocValuesUpdate> numericUpdates : numericDVUpdates.values()) {
|
||||
numericDVUpdateCount += numericUpdates.size();
|
||||
for (NumericDocValuesUpdate update : numericUpdates.values()) {
|
||||
|
||||
int code = update.term.bytes().length << 2;
|
||||
|
||||
String termField = update.term.field();
|
||||
if (termField.equals(lastTermField) == false) {
|
||||
code |= 1;
|
||||
}
|
||||
String updateField = update.field;
|
||||
if (updateField.equals(lastUpdateField) == false) {
|
||||
code |= 2;
|
||||
}
|
||||
out.writeVInt(code);
|
||||
out.writeVInt(update.docIDUpto);
|
||||
if ((code & 1) != 0) {
|
||||
out.writeString(termField);
|
||||
lastTermField = termField;
|
||||
}
|
||||
if ((code & 2) != 0) {
|
||||
out.writeString(updateField);
|
||||
lastUpdateField = updateField;
|
||||
}
|
||||
|
||||
out.writeBytes(update.term.bytes().bytes, update.term.bytes().offset, update.term.bytes().length);
|
||||
out.writeZLong(((Long) update.value).longValue());
|
||||
}
|
||||
}
|
||||
binaryDVUpdates = allBinaryUpdates.toArray(new BinaryDocValuesUpdate[allBinaryUpdates.size()]);
|
||||
byte[] bytes = new byte[(int) out.getFilePointer()];
|
||||
out.writeTo(bytes, 0);
|
||||
return bytes;
|
||||
}
|
||||
|
||||
private byte[] freezeBinaryDVUpdates(Map<String,LinkedHashMap<Term,BinaryDocValuesUpdate>> binaryDVUpdates)
|
||||
throws IOException {
|
||||
// TODO: we could do better here, e.g. collate the updates by field
|
||||
// so if you are updating 2 fields interleaved we don't keep writing the field strings
|
||||
|
||||
RAMOutputStream out = new RAMOutputStream();
|
||||
String lastTermField = null;
|
||||
String lastUpdateField = null;
|
||||
for (LinkedHashMap<Term,BinaryDocValuesUpdate> binaryUpdates : binaryDVUpdates.values()) {
|
||||
binaryDVUpdateCount += binaryUpdates.size();
|
||||
for (BinaryDocValuesUpdate update : binaryUpdates.values()) {
|
||||
|
||||
int code = update.term.bytes().length << 2;
|
||||
|
||||
String termField = update.term.field();
|
||||
if (termField.equals(lastTermField) == false) {
|
||||
code |= 1;
|
||||
}
|
||||
String updateField = update.field;
|
||||
if (updateField.equals(lastUpdateField) == false) {
|
||||
code |= 2;
|
||||
}
|
||||
out.writeVInt(code);
|
||||
out.writeVInt(update.docIDUpto);
|
||||
if (termField.equals(lastTermField) == false) {
|
||||
out.writeString(termField);
|
||||
lastTermField = termField;
|
||||
}
|
||||
if (updateField.equals(lastUpdateField) == false) {
|
||||
out.writeString(updateField);
|
||||
lastUpdateField = updateField;
|
||||
}
|
||||
out.writeBytes(update.term.bytes().bytes, update.term.bytes().offset, update.term.bytes().length);
|
||||
|
||||
BytesRef value = (BytesRef) update.value;
|
||||
out.writeVInt(value.length);
|
||||
out.writeBytes(value.bytes, value.offset, value.length);
|
||||
}
|
||||
}
|
||||
byte[] bytes = new byte[(int) out.getFilePointer()];
|
||||
out.writeTo(bytes, 0);
|
||||
return bytes;
|
||||
}
|
||||
|
||||
/** Returns the {@link SegmentCommitInfo} that this packet is supposed to apply its deletes to, or null
|
||||
* if the private segment was already merged away. */
|
||||
private List<SegmentCommitInfo> getInfosToApply(IndexWriter writer) {
|
||||
assert Thread.holdsLock(writer);
|
||||
List<SegmentCommitInfo> infos;
|
||||
if (privateSegment != null) {
|
||||
if (writer.segmentInfos.indexOf(privateSegment) == -1) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "private segment already gone; skip processing updates");
|
||||
}
|
||||
return null;
|
||||
} else {
|
||||
infos = Collections.singletonList(privateSegment);
|
||||
}
|
||||
} else {
|
||||
infos = writer.segmentInfos.asList();
|
||||
}
|
||||
return infos;
|
||||
}
|
||||
|
||||
/** Translates a frozen packet of delete term/query, or doc values
|
||||
* updates, into their actual docIDs in the index, and applies the change. This is a heavy
|
||||
* operation and is done concurrently by incoming indexing threads. */
|
||||
|
||||
public synchronized void apply(IndexWriter writer) throws IOException {
|
||||
if (applied.getCount() == 0) {
|
||||
// already done
|
||||
return;
|
||||
}
|
||||
|
||||
long startNS = System.nanoTime();
|
||||
|
||||
assert any();
|
||||
|
||||
Set<SegmentCommitInfo> seenSegments = new HashSet<>();
|
||||
|
||||
int iter = 0;
|
||||
int totalSegmentCount = 0;
|
||||
long totalDelCount = 0;
|
||||
|
||||
// Optimistic concurrency: assume we are free to resolve the deletes against all current segments in the index, despite that
|
||||
// concurrent merges are running. Once we are done, we check to see if a merge completed while we were running. If so, we must retry
|
||||
// resolving against the newly merged segment(s). Eventually no merge finishes while we were running and we are done.
|
||||
while (true) {
|
||||
String messagePrefix;
|
||||
if (iter == 0) {
|
||||
messagePrefix = "";
|
||||
} else {
|
||||
messagePrefix = "iter " + iter;
|
||||
}
|
||||
|
||||
long iterStartNS = System.nanoTime();
|
||||
|
||||
long mergeGenStart = writer.mergeFinishedGen.get();
|
||||
|
||||
Set<String> delFiles = new HashSet<>();
|
||||
BufferedUpdatesStream.SegmentState[] segStates;
|
||||
|
||||
synchronized (writer) {
|
||||
List<SegmentCommitInfo> infos = getInfosToApply(writer);
|
||||
if (infos == null) {
|
||||
break;
|
||||
}
|
||||
|
||||
for (SegmentCommitInfo info : infos) {
|
||||
delFiles.addAll(info.files());
|
||||
}
|
||||
|
||||
// Must open while holding IW lock so that e.g. segments are not merged
|
||||
// away, dropped from 100% deletions, etc., before we can open the readers
|
||||
segStates = writer.bufferedUpdatesStream.openSegmentStates(writer.readerPool, infos, seenSegments, delGen());
|
||||
|
||||
if (segStates.length == 0) {
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "packet matches no segments");
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT,
|
||||
messagePrefix + "now apply del packet (%s) to %d segments, mergeGen %d",
|
||||
this, segStates.length, mergeGenStart));
|
||||
}
|
||||
|
||||
totalSegmentCount += segStates.length;
|
||||
|
||||
// Important, else IFD may try to delete our files while we are still using them,
|
||||
// if e.g. a merge finishes on some of the segments we are resolving on:
|
||||
writer.deleter.incRef(delFiles);
|
||||
}
|
||||
|
||||
boolean success = false;
|
||||
long delCount;
|
||||
try {
|
||||
// don't hold IW monitor lock here so threads are free concurrently resolve deletes/updates:
|
||||
delCount = apply(segStates);
|
||||
success = true;
|
||||
} finally {
|
||||
finishApply(writer, segStates, success, delFiles);
|
||||
}
|
||||
|
||||
// Since we jus resolved some more deletes/updates, now is a good time to write them:
|
||||
writer.readerPool.writeSomeDocValuesUpdates();
|
||||
|
||||
// It's OK to add this here, even if the while loop retries, because delCount only includes newly
|
||||
// deleted documents, on the segments we didn't already do in previous iterations:
|
||||
totalDelCount += delCount;
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT,
|
||||
messagePrefix + "done apply del packet (%s) to %d segments; %d new deletes/updates; took %.3f sec",
|
||||
this, segStates.length, delCount, (System.nanoTime() - iterStartNS) / 1000000000.));
|
||||
}
|
||||
|
||||
if (privateSegment != null) {
|
||||
// No need to retry for a segment-private packet: the merge that folds in our private segment already waits for all deletes to
|
||||
// be applied before it kicks off, so this private segment must already not be in the set of merging segments
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
// Must sync on writer here so that IW.mergeCommit is not running concurrently, so that if we exit, we know mergeCommit will succeed
|
||||
// in pulling all our delGens into a merge:
|
||||
synchronized (writer) {
|
||||
long mergeGenCur = writer.mergeFinishedGen.get();
|
||||
|
||||
if (mergeGenCur == mergeGenStart) {
|
||||
|
||||
// No merge finished while we were applying, so we are done!
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", messagePrefix + "concurrent merges finished; move to next iter");
|
||||
}
|
||||
|
||||
// A merge completed while we were running. In this case, that merge may have picked up some of the updates we did, but not
|
||||
// necessarily all of them, so we cycle again, re-applying all our updates to the newly merged segment.
|
||||
|
||||
iter++;
|
||||
}
|
||||
|
||||
// Record that this packet is finished:
|
||||
writer.bufferedUpdatesStream.finished(this);
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
String message = String.format(Locale.ROOT,
|
||||
"done apply del packet (%s) to %d segments; %d new deletes/updates; took %.3f sec",
|
||||
this, totalSegmentCount, totalDelCount, (System.nanoTime() - startNS) / 1000000000.);
|
||||
if (iter > 0) {
|
||||
message += "; " + (iter+1) + " iters due to concurrent merges";
|
||||
}
|
||||
message += "; " + writer.bufferedUpdatesStream.getPendingUpdatesCount() + " packets remain";
|
||||
infoStream.message("BD", message);
|
||||
}
|
||||
}
|
||||
|
||||
private void finishApply(IndexWriter writer, BufferedUpdatesStream.SegmentState[] segStates,
|
||||
boolean success, Set<String> delFiles) throws IOException {
|
||||
synchronized (writer) {
|
||||
|
||||
BufferedUpdatesStream.ApplyDeletesResult result;
|
||||
try {
|
||||
result = writer.bufferedUpdatesStream.closeSegmentStates(writer.readerPool, segStates, success);
|
||||
} finally {
|
||||
// Matches the incRef we did above, but we must do the decRef after closing segment states else
|
||||
// IFD can't delete still-open files
|
||||
writer.deleter.decRef(delFiles);
|
||||
}
|
||||
|
||||
if (result.anyDeletes) {
|
||||
writer.maybeMerge.set(true);
|
||||
writer.checkpoint();
|
||||
}
|
||||
|
||||
if (writer.keepFullyDeletedSegments == false && result.allDeleted != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "drop 100% deleted segments: " + writer.segString(result.allDeleted));
|
||||
}
|
||||
for (SegmentCommitInfo info : result.allDeleted) {
|
||||
writer.dropDeletedSegment(info);
|
||||
}
|
||||
writer.checkpoint();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Applies pending delete-by-term, delete-by-query and doc values updates to all segments in the index, returning
|
||||
* the number of new deleted or updated documents. */
|
||||
private synchronized long apply(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
|
||||
|
||||
if (delGen == -1) {
|
||||
// we were not yet pushed
|
||||
throw new IllegalArgumentException("gen is not yet set; call BufferedUpdatesStream.push first");
|
||||
}
|
||||
|
||||
if (applied.getCount() == 0) {
|
||||
// already done
|
||||
return totalDelCount;
|
||||
}
|
||||
|
||||
if (privateSegment != null) {
|
||||
assert segStates.length == 1;
|
||||
assert privateSegment == segStates[0].reader.getSegmentInfo();
|
||||
}
|
||||
|
||||
totalDelCount += applyTermDeletes(segStates);
|
||||
totalDelCount += applyQueryDeletes(segStates);
|
||||
totalDelCount += applyDocValuesUpdates(segStates);
|
||||
|
||||
return totalDelCount;
|
||||
}
|
||||
|
||||
private long applyDocValuesUpdates(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
|
||||
|
||||
if (numericDVUpdates.length == 0 && binaryDVUpdates.length == 0) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
long startNS = System.nanoTime();
|
||||
|
||||
long updateCount = 0;
|
||||
|
||||
for (BufferedUpdatesStream.SegmentState segState : segStates) {
|
||||
|
||||
if (delGen < segState.delGen) {
|
||||
// segment is newer than this deletes packet
|
||||
continue;
|
||||
}
|
||||
|
||||
if (segState.rld.refCount() == 1) {
|
||||
// This means we are the only remaining reference to this segment, meaning
|
||||
// it was merged away while we were running, so we can safely skip running
|
||||
// because we will run on the newly merged segment next:
|
||||
continue;
|
||||
}
|
||||
|
||||
if (numericDVUpdates.length > 0) {
|
||||
updateCount += applyDocValuesUpdates(segState, numericDVUpdates, true);
|
||||
}
|
||||
|
||||
if (binaryDVUpdates.length > 0) {
|
||||
updateCount += applyDocValuesUpdates(segState, binaryDVUpdates, false);
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD",
|
||||
String.format(Locale.ROOT, "applyDocValuesUpdates %.1f msec for %d segments, %d numeric updates and %d binary updates; %d new updates",
|
||||
(System.nanoTime()-startNS)/1000000.,
|
||||
segStates.length,
|
||||
numericDVUpdateCount,
|
||||
binaryDVUpdateCount,
|
||||
updateCount));
|
||||
}
|
||||
|
||||
return updateCount;
|
||||
}
|
||||
|
||||
private long applyDocValuesUpdates(BufferedUpdatesStream.SegmentState segState,
|
||||
byte[] updates, boolean isNumeric) throws IOException {
|
||||
|
||||
TermsEnum termsEnum = null;
|
||||
PostingsEnum postingsEnum = null;
|
||||
|
||||
// TODO: we can process the updates per DV field, from last to first so that
|
||||
// if multiple terms affect same document for the same field, we add an update
|
||||
// only once (that of the last term). To do that, we can keep a bitset which
|
||||
// marks which documents have already been updated. So e.g. if term T1
|
||||
// updates doc 7, and then we process term T2 and it updates doc 7 as well,
|
||||
// we don't apply the update since we know T1 came last and therefore wins
|
||||
// the update.
|
||||
// We can also use that bitset as 'liveDocs' to pass to TermEnum.docs(), so
|
||||
// that these documents aren't even returned.
|
||||
|
||||
long updateCount = 0;
|
||||
|
||||
// We first write all our updates private, and only in the end publish to the ReadersAndUpdates */
|
||||
Map<String, DocValuesFieldUpdates> holder = new HashMap<>();
|
||||
|
||||
ByteArrayDataInput in = new ByteArrayDataInput(updates);
|
||||
|
||||
String termField = null;
|
||||
String updateField = null;
|
||||
BytesRef term = new BytesRef();
|
||||
term.bytes = new byte[16];
|
||||
|
||||
bytesUsed = (int) (terms.ramBytesUsed() + queries.length * BYTES_PER_DEL_QUERY
|
||||
+ numericUpdatesSize + RamUsageEstimator.shallowSizeOf(numericDVUpdates)
|
||||
+ binaryUpdatesSize + RamUsageEstimator.shallowSizeOf(binaryDVUpdates));
|
||||
BytesRef scratch = new BytesRef();
|
||||
scratch.bytes = new byte[16];
|
||||
|
||||
numTermDeletes = deletes.numTermDeletes.get();
|
||||
while (in.getPosition() != updates.length) {
|
||||
int code = in.readVInt();
|
||||
int docIDUpto = in.readVInt();
|
||||
term.length = code >> 2;
|
||||
|
||||
if ((code & 1) != 0) {
|
||||
termField = in.readString();
|
||||
}
|
||||
if ((code & 2) != 0) {
|
||||
updateField = in.readString();
|
||||
}
|
||||
|
||||
if (term.bytes.length < term.length) {
|
||||
term.bytes = ArrayUtil.grow(term.bytes, term.length);
|
||||
}
|
||||
in.readBytes(term.bytes, 0, term.length);
|
||||
|
||||
int limit;
|
||||
if (delGen == segState.delGen) {
|
||||
assert privateSegment != null;
|
||||
limit = docIDUpto;
|
||||
} else {
|
||||
limit = Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
// TODO: we traverse the terms in update order (not term order) so that we
|
||||
// apply the updates in the correct order, i.e. if two terms udpate the
|
||||
// same document, the last one that came in wins, irrespective of the
|
||||
// terms lexical order.
|
||||
// we can apply the updates in terms order if we keep an updatesGen (and
|
||||
// increment it with every update) and attach it to each NumericUpdate. Note
|
||||
// that we cannot rely only on docIDUpto because an app may send two updates
|
||||
// which will get same docIDUpto, yet will still need to respect the order
|
||||
// those updates arrived.
|
||||
|
||||
// TODO: we could at least *collate* by field?
|
||||
|
||||
// This is the field used to resolve to docIDs, e.g. an "id" field, not the doc values field we are updating!
|
||||
if ((code & 1) != 0) {
|
||||
Terms terms = segState.reader.terms(termField);
|
||||
if (terms != null) {
|
||||
termsEnum = terms.iterator();
|
||||
} else {
|
||||
termsEnum = null;
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: can we avoid boxing here w/o fully forking this method?
|
||||
Object value;
|
||||
if (isNumeric) {
|
||||
value = Long.valueOf(in.readZLong());
|
||||
} else {
|
||||
value = scratch;
|
||||
scratch.length = in.readVInt();
|
||||
if (scratch.bytes.length < scratch.length) {
|
||||
scratch.bytes = ArrayUtil.grow(scratch.bytes, scratch.length);
|
||||
}
|
||||
in.readBytes(scratch.bytes, 0, scratch.length);
|
||||
}
|
||||
|
||||
if (termsEnum == null) {
|
||||
// no terms in this segment for this field
|
||||
continue;
|
||||
}
|
||||
|
||||
if (termsEnum.seekExact(term)) {
|
||||
|
||||
// we don't need term frequencies for this
|
||||
postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
|
||||
|
||||
DocValuesFieldUpdates dvUpdates = holder.get(updateField);
|
||||
if (dvUpdates == null) {
|
||||
if (isNumeric) {
|
||||
dvUpdates = new NumericDocValuesFieldUpdates(delGen, updateField, segState.reader.maxDoc());
|
||||
} else {
|
||||
dvUpdates = new BinaryDocValuesFieldUpdates(delGen, updateField, segState.reader.maxDoc());
|
||||
}
|
||||
|
||||
holder.put(updateField, dvUpdates);
|
||||
}
|
||||
|
||||
if (segState.rld.sortMap != null && privateSegment != null) {
|
||||
// This segment was sorted on flush; we must apply seg-private deletes carefully in this case:
|
||||
int doc;
|
||||
final Bits acceptDocs = segState.rld.getLiveDocs();
|
||||
while ((doc = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
|
||||
if (acceptDocs != null && acceptDocs.get(doc) == false) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// The limit is in the pre-sorted doc space:
|
||||
if (segState.rld.sortMap.newToOld(doc) < limit) {
|
||||
dvUpdates.add(doc, value);
|
||||
updateCount++;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
int doc;
|
||||
final Bits acceptDocs = segState.rld.getLiveDocs();
|
||||
while ((doc = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
if (doc >= limit) {
|
||||
break; // no more docs that can be updated for this term
|
||||
}
|
||||
if (acceptDocs != null && acceptDocs.get(doc) == false) {
|
||||
continue;
|
||||
}
|
||||
dvUpdates.add(doc, value);
|
||||
updateCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// now freeze & publish:
|
||||
for (DocValuesFieldUpdates update : holder.values()) {
|
||||
if (update.any()) {
|
||||
update.finish();
|
||||
segState.rld.addDVUpdate(update);
|
||||
}
|
||||
}
|
||||
|
||||
return updateCount;
|
||||
}
|
||||
|
||||
// Delete by query
|
||||
private long applyQueryDeletes(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
|
||||
|
||||
if (deleteQueries.length == 0) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
long startNS = System.nanoTime();
|
||||
|
||||
long delCount = 0;
|
||||
for (BufferedUpdatesStream.SegmentState segState : segStates) {
|
||||
|
||||
if (delGen < segState.delGen) {
|
||||
// segment is newer than this deletes packet
|
||||
continue;
|
||||
}
|
||||
|
||||
if (segState.rld.refCount() == 1) {
|
||||
// This means we are the only remaining reference to this segment, meaning
|
||||
// it was merged away while we were running, so we can safely skip running
|
||||
// because we will run on the newly merged segment next:
|
||||
continue;
|
||||
}
|
||||
|
||||
final LeafReaderContext readerContext = segState.reader.getContext();
|
||||
for (int i = 0; i < deleteQueries.length; i++) {
|
||||
Query query = deleteQueries[i];
|
||||
int limit;
|
||||
if (delGen == segState.delGen) {
|
||||
assert privateSegment != null;
|
||||
limit = deleteQueryLimits[i];
|
||||
} else {
|
||||
limit = Integer.MAX_VALUE;
|
||||
}
|
||||
final IndexSearcher searcher = new IndexSearcher(readerContext.reader());
|
||||
searcher.setQueryCache(null);
|
||||
final Weight weight = searcher.createNormalizedWeight(query, false);
|
||||
final Scorer scorer = weight.scorer(readerContext);
|
||||
if (scorer != null) {
|
||||
final DocIdSetIterator it = scorer.iterator();
|
||||
|
||||
int docID;
|
||||
while ((docID = it.nextDoc()) < limit) {
|
||||
if (segState.rld.delete(docID)) {
|
||||
delCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD",
|
||||
String.format(Locale.ROOT, "applyQueryDeletes took %.2f msec for %d segments and %d queries; %d new deletions",
|
||||
(System.nanoTime()-startNS)/1000000.,
|
||||
segStates.length,
|
||||
deleteQueries.length,
|
||||
delCount));
|
||||
}
|
||||
|
||||
return delCount;
|
||||
}
|
||||
|
||||
public void setDelGen(long gen) {
|
||||
assert this.gen == -1;
|
||||
this.gen = gen;
|
||||
terms.setDelGen(gen);
|
||||
private long applyTermDeletes(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
|
||||
|
||||
if (deleteTerms.size() == 0) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
// We apply segment-private deletes on flush:
|
||||
assert privateSegment == null;
|
||||
|
||||
try {
|
||||
long startNS = System.nanoTime();
|
||||
|
||||
long delCount = 0;
|
||||
|
||||
for (BufferedUpdatesStream.SegmentState segState : segStates) {
|
||||
assert segState.delGen != delGen: "segState.delGen=" + segState.delGen + " vs this.gen=" + delGen;
|
||||
if (segState.delGen > delGen) {
|
||||
// our deletes don't apply to this segment
|
||||
continue;
|
||||
}
|
||||
if (segState.rld.refCount() == 1) {
|
||||
// This means we are the only remaining reference to this segment, meaning
|
||||
// it was merged away while we were running, so we can safely skip running
|
||||
// because we will run on the newly merged segment next:
|
||||
continue;
|
||||
}
|
||||
|
||||
FieldTermIterator iter = deleteTerms.iterator();
|
||||
|
||||
BytesRef delTerm;
|
||||
String field = null;
|
||||
TermsEnum termsEnum = null;
|
||||
BytesRef readerTerm = null;
|
||||
PostingsEnum postingsEnum = null;
|
||||
while ((delTerm = iter.next()) != null) {
|
||||
|
||||
if (iter.field() != field) {
|
||||
// field changed
|
||||
field = iter.field();
|
||||
Terms terms = segState.reader.terms(field);
|
||||
if (terms != null) {
|
||||
termsEnum = terms.iterator();
|
||||
readerTerm = termsEnum.next();
|
||||
} else {
|
||||
termsEnum = null;
|
||||
}
|
||||
}
|
||||
|
||||
if (termsEnum != null) {
|
||||
int cmp = delTerm.compareTo(readerTerm);
|
||||
if (cmp < 0) {
|
||||
// TODO: can we advance across del terms here?
|
||||
// move to next del term
|
||||
continue;
|
||||
} else if (cmp == 0) {
|
||||
// fall through
|
||||
} else if (cmp > 0) {
|
||||
TermsEnum.SeekStatus status = termsEnum.seekCeil(delTerm);
|
||||
if (status == TermsEnum.SeekStatus.FOUND) {
|
||||
// fall through
|
||||
} else if (status == TermsEnum.SeekStatus.NOT_FOUND) {
|
||||
readerTerm = termsEnum.term();
|
||||
continue;
|
||||
} else {
|
||||
// TODO: can we advance to next field in deleted terms?
|
||||
// no more terms in this segment
|
||||
termsEnum = null;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
// we don't need term frequencies for this
|
||||
postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
|
||||
|
||||
assert postingsEnum != null;
|
||||
|
||||
int docID;
|
||||
while ((docID = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
|
||||
// NOTE: there is no limit check on the docID
|
||||
// when deleting by Term (unlike by Query)
|
||||
// because on flush we apply all Term deletes to
|
||||
// each segment. So all Term deleting here is
|
||||
// against prior segments:
|
||||
if (segState.rld.delete(docID)) {
|
||||
delCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD",
|
||||
String.format(Locale.ROOT, "applyTermDeletes took %.2f msec for %d segments and %d del terms; %d new deletions",
|
||||
(System.nanoTime()-startNS)/1000000.,
|
||||
segStates.length,
|
||||
deleteTerms.size(),
|
||||
delCount));
|
||||
}
|
||||
|
||||
return delCount;
|
||||
|
||||
} catch (Throwable t) {
|
||||
throw IOUtils.rethrowAlways(t);
|
||||
}
|
||||
}
|
||||
|
||||
public void setDelGen(long delGen) {
|
||||
assert this.delGen == -1: "delGen was already previously set to " + this.delGen;
|
||||
this.delGen = delGen;
|
||||
deleteTerms.setDelGen(delGen);
|
||||
}
|
||||
|
||||
public long delGen() {
|
||||
assert gen != -1;
|
||||
return gen;
|
||||
}
|
||||
|
||||
public TermIterator termIterator() {
|
||||
return terms.iterator();
|
||||
}
|
||||
|
||||
public Iterable<QueryAndLimit> queriesIterable() {
|
||||
return new Iterable<QueryAndLimit>() {
|
||||
@Override
|
||||
public Iterator<QueryAndLimit> iterator() {
|
||||
return new Iterator<QueryAndLimit>() {
|
||||
private int upto;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return upto < queries.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryAndLimit next() {
|
||||
QueryAndLimit ret = new QueryAndLimit(queries[upto], queryLimits[upto]);
|
||||
upto++;
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
assert delGen != -1;
|
||||
return delGen;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
String s = "";
|
||||
String s = "delGen=" + delGen;
|
||||
if (numTermDeletes != 0) {
|
||||
s += " " + numTermDeletes + " deleted terms (unique count=" + terms.size() + ")";
|
||||
s += " numDeleteTerms=" + numTermDeletes;
|
||||
if (numTermDeletes != deleteTerms.size()) {
|
||||
s += " (" + deleteTerms.size() + " unique)";
|
||||
}
|
||||
}
|
||||
if (queries.length != 0) {
|
||||
s += " " + queries.length + " deleted queries";
|
||||
if (deleteQueries.length != 0) {
|
||||
s += " numDeleteQuerys=" + deleteQueries.length;
|
||||
}
|
||||
if (numericDVUpdates.length > 0) {
|
||||
s += " numNumericDVUpdates=" + numericDVUpdateCount;
|
||||
}
|
||||
if (binaryDVUpdates.length > 0) {
|
||||
s += " numBinaryDVUpdates=" + binaryDVUpdateCount;
|
||||
}
|
||||
if (bytesUsed != 0) {
|
||||
s += " bytesUsed=" + bytesUsed;
|
||||
}
|
||||
if (privateSegment != null) {
|
||||
s += " privateSegment=" + privateSegment;
|
||||
}
|
||||
|
||||
return s;
|
||||
}
|
||||
|
||||
boolean any() {
|
||||
return terms.size() > 0 || queries.length > 0 || numericDVUpdates.length > 0 || binaryDVUpdates.length > 0;
|
||||
return deleteTerms.size() > 0 || deleteQueries.length > 0 || numericDVUpdates.length > 0 || binaryDVUpdates.length > 0;
|
||||
}
|
||||
|
||||
boolean anyDeleteTerms() {
|
||||
return deleteTerms.size() > 0;
|
||||
}
|
||||
}
|
||||
|
@ -510,9 +510,8 @@ final class IndexFileDeleter implements Closeable {
|
||||
assert locked();
|
||||
|
||||
assert Thread.holdsLock(writer);
|
||||
long t0 = 0;
|
||||
long t0 = System.nanoTime();
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
t0 = System.nanoTime();
|
||||
infoStream.message("IFD", "now checkpoint \"" + writer.segString(writer.toLiveInfos(segmentInfos)) + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
|
||||
}
|
||||
|
||||
@ -698,7 +697,9 @@ final class IndexFileDeleter implements Closeable {
|
||||
ensureOpen();
|
||||
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "delete " + names + "");
|
||||
if (names.size() > 0) {
|
||||
infoStream.message("IFD", "delete " + names + "");
|
||||
}
|
||||
}
|
||||
|
||||
// We make two passes, first deleting any segments_N files, second deleting the rest. We do this so that if we throw exc or JVM
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -91,8 +91,12 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
|
||||
*/
|
||||
public final static double DEFAULT_RAM_BUFFER_SIZE_MB = 16.0;
|
||||
|
||||
/** Default setting for {@link #setReaderPooling}. */
|
||||
public final static boolean DEFAULT_READER_POOLING = false;
|
||||
/** Default setting (true) for {@link #setReaderPooling}. */
|
||||
// We changed this default to true with concurrent deletes/updates (LUCENE-7868),
|
||||
// because we will otherwise need to open and close segment readers more frequently.
|
||||
// False is still supported, but will have worse performance since readers will
|
||||
// be forced to aggressively move all state to disk.
|
||||
public final static boolean DEFAULT_READER_POOLING = true;
|
||||
|
||||
/** Default value is 1945. Change using {@link #setRAMPerThreadHardLimitMB(int)} */
|
||||
public static final int DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB = 1945;
|
||||
@ -323,7 +327,6 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
|
||||
* Expert: Controls when segments are flushed to disk during indexing.
|
||||
* The {@link FlushPolicy} initialized during {@link IndexWriter} instantiation and once initialized
|
||||
* the given instance is bound to this {@link IndexWriter} and should not be used with another writer.
|
||||
* @see #setMaxBufferedDeleteTerms(int)
|
||||
* @see #setMaxBufferedDocs(int)
|
||||
* @see #setRAMBufferSizeMB(double)
|
||||
*/
|
||||
@ -374,11 +377,6 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
|
||||
return super.getAnalyzer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxBufferedDeleteTerms() {
|
||||
return super.getMaxBufferedDeleteTerms();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxBufferedDocs() {
|
||||
return super.getMaxBufferedDocs();
|
||||
@ -424,11 +422,6 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
|
||||
return (IndexWriterConfig) super.setMergePolicy(mergePolicy);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexWriterConfig setMaxBufferedDeleteTerms(int maxBufferedDeleteTerms) {
|
||||
return (IndexWriterConfig) super.setMaxBufferedDeleteTerms(maxBufferedDeleteTerms);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexWriterConfig setMaxBufferedDocs(int maxBufferedDocs) {
|
||||
return (IndexWriterConfig) super.setMaxBufferedDocs(maxBufferedDocs);
|
||||
|
@ -42,7 +42,6 @@ public class LiveIndexWriterConfig {
|
||||
|
||||
private volatile int maxBufferedDocs;
|
||||
private volatile double ramBufferSizeMB;
|
||||
private volatile int maxBufferedDeleteTerms;
|
||||
private volatile IndexReaderWarmer mergedSegmentWarmer;
|
||||
|
||||
// modified by IndexWriterConfig
|
||||
@ -109,7 +108,6 @@ public class LiveIndexWriterConfig {
|
||||
this.analyzer = analyzer;
|
||||
ramBufferSizeMB = IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB;
|
||||
maxBufferedDocs = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS;
|
||||
maxBufferedDeleteTerms = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DELETE_TERMS;
|
||||
mergedSegmentWarmer = null;
|
||||
delPolicy = new KeepOnlyLastCommitDeletionPolicy();
|
||||
commit = null;
|
||||
@ -135,43 +133,6 @@ public class LiveIndexWriterConfig {
|
||||
return analyzer;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines the maximum number of delete-by-term operations that will be
|
||||
* buffered before both the buffered in-memory delete terms and queries are
|
||||
* applied and flushed.
|
||||
* <p>
|
||||
* Disabled by default (writer flushes by RAM usage).
|
||||
* <p>
|
||||
* NOTE: This setting won't trigger a segment flush.
|
||||
*
|
||||
* <p>
|
||||
* Takes effect immediately, but only the next time a document is added,
|
||||
* updated or deleted. Also, if you only delete-by-query, this setting has no
|
||||
* effect, i.e. delete queries are buffered until the next segment is flushed.
|
||||
*
|
||||
* @throws IllegalArgumentException
|
||||
* if maxBufferedDeleteTerms is enabled but smaller than 1
|
||||
*
|
||||
* @see #setRAMBufferSizeMB
|
||||
*/
|
||||
public LiveIndexWriterConfig setMaxBufferedDeleteTerms(int maxBufferedDeleteTerms) {
|
||||
if (maxBufferedDeleteTerms != IndexWriterConfig.DISABLE_AUTO_FLUSH && maxBufferedDeleteTerms < 1) {
|
||||
throw new IllegalArgumentException("maxBufferedDeleteTerms must at least be 1 when enabled");
|
||||
}
|
||||
this.maxBufferedDeleteTerms = maxBufferedDeleteTerms;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of buffered deleted terms that will trigger a flush of all
|
||||
* buffered deletes if enabled.
|
||||
*
|
||||
* @see #setMaxBufferedDeleteTerms(int)
|
||||
*/
|
||||
public int getMaxBufferedDeleteTerms() {
|
||||
return maxBufferedDeleteTerms;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines the amount of RAM that may be used for buffering added documents
|
||||
* and deletions before they are flushed to the Directory. Generally for
|
||||
@ -195,12 +156,8 @@ public class LiveIndexWriterConfig {
|
||||
* <b>NOTE</b>: the account of RAM usage for pending deletions is only
|
||||
* approximate. Specifically, if you delete by Query, Lucene currently has no
|
||||
* way to measure the RAM usage of individual Queries so the accounting will
|
||||
* under-estimate and you should compensate by either calling commit()
|
||||
* periodically yourself, or by using {@link #setMaxBufferedDeleteTerms(int)}
|
||||
* to flush and apply buffered deletes by count instead of RAM usage (for each
|
||||
* buffered delete Query a constant number of bytes is used to estimate RAM
|
||||
* usage). Note that enabling {@link #setMaxBufferedDeleteTerms(int)} will not
|
||||
* trigger any segment flushes.
|
||||
* under-estimate and you should compensate by either calling commit() or refresh()
|
||||
* periodically yourself.
|
||||
* <p>
|
||||
* <b>NOTE</b>: It's not guaranteed that all memory resident documents are
|
||||
* flushed once this limit is exceeded. Depending on the configured
|
||||
@ -476,7 +433,6 @@ public class LiveIndexWriterConfig {
|
||||
sb.append("analyzer=").append(analyzer == null ? "null" : analyzer.getClass().getName()).append("\n");
|
||||
sb.append("ramBufferSizeMB=").append(getRAMBufferSizeMB()).append("\n");
|
||||
sb.append("maxBufferedDocs=").append(getMaxBufferedDocs()).append("\n");
|
||||
sb.append("maxBufferedDeleteTerms=").append(getMaxBufferedDeleteTerms()).append("\n");
|
||||
sb.append("mergedSegmentWarmer=").append(getMergedSegmentWarmer()).append("\n");
|
||||
sb.append("delPolicy=").append(getIndexDeletionPolicy().getClass().getName()).append("\n");
|
||||
IndexCommit commit = getIndexCommit();
|
||||
|
@ -1,132 +0,0 @@
|
||||
/*
|
||||
* 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.lucene.index;
|
||||
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
/** Merges multiple {@link FieldTermIterator}s */
|
||||
class MergedPrefixCodedTermsIterator extends FieldTermIterator {
|
||||
|
||||
private static class TermMergeQueue extends PriorityQueue<TermIterator> {
|
||||
TermMergeQueue(int size) {
|
||||
super(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean lessThan(TermIterator a, TermIterator b) {
|
||||
int cmp = a.bytes.compareTo(b.bytes);
|
||||
if (cmp < 0) {
|
||||
return true;
|
||||
} else if (cmp > 0) {
|
||||
return false;
|
||||
} else {
|
||||
return a.delGen() > b.delGen();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static class FieldMergeQueue extends PriorityQueue<TermIterator> {
|
||||
FieldMergeQueue(int size) {
|
||||
super(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean lessThan(TermIterator a, TermIterator b) {
|
||||
return a.field.compareTo(b.field) < 0;
|
||||
}
|
||||
}
|
||||
|
||||
final TermMergeQueue termQueue;
|
||||
final FieldMergeQueue fieldQueue;
|
||||
|
||||
public MergedPrefixCodedTermsIterator(List<PrefixCodedTerms> termsList) {
|
||||
fieldQueue = new FieldMergeQueue(termsList.size());
|
||||
for (PrefixCodedTerms terms : termsList) {
|
||||
TermIterator iter = terms.iterator();
|
||||
iter.next();
|
||||
if (iter.field != null) {
|
||||
fieldQueue.add(iter);
|
||||
}
|
||||
}
|
||||
|
||||
termQueue = new TermMergeQueue(termsList.size());
|
||||
}
|
||||
|
||||
String field;
|
||||
|
||||
@Override
|
||||
public BytesRef next() {
|
||||
if (termQueue.size() == 0) {
|
||||
// No more terms in current field:
|
||||
if (fieldQueue.size() == 0) {
|
||||
// No more fields:
|
||||
field = null;
|
||||
return null;
|
||||
}
|
||||
|
||||
// Transfer all iterators on the next field into the term queue:
|
||||
TermIterator top = fieldQueue.pop();
|
||||
termQueue.add(top);
|
||||
field = top.field;
|
||||
assert field != null;
|
||||
|
||||
while (fieldQueue.size() != 0 && fieldQueue.top().field.equals(top.field)) {
|
||||
TermIterator iter = fieldQueue.pop();
|
||||
assert iter.field.equals(field);
|
||||
// TODO: a little bit evil; we do this so we can == on field down below:
|
||||
iter.field = field;
|
||||
termQueue.add(iter);
|
||||
}
|
||||
|
||||
return termQueue.top().bytes;
|
||||
} else {
|
||||
TermIterator top = termQueue.top();
|
||||
if (top.next() == null) {
|
||||
termQueue.pop();
|
||||
} else if (top.field() != field) {
|
||||
// Field changed
|
||||
termQueue.pop();
|
||||
fieldQueue.add(top);
|
||||
} else {
|
||||
termQueue.updateTop();
|
||||
}
|
||||
if (termQueue.size() == 0) {
|
||||
// Recurse (just once) to go to next field:
|
||||
return next();
|
||||
} else {
|
||||
// Still terms left in this field
|
||||
return termQueue.top().bytes;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String field() {
|
||||
return field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long delGen() {
|
||||
return termQueue.top().delGen();
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,7 @@ package org.apache.lucene.index;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.util.InPlaceMergeSorter;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.apache.lucene.util.packed.PagedGrowableWriter;
|
||||
import org.apache.lucene.util.packed.PagedMutable;
|
||||
@ -40,11 +41,13 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
private long idx = 0; // long so we don't overflow if size == Integer.MAX_VALUE
|
||||
private int doc = -1;
|
||||
private Long value = null;
|
||||
private final long delGen;
|
||||
|
||||
Iterator(int size, PagedGrowableWriter values, PagedMutable docs) {
|
||||
Iterator(int size, PagedGrowableWriter values, PagedMutable docs, long delGen) {
|
||||
this.size = size;
|
||||
this.values = values;
|
||||
this.docs = docs;
|
||||
this.delGen = delGen;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -61,6 +64,7 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
doc = (int) docs.get(idx);
|
||||
++idx;
|
||||
while (idx < size && docs.get(idx) == doc) {
|
||||
// scan forward to last update to this doc
|
||||
++idx;
|
||||
}
|
||||
// idx points to the "next" element
|
||||
@ -72,12 +76,10 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
int doc() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
void reset() {
|
||||
doc = -1;
|
||||
value = null;
|
||||
idx = 0;
|
||||
long delGen() {
|
||||
return delGen;
|
||||
}
|
||||
}
|
||||
|
||||
@ -86,16 +88,26 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
private PagedGrowableWriter values;
|
||||
private int size;
|
||||
|
||||
public NumericDocValuesFieldUpdates(String field, int maxDoc) {
|
||||
super(field, DocValuesType.NUMERIC);
|
||||
public NumericDocValuesFieldUpdates(long delGen, String field, int maxDoc) {
|
||||
super(maxDoc, delGen, field, DocValuesType.NUMERIC);
|
||||
bitsPerValue = PackedInts.bitsRequired(maxDoc - 1);
|
||||
docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
|
||||
values = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
|
||||
size = 0;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void add(int doc, Object value) {
|
||||
public int size() {
|
||||
return size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void add(int doc, Object value) {
|
||||
if (finished) {
|
||||
throw new IllegalStateException("already finished");
|
||||
}
|
||||
|
||||
assert doc < maxDoc;
|
||||
|
||||
// TODO: if the Sorter interface changes to take long indexes, we can remove that limitation
|
||||
if (size == Integer.MAX_VALUE) {
|
||||
throw new IllegalStateException("cannot support more than Integer.MAX_VALUE doc/value entries");
|
||||
@ -113,11 +125,20 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
values.set(size, val.longValue());
|
||||
++size;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Iterator iterator() {
|
||||
final PagedMutable docs = this.docs;
|
||||
final PagedGrowableWriter values = this.values;
|
||||
public void finish() {
|
||||
if (finished) {
|
||||
throw new IllegalStateException("already finished");
|
||||
}
|
||||
finished = true;
|
||||
|
||||
// shrink wrap
|
||||
if (size < docs.size()) {
|
||||
docs = docs.resize(size);
|
||||
values = values.resize(size);
|
||||
}
|
||||
|
||||
new InPlaceMergeSorter() {
|
||||
@Override
|
||||
protected void swap(int i, int j) {
|
||||
@ -129,48 +150,36 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
values.set(j, values.get(i));
|
||||
values.set(i, tmpVal);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected int compare(int i, int j) {
|
||||
int x = (int) docs.get(i);
|
||||
int y = (int) docs.get(j);
|
||||
return (x < y) ? -1 : ((x == y) ? 0 : 1);
|
||||
// increasing docID order:
|
||||
// NOTE: we can have ties here, when the same docID was updated in the same segment, in which case we rely on sort being
|
||||
// stable and preserving original order so the last update to that docID wins
|
||||
return Integer.compare((int) docs.get(i), (int) docs.get(j));
|
||||
}
|
||||
}.sort(0, size);
|
||||
|
||||
return new Iterator(size, values, docs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void merge(DocValuesFieldUpdates other) {
|
||||
assert other instanceof NumericDocValuesFieldUpdates;
|
||||
NumericDocValuesFieldUpdates otherUpdates = (NumericDocValuesFieldUpdates) other;
|
||||
if (otherUpdates.size > Integer.MAX_VALUE - size) {
|
||||
throw new IllegalStateException(
|
||||
"cannot support more than Integer.MAX_VALUE doc/value entries; size="
|
||||
+ size + " other.size=" + otherUpdates.size);
|
||||
}
|
||||
docs = docs.grow(size + otherUpdates.size);
|
||||
values = values.grow(size + otherUpdates.size);
|
||||
for (int i = 0; i < otherUpdates.size; i++) {
|
||||
int doc = (int) otherUpdates.docs.get(i);
|
||||
docs.set(size, doc);
|
||||
values.set(size, otherUpdates.values.get(i));
|
||||
++size;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator iterator() {
|
||||
if (finished == false) {
|
||||
throw new IllegalStateException("call finish first");
|
||||
}
|
||||
return new Iterator(size, values, docs, delGen);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean any() {
|
||||
return size > 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesPerDoc() {
|
||||
long bytesPerDoc = (long) Math.ceil((double) (bitsPerValue) / 8);
|
||||
final int capacity = estimateCapacity(size);
|
||||
bytesPerDoc += (long) Math.ceil((double) values.ramBytesUsed() / capacity); // values
|
||||
return bytesPerDoc;
|
||||
public long ramBytesUsed() {
|
||||
return values.ramBytesUsed()
|
||||
+ docs.ramBytesUsed()
|
||||
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
|
||||
+ 2 * RamUsageEstimator.NUM_BYTES_INT
|
||||
+ 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -129,7 +129,7 @@ public class PrefixCodedTerms implements Accountable {
|
||||
|
||||
private TermIterator(long delGen, RAMFile buffer) {
|
||||
try {
|
||||
input = new RAMInputStream("MergedPrefixCodedTermsIterator", buffer);
|
||||
input = new RAMInputStream("PrefixCodedTermsIterator", buffer);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
@ -18,11 +18,17 @@ package org.apache.lucene.index;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
@ -36,6 +42,7 @@ import org.apache.lucene.store.TrackingDirectoryWrapper;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.MutableBits;
|
||||
|
||||
// Used by IndexWriter to hold open SegmentReaders (for
|
||||
@ -76,8 +83,20 @@ class ReadersAndUpdates {
|
||||
// That way, when the segment is done merging, IndexWriter can apply the
|
||||
// updates on the merged segment too.
|
||||
private boolean isMerging = false;
|
||||
|
||||
private final Map<String,DocValuesFieldUpdates> mergingDVUpdates = new HashMap<>();
|
||||
|
||||
// Holds resolved (to docIDs) doc values updates that have not yet been
|
||||
// written to the index
|
||||
private final Map<String,List<DocValuesFieldUpdates>> pendingDVUpdates = new HashMap<>();
|
||||
|
||||
// Holds resolved (to docIDs) doc values updates that were resolved while
|
||||
// this segment was being merged; at the end of the merge we carry over
|
||||
// these updates (remapping their docIDs) to the newly merged segment
|
||||
private final Map<String,List<DocValuesFieldUpdates>> mergingDVUpdates = new HashMap<>();
|
||||
|
||||
// Only set if there are doc values updates against this segment, and the index is sorted:
|
||||
Sorter.DocMap sortMap;
|
||||
|
||||
public final AtomicLong ramBytesUsed = new AtomicLong();
|
||||
|
||||
public ReadersAndUpdates(IndexWriter writer, SegmentCommitInfo info) {
|
||||
this.writer = writer;
|
||||
@ -100,12 +119,12 @@ class ReadersAndUpdates {
|
||||
|
||||
public void incRef() {
|
||||
final int rc = refCount.incrementAndGet();
|
||||
assert rc > 1;
|
||||
assert rc > 1: "seg=" + info;
|
||||
}
|
||||
|
||||
public void decRef() {
|
||||
final int rc = refCount.decrementAndGet();
|
||||
assert rc >= 0;
|
||||
assert rc >= 0: "seg=" + info;
|
||||
}
|
||||
|
||||
public int refCount() {
|
||||
@ -117,6 +136,52 @@ class ReadersAndUpdates {
|
||||
public synchronized int getPendingDeleteCount() {
|
||||
return pendingDeleteCount;
|
||||
}
|
||||
|
||||
private synchronized boolean assertNoDupGen(List<DocValuesFieldUpdates> fieldUpdates, DocValuesFieldUpdates update) {
|
||||
for (int i=0;i<fieldUpdates.size();i++) {
|
||||
DocValuesFieldUpdates oldUpdate = fieldUpdates.get(i);
|
||||
if (oldUpdate.delGen == update.delGen) {
|
||||
throw new AssertionError("duplicate delGen=" + update.delGen + " for seg=" + info);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/** Adds a new resolved (meaning it maps docIDs to new values) doc values packet. We buffer these in RAM and write to disk when too much
|
||||
* RAM is used or when a merge needs to kick off, or a commit/refresh. */
|
||||
public synchronized void addDVUpdate(DocValuesFieldUpdates update) {
|
||||
if (update.getFinished() == false) {
|
||||
throw new IllegalArgumentException("call finish first");
|
||||
}
|
||||
List<DocValuesFieldUpdates> fieldUpdates = pendingDVUpdates.get(update.field);
|
||||
if (fieldUpdates == null) {
|
||||
fieldUpdates = new ArrayList<>();
|
||||
pendingDVUpdates.put(update.field, fieldUpdates);
|
||||
}
|
||||
|
||||
assert assertNoDupGen(fieldUpdates, update);
|
||||
|
||||
ramBytesUsed.addAndGet(update.ramBytesUsed());
|
||||
|
||||
fieldUpdates.add(update);
|
||||
|
||||
if (isMerging) {
|
||||
fieldUpdates = mergingDVUpdates.get(update.field);
|
||||
if (fieldUpdates == null) {
|
||||
fieldUpdates = new ArrayList<>();
|
||||
mergingDVUpdates.put(update.field, fieldUpdates);
|
||||
}
|
||||
fieldUpdates.add(update);
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized long getNumDVUpdates() {
|
||||
long count = 0;
|
||||
for (List<DocValuesFieldUpdates> updates : pendingDVUpdates.values()) {
|
||||
count += updates.size();
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
// Call only from assert!
|
||||
public synchronized boolean verifyDocCounts() {
|
||||
@ -137,7 +202,7 @@ class ReadersAndUpdates {
|
||||
}
|
||||
|
||||
/** Returns a {@link SegmentReader}. */
|
||||
public SegmentReader getReader(IOContext context) throws IOException {
|
||||
public synchronized SegmentReader getReader(IOContext context) throws IOException {
|
||||
if (reader == null) {
|
||||
// We steal returned ref:
|
||||
reader = new SegmentReader(info, writer.segmentInfos.getIndexCreatedVersionMajor(), context);
|
||||
@ -156,16 +221,15 @@ class ReadersAndUpdates {
|
||||
sr.decRef();
|
||||
}
|
||||
|
||||
public synchronized boolean delete(int docID) {
|
||||
public synchronized boolean delete(int docID) throws IOException {
|
||||
initWritableLiveDocs();
|
||||
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 + " 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.info.maxDoc()-liveDocs.count()));
|
||||
}
|
||||
return didDelete;
|
||||
}
|
||||
@ -175,7 +239,6 @@ class ReadersAndUpdates {
|
||||
// TODO: can we somehow use IOUtils here...? problem is
|
||||
// we are calling .decRef not .close)...
|
||||
if (reader != null) {
|
||||
//System.out.println(" pool.drop info=" + info + " rc=" + reader.getRefCount());
|
||||
try {
|
||||
reader.decRef();
|
||||
} finally {
|
||||
@ -207,10 +270,8 @@ class ReadersAndUpdates {
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void initWritableLiveDocs() throws IOException {
|
||||
assert Thread.holdsLock(writer);
|
||||
private synchronized void initWritableLiveDocs() throws IOException {
|
||||
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
|
||||
// SegmentReader sharing the current liveDocs
|
||||
@ -218,7 +279,6 @@ class ReadersAndUpdates {
|
||||
// change it:
|
||||
LiveDocsFormat liveDocsFormat = info.info.getCodec().liveDocsFormat();
|
||||
if (liveDocs == null) {
|
||||
//System.out.println("create BV seg=" + info);
|
||||
liveDocs = liveDocsFormat.newLiveDocs(info.info.maxDoc());
|
||||
} else {
|
||||
liveDocs = liveDocsFormat.newLiveDocs(liveDocs);
|
||||
@ -228,21 +288,16 @@ class ReadersAndUpdates {
|
||||
}
|
||||
|
||||
public synchronized Bits getLiveDocs() {
|
||||
assert Thread.holdsLock(writer);
|
||||
return liveDocs;
|
||||
}
|
||||
|
||||
public synchronized Bits getReadOnlyLiveDocs() {
|
||||
//System.out.println("getROLiveDocs seg=" + info);
|
||||
assert Thread.holdsLock(writer);
|
||||
liveDocsShared = true;
|
||||
//if (liveDocs != null) {
|
||||
//System.out.println(" liveCount=" + liveDocs.count());
|
||||
//}
|
||||
return liveDocs;
|
||||
}
|
||||
|
||||
public synchronized void dropChanges() {
|
||||
assert Thread.holdsLock(writer);
|
||||
// Discard (don't save) changes when we are dropping
|
||||
// the reader; this is used only on the sub-readers
|
||||
// after a successful merge. If deletes had
|
||||
@ -258,8 +313,6 @@ class ReadersAndUpdates {
|
||||
// _X_N updates files) to the directory; returns true if it wrote any file
|
||||
// and false if there were no new deletes or updates to write:
|
||||
public synchronized boolean writeLiveDocs(Directory dir) throws IOException {
|
||||
assert Thread.holdsLock(writer);
|
||||
//System.out.println("rld.writeLiveDocs seg=" + info + " pendingDelCount=" + pendingDeleteCount + " numericUpdates=" + numericUpdates);
|
||||
if (pendingDeleteCount == 0) {
|
||||
return false;
|
||||
}
|
||||
@ -304,16 +357,43 @@ class ReadersAndUpdates {
|
||||
}
|
||||
|
||||
@SuppressWarnings("synthetic-access")
|
||||
private void handleNumericDVUpdates(FieldInfos infos, Map<String,NumericDocValuesFieldUpdates> updates,
|
||||
Directory dir, DocValuesFormat dvFormat, final SegmentReader reader, Map<Integer,Set<String>> fieldFiles) throws IOException {
|
||||
for (Entry<String,NumericDocValuesFieldUpdates> e : updates.entrySet()) {
|
||||
final String field = e.getKey();
|
||||
final NumericDocValuesFieldUpdates fieldUpdates = e.getValue();
|
||||
private synchronized void handleNumericDVUpdates(FieldInfos infos,
|
||||
Directory dir, DocValuesFormat dvFormat, final SegmentReader reader,
|
||||
Map<Integer,Set<String>> fieldFiles, long maxDelGen, InfoStream infoStream) throws IOException {
|
||||
|
||||
for (Entry<String,List<DocValuesFieldUpdates>> ent : pendingDVUpdates.entrySet()) {
|
||||
final String field = ent.getKey();
|
||||
final List<DocValuesFieldUpdates> updates = ent.getValue();
|
||||
if (updates.get(0).type != DocValuesType.NUMERIC) {
|
||||
continue;
|
||||
}
|
||||
|
||||
final List<DocValuesFieldUpdates> updatesToApply = new ArrayList<>();
|
||||
long bytes = 0;
|
||||
for(DocValuesFieldUpdates update : updates) {
|
||||
if (update.delGen <= maxDelGen) {
|
||||
// safe to apply this one
|
||||
bytes += update.ramBytesUsed();
|
||||
updatesToApply.add(update);
|
||||
}
|
||||
}
|
||||
if (updatesToApply.isEmpty()) {
|
||||
// nothing to apply yet
|
||||
continue;
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT,
|
||||
"now write %d pending numeric DV updates for field=%s, seg=%s, bytes=%.3f MB",
|
||||
updatesToApply.size(),
|
||||
field,
|
||||
info,
|
||||
bytes/1024./1024.));
|
||||
}
|
||||
|
||||
final long nextDocValuesGen = info.getNextDocValuesGen();
|
||||
final String segmentSuffix = Long.toString(nextDocValuesGen, Character.MAX_RADIX);
|
||||
final long estUpdatesSize = fieldUpdates.ramBytesPerDoc() * info.info.maxDoc();
|
||||
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.maxDoc(), estUpdatesSize));
|
||||
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.maxDoc(), bytes));
|
||||
final FieldInfo fieldInfo = infos.fieldInfo(field);
|
||||
assert fieldInfo != null;
|
||||
fieldInfo.setDocValuesGen(nextDocValuesGen);
|
||||
@ -330,11 +410,14 @@ class ReadersAndUpdates {
|
||||
throw new IllegalArgumentException("wrong fieldInfo");
|
||||
}
|
||||
final int maxDoc = reader.maxDoc();
|
||||
DocValuesFieldUpdates.Iterator[] subs = new DocValuesFieldUpdates.Iterator[updatesToApply.size()];
|
||||
for(int i=0;i<subs.length;i++) {
|
||||
subs[i] = updatesToApply.get(i).iterator();
|
||||
}
|
||||
|
||||
final NumericDocValuesFieldUpdates.Iterator updatesIter = fieldUpdates.iterator();
|
||||
final DocValuesFieldUpdates.Iterator updatesIter = DocValuesFieldUpdates.mergedIterator(subs);
|
||||
|
||||
final NumericDocValues currentValues = reader.getNumericDocValues(field);
|
||||
updatesIter.reset();
|
||||
|
||||
// Merge sort of the original doc values with updated doc values:
|
||||
return new NumericDocValues() {
|
||||
@ -394,7 +477,7 @@ class ReadersAndUpdates {
|
||||
} else {
|
||||
docIDOut = updateDocID;
|
||||
if (docIDOut != NO_MORE_DOCS) {
|
||||
value = updatesIter.value();
|
||||
value = (Long) updatesIter.value();
|
||||
}
|
||||
}
|
||||
return docIDOut;
|
||||
@ -410,16 +493,42 @@ class ReadersAndUpdates {
|
||||
}
|
||||
|
||||
@SuppressWarnings("synthetic-access")
|
||||
private void handleBinaryDVUpdates(FieldInfos infos, Map<String,BinaryDocValuesFieldUpdates> updates,
|
||||
TrackingDirectoryWrapper dir, DocValuesFormat dvFormat, final SegmentReader reader, Map<Integer,Set<String>> fieldFiles) throws IOException {
|
||||
for (Entry<String,BinaryDocValuesFieldUpdates> e : updates.entrySet()) {
|
||||
final String field = e.getKey();
|
||||
final BinaryDocValuesFieldUpdates fieldUpdates = e.getValue();
|
||||
private synchronized void handleBinaryDVUpdates(FieldInfos infos,
|
||||
TrackingDirectoryWrapper dir, DocValuesFormat dvFormat, final SegmentReader reader,
|
||||
Map<Integer,Set<String>> fieldFiles, long maxDelGen, InfoStream infoStream) throws IOException {
|
||||
for (Entry<String,List<DocValuesFieldUpdates>> ent : pendingDVUpdates.entrySet()) {
|
||||
final String field = ent.getKey();
|
||||
final List<DocValuesFieldUpdates> updates = ent.getValue();
|
||||
if (updates.get(0).type != DocValuesType.BINARY) {
|
||||
continue;
|
||||
}
|
||||
|
||||
final List<DocValuesFieldUpdates> updatesToApply = new ArrayList<>();
|
||||
long bytes = 0;
|
||||
for(DocValuesFieldUpdates update : updates) {
|
||||
if (update.delGen <= maxDelGen) {
|
||||
// safe to apply this one
|
||||
bytes += update.ramBytesUsed();
|
||||
updatesToApply.add(update);
|
||||
}
|
||||
}
|
||||
if (updatesToApply.isEmpty()) {
|
||||
// nothing to apply yet
|
||||
continue;
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT,
|
||||
"now write %d pending binary DV updates for field=%s, seg=%s, bytes=%.3fMB",
|
||||
updatesToApply.size(),
|
||||
field,
|
||||
info,
|
||||
bytes/1024./1024.));
|
||||
}
|
||||
|
||||
final long nextDocValuesGen = info.getNextDocValuesGen();
|
||||
final String segmentSuffix = Long.toString(nextDocValuesGen, Character.MAX_RADIX);
|
||||
final long estUpdatesSize = fieldUpdates.ramBytesPerDoc() * info.info.maxDoc();
|
||||
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.maxDoc(), estUpdatesSize));
|
||||
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.maxDoc(), bytes));
|
||||
final FieldInfo fieldInfo = infos.fieldInfo(field);
|
||||
assert fieldInfo != null;
|
||||
fieldInfo.setDocValuesGen(nextDocValuesGen);
|
||||
@ -438,8 +547,12 @@ class ReadersAndUpdates {
|
||||
}
|
||||
final int maxDoc = reader.maxDoc();
|
||||
|
||||
final BinaryDocValuesFieldUpdates.Iterator updatesIter = fieldUpdates.iterator();
|
||||
updatesIter.reset();
|
||||
DocValuesFieldUpdates.Iterator[] subs = new DocValuesFieldUpdates.Iterator[updatesToApply.size()];
|
||||
for(int i=0;i<subs.length;i++) {
|
||||
subs[i] = updatesToApply.get(i).iterator();
|
||||
}
|
||||
|
||||
final DocValuesFieldUpdates.Iterator updatesIter = DocValuesFieldUpdates.mergedIterator(subs);
|
||||
|
||||
final BinaryDocValues currentValues = reader.getBinaryDocValues(field);
|
||||
|
||||
@ -500,7 +613,7 @@ class ReadersAndUpdates {
|
||||
} else {
|
||||
docIDOut = updateDocID;
|
||||
if (docIDOut != NO_MORE_DOCS) {
|
||||
value = updatesIter.value();
|
||||
value = (BytesRef) updatesIter.value();
|
||||
}
|
||||
}
|
||||
return docIDOut;
|
||||
@ -515,7 +628,7 @@ class ReadersAndUpdates {
|
||||
}
|
||||
}
|
||||
|
||||
private Set<String> writeFieldInfosGen(FieldInfos fieldInfos, Directory dir, DocValuesFormat dvFormat,
|
||||
private synchronized Set<String> writeFieldInfosGen(FieldInfos fieldInfos, Directory dir, DocValuesFormat dvFormat,
|
||||
FieldInfosFormat infosFormat) throws IOException {
|
||||
final long nextFieldInfosGen = info.getNextFieldInfosGen();
|
||||
final String segmentSuffix = Long.toString(nextFieldInfosGen, Character.MAX_RADIX);
|
||||
@ -531,28 +644,52 @@ class ReadersAndUpdates {
|
||||
return trackingDir.getCreatedFiles();
|
||||
}
|
||||
|
||||
// Writes field updates (new _X_N updates files) to the directory
|
||||
public synchronized void writeFieldUpdates(Directory dir, DocValuesFieldUpdates.Container dvUpdates) throws IOException {
|
||||
public synchronized boolean writeFieldUpdates(Directory dir, long maxDelGen, InfoStream infoStream) throws IOException {
|
||||
|
||||
long startTimeNS = System.nanoTime();
|
||||
|
||||
assert Thread.holdsLock(writer);
|
||||
//System.out.println("rld.writeFieldUpdates: seg=" + info + " numericFieldUpdates=" + numericFieldUpdates);
|
||||
|
||||
assert dvUpdates.any();
|
||||
|
||||
// Do this so we can delete any created files on
|
||||
// exception; this saves all codecs from having to do
|
||||
// it:
|
||||
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
|
||||
|
||||
|
||||
final Map<Integer,Set<String>> newDVFiles = new HashMap<>();
|
||||
Set<String> fieldInfosFiles = null;
|
||||
FieldInfos fieldInfos = null;
|
||||
|
||||
boolean any = false;
|
||||
int count = 0;
|
||||
for (List<DocValuesFieldUpdates> updates : pendingDVUpdates.values()) {
|
||||
// Sort by increasing delGen:
|
||||
Collections.sort(updates, (a, b) -> Long.compare(a.delGen, b.delGen));
|
||||
count += updates.size();
|
||||
for (DocValuesFieldUpdates update : updates) {
|
||||
if (update.delGen <= maxDelGen && update.any()) {
|
||||
any = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (any == false) {
|
||||
// no updates
|
||||
return false;
|
||||
}
|
||||
|
||||
// Do this so we can delete any created files on
|
||||
// exception; this saves all codecs from having to do it:
|
||||
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
final Codec codec = info.info.getCodec();
|
||||
|
||||
// reader could be null e.g. for a just merged segment (from
|
||||
// IndexWriter.commitMergedDeletes).
|
||||
final SegmentReader reader = this.reader == null ? new SegmentReader(info, writer.segmentInfos.getIndexCreatedVersionMajor(), IOContext.READONCE) : this.reader;
|
||||
final SegmentReader reader;
|
||||
if (this.reader == null) {
|
||||
reader = new SegmentReader(info, writer.segmentInfos.getIndexCreatedVersionMajor(), IOContext.READONCE);
|
||||
} else {
|
||||
reader = this.reader;
|
||||
}
|
||||
|
||||
try {
|
||||
// clone FieldInfos so that we can update their dvGen separately from
|
||||
// the reader's infos and write them to a new fieldInfos_gen file
|
||||
@ -567,38 +704,30 @@ class ReadersAndUpdates {
|
||||
}
|
||||
clone.setDocValuesGen(fi.getDocValuesGen());
|
||||
}
|
||||
// create new fields or update existing ones to have NumericDV type
|
||||
for (String f : dvUpdates.numericDVUpdates.keySet()) {
|
||||
FieldInfo fieldInfo = builder.getOrAdd(f);
|
||||
fieldInfo.setDocValuesType(DocValuesType.NUMERIC);
|
||||
}
|
||||
// create new fields or update existing ones to have BinaryDV type
|
||||
for (String f : dvUpdates.binaryDVUpdates.keySet()) {
|
||||
FieldInfo fieldInfo = builder.getOrAdd(f);
|
||||
fieldInfo.setDocValuesType(DocValuesType.BINARY);
|
||||
|
||||
// create new fields with the right DV type
|
||||
for (List<DocValuesFieldUpdates> updates : pendingDVUpdates.values()) {
|
||||
DocValuesFieldUpdates update = updates.get(0);
|
||||
FieldInfo fieldInfo = builder.getOrAdd(update.field);
|
||||
fieldInfo.setDocValuesType(update.type);
|
||||
}
|
||||
|
||||
fieldInfos = builder.finish();
|
||||
final DocValuesFormat docValuesFormat = codec.docValuesFormat();
|
||||
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeFieldUpdates: applying numeric updates; seg=" + info + " updates=" + numericFieldUpdates);
|
||||
handleNumericDVUpdates(fieldInfos, dvUpdates.numericDVUpdates, trackingDir, docValuesFormat, reader, newDVFiles);
|
||||
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: applying binary updates; seg=" + info + " updates=" + dvUpdates.binaryDVUpdates);
|
||||
handleBinaryDVUpdates(fieldInfos, dvUpdates.binaryDVUpdates, trackingDir, docValuesFormat, reader, newDVFiles);
|
||||
handleNumericDVUpdates(fieldInfos, trackingDir, docValuesFormat, reader, newDVFiles, maxDelGen, infoStream);
|
||||
handleBinaryDVUpdates(fieldInfos, trackingDir, docValuesFormat, reader, newDVFiles, maxDelGen, infoStream);
|
||||
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: write fieldInfos; seg=" + info);
|
||||
fieldInfosFiles = writeFieldInfosGen(fieldInfos, trackingDir, docValuesFormat, codec.fieldInfosFormat());
|
||||
} finally {
|
||||
if (reader != this.reader) {
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeLiveDocs: closeReader " + reader);
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (success == false) {
|
||||
// Advance only the nextWriteFieldInfosGen and nextWriteDocValuesGen, so
|
||||
// that a 2nd attempt to write will write to a new file
|
||||
info.advanceNextWriteFieldInfosGen();
|
||||
@ -610,27 +739,48 @@ class ReadersAndUpdates {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// copy all the updates to mergingUpdates, so they can later be applied to the merged segment
|
||||
if (isMerging) {
|
||||
for (Entry<String,NumericDocValuesFieldUpdates> e : dvUpdates.numericDVUpdates.entrySet()) {
|
||||
DocValuesFieldUpdates updates = mergingDVUpdates.get(e.getKey());
|
||||
if (updates == null) {
|
||||
mergingDVUpdates.put(e.getKey(), e.getValue());
|
||||
|
||||
// Prune the now-written DV updates:
|
||||
long bytesFreed = 0;
|
||||
Iterator<Map.Entry<String,List<DocValuesFieldUpdates>>> it = pendingDVUpdates.entrySet().iterator();
|
||||
while (it.hasNext()) {
|
||||
Map.Entry<String,List<DocValuesFieldUpdates>> ent = it.next();
|
||||
int upto = 0;
|
||||
List<DocValuesFieldUpdates> updates = ent.getValue();
|
||||
for (DocValuesFieldUpdates update : updates) {
|
||||
if (update.delGen > maxDelGen) {
|
||||
// not yet applied
|
||||
updates.set(upto, update);
|
||||
upto++;
|
||||
} else {
|
||||
updates.merge(e.getValue());
|
||||
bytesFreed += update.ramBytesUsed();
|
||||
}
|
||||
}
|
||||
for (Entry<String,BinaryDocValuesFieldUpdates> e : dvUpdates.binaryDVUpdates.entrySet()) {
|
||||
DocValuesFieldUpdates updates = mergingDVUpdates.get(e.getKey());
|
||||
if (updates == null) {
|
||||
mergingDVUpdates.put(e.getKey(), e.getValue());
|
||||
} else {
|
||||
updates.merge(e.getValue());
|
||||
if (upto == 0) {
|
||||
it.remove();
|
||||
} else {
|
||||
updates.subList(upto, updates.size()).clear();
|
||||
}
|
||||
}
|
||||
|
||||
long bytes = ramBytesUsed.addAndGet(-bytesFreed);
|
||||
assert bytes >= 0;
|
||||
|
||||
// if there is a reader open, reopen it to reflect the updates
|
||||
if (reader != null) {
|
||||
SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.maxDoc() - info.getDelCount() - pendingDeleteCount);
|
||||
boolean success2 = false;
|
||||
try {
|
||||
reader.decRef();
|
||||
reader = newReader;
|
||||
success2 = true;
|
||||
} finally {
|
||||
if (success2 == false) {
|
||||
newReader.decRef();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// writing field updates succeeded
|
||||
assert fieldInfosFiles != null;
|
||||
info.setFieldInfosFiles(fieldInfosFiles);
|
||||
@ -639,46 +789,75 @@ class ReadersAndUpdates {
|
||||
// of files, hence we copy from the existing map all fields w/ updates that
|
||||
// were not updated in this session, and add new mappings for fields that
|
||||
// were updated now.
|
||||
assert !newDVFiles.isEmpty();
|
||||
assert newDVFiles.isEmpty() == false;
|
||||
for (Entry<Integer,Set<String>> e : info.getDocValuesUpdatesFiles().entrySet()) {
|
||||
if (!newDVFiles.containsKey(e.getKey())) {
|
||||
if (newDVFiles.containsKey(e.getKey()) == false) {
|
||||
newDVFiles.put(e.getKey(), e.getValue());
|
||||
}
|
||||
}
|
||||
info.setDocValuesUpdatesFiles(newDVFiles);
|
||||
|
||||
// wrote new files, should checkpoint()
|
||||
writer.checkpoint();
|
||||
|
||||
// if there is a reader open, reopen it to reflect the updates
|
||||
if (reader != null) {
|
||||
SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.maxDoc() - info.getDelCount() - pendingDeleteCount);
|
||||
boolean reopened = false;
|
||||
// wrote new files, should checkpoint()
|
||||
writer.checkpointNoSIS();
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT, "done write field updates for seg=%s; took %.3fs; new files: %s",
|
||||
info, (System.nanoTime() - startTimeNS)/1000000000.0, newDVFiles));
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/** Returns a reader for merge, with the latest doc values updates and deletions. */
|
||||
synchronized SegmentReader getReaderForMerge(IOContext context) throws IOException {
|
||||
|
||||
// This ensures any newly resolved doc value updates while we are merging are
|
||||
// saved for re-applying after this segment is done merging:
|
||||
isMerging = true;
|
||||
|
||||
assert mergingDVUpdates.isEmpty();
|
||||
|
||||
// We must carry over any still-pending DV updates because they were not
|
||||
// successfully written, e.g. because there was a hole in the delGens,
|
||||
// or they arrived after we wrote all DVs for merge but before we set
|
||||
// isMerging here:
|
||||
for (Map.Entry<String, List<DocValuesFieldUpdates>> ent : pendingDVUpdates.entrySet()) {
|
||||
List<DocValuesFieldUpdates> mergingUpdates = mergingDVUpdates.get(ent.getKey());
|
||||
if (mergingUpdates == null) {
|
||||
mergingUpdates = new ArrayList<>();
|
||||
mergingDVUpdates.put(ent.getKey(), mergingUpdates);
|
||||
}
|
||||
mergingUpdates.addAll(ent.getValue());
|
||||
}
|
||||
|
||||
SegmentReader reader = getReader(context);
|
||||
int delCount = pendingDeleteCount + info.getDelCount();
|
||||
if (delCount != reader.numDeletedDocs()) {
|
||||
|
||||
// beware of zombies:
|
||||
assert delCount > reader.numDeletedDocs(): "delCount=" + delCount + " reader.numDeletedDocs()=" + reader.numDeletedDocs();
|
||||
|
||||
assert liveDocs != null;
|
||||
|
||||
// Create a new reader with the latest live docs:
|
||||
SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.maxDoc() - delCount);
|
||||
boolean success = false;
|
||||
try {
|
||||
reader.decRef();
|
||||
reader = newReader;
|
||||
reopened = true;
|
||||
success = true;
|
||||
} finally {
|
||||
if (!reopened) {
|
||||
if (success == false) {
|
||||
newReader.decRef();
|
||||
}
|
||||
}
|
||||
reader = newReader;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a reader for merge. This method applies field updates if there are
|
||||
* any and marks that this segment is currently merging.
|
||||
*/
|
||||
synchronized SegmentReader getReaderForMerge(IOContext context) throws IOException {
|
||||
assert Thread.holdsLock(writer);
|
||||
// must execute these two statements as atomic operation, otherwise we
|
||||
// could lose updates if e.g. another thread calls writeFieldUpdates in
|
||||
// between, or the updates are applied to the obtained reader, but then
|
||||
// re-applied in IW.commitMergedDeletes (unnecessary work and potential
|
||||
// bugs).
|
||||
isMerging = true;
|
||||
return getReader(context);
|
||||
liveDocsShared = true;
|
||||
|
||||
assert verifyDocCounts();
|
||||
|
||||
return reader;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -689,12 +868,13 @@ class ReadersAndUpdates {
|
||||
mergingDVUpdates.clear();
|
||||
isMerging = false;
|
||||
}
|
||||
|
||||
/** Returns updates that came in while this segment was merging. */
|
||||
public synchronized Map<String,DocValuesFieldUpdates> getMergingFieldUpdates() {
|
||||
|
||||
public synchronized Map<String,List<DocValuesFieldUpdates>> getMergingDVUpdates() {
|
||||
// We must atomically (in single sync'd block) clear isMerging when we return the DV updates otherwise we can lose updates:
|
||||
isMerging = false;
|
||||
return mergingDVUpdates;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
@ -703,5 +883,4 @@ class ReadersAndUpdates {
|
||||
sb.append(" liveDocsShared=").append(liveDocsShared);
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -70,6 +70,10 @@ public class SegmentCommitInfo {
|
||||
|
||||
private volatile long sizeInBytes = -1;
|
||||
|
||||
// NOTE: only used in-RAM by IW to track buffered deletes;
|
||||
// this is never written to/read from the Directory
|
||||
private long bufferedDeletesGen = -1;
|
||||
|
||||
/**
|
||||
* Sole constructor.
|
||||
*
|
||||
@ -236,17 +240,17 @@ public class SegmentCommitInfo {
|
||||
return files;
|
||||
}
|
||||
|
||||
// NOTE: only used in-RAM by IW to track buffered deletes;
|
||||
// this is never written to/read from the Directory
|
||||
private long bufferedDeletesGen;
|
||||
|
||||
long getBufferedDeletesGen() {
|
||||
return bufferedDeletesGen;
|
||||
}
|
||||
|
||||
void setBufferedDeletesGen(long v) {
|
||||
bufferedDeletesGen = v;
|
||||
sizeInBytes = -1;
|
||||
if (bufferedDeletesGen == -1) {
|
||||
bufferedDeletesGen = v;
|
||||
sizeInBytes = -1;
|
||||
} else {
|
||||
throw new IllegalStateException("buffered deletes gen should only be set once");
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns true if there are any deletions for the
|
||||
|
@ -92,7 +92,6 @@ final class SegmentCoreReaders {
|
||||
|
||||
final Codec codec = si.info.getCodec();
|
||||
final Directory cfsDir; // confusing name: if (cfs) it's the cfsdir, otherwise it's the segment's directory.
|
||||
|
||||
boolean success = false;
|
||||
|
||||
try {
|
||||
@ -164,7 +163,6 @@ final class SegmentCoreReaders {
|
||||
|
||||
void decRef() throws IOException {
|
||||
if (ref.decrementAndGet() == 0) {
|
||||
// System.err.println("--- closing core readers");
|
||||
Throwable th = null;
|
||||
try {
|
||||
IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
|
||||
|
@ -172,7 +172,7 @@ public final class SegmentInfo {
|
||||
/** Return all files referenced by this SegmentInfo. */
|
||||
public Set<String> files() {
|
||||
if (setFiles == null) {
|
||||
throw new IllegalStateException("files were not computed yet");
|
||||
throw new IllegalStateException("files were not computed yet; segment=" + name + " maxDoc=" + maxDoc);
|
||||
}
|
||||
return Collections.unmodifiableSet(setFiles);
|
||||
}
|
||||
|
@ -158,15 +158,22 @@ public final class SegmentReader extends CodecReader {
|
||||
* init most recent DocValues for the current commit
|
||||
*/
|
||||
private DocValuesProducer initDocValuesProducer() throws IOException {
|
||||
final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
|
||||
|
||||
if (!fieldInfos.hasDocValues()) {
|
||||
if (fieldInfos.hasDocValues() == false) {
|
||||
return null;
|
||||
} else if (si.hasFieldUpdates()) {
|
||||
return new SegmentDocValuesProducer(si, dir, core.coreFieldInfos, fieldInfos, segDocValues);
|
||||
} else {
|
||||
// simple case, no DocValues updates
|
||||
return segDocValues.getDocValuesProducer(-1L, si, dir, fieldInfos);
|
||||
Directory dir;
|
||||
if (core.cfsReader != null) {
|
||||
dir = core.cfsReader;
|
||||
} else {
|
||||
dir = si.info.dir;
|
||||
}
|
||||
if (si.hasFieldUpdates()) {
|
||||
return new SegmentDocValuesProducer(si, dir, core.coreFieldInfos, fieldInfos, segDocValues);
|
||||
} else {
|
||||
// simple case, no DocValues updates
|
||||
return segDocValues.getDocValuesProducer(-1L, si, dir, fieldInfos);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -32,11 +32,11 @@ public class SerialMergeScheduler extends MergeScheduler {
|
||||
* multiple threads, only one merge may run at a time. */
|
||||
@Override
|
||||
synchronized public void merge(IndexWriter writer, MergeTrigger trigger, boolean newMergesFound) throws IOException {
|
||||
|
||||
while(true) {
|
||||
MergePolicy.OneMerge merge = writer.getNextMerge();
|
||||
if (merge == null)
|
||||
if (merge == null) {
|
||||
break;
|
||||
}
|
||||
writer.merge(merge);
|
||||
}
|
||||
}
|
||||
|
@ -22,6 +22,7 @@ import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
@ -237,26 +238,20 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
|
||||
private class SegmentByteSizeDescending implements Comparator<SegmentCommitInfo> {
|
||||
|
||||
private final IndexWriter writer;
|
||||
private final Map<SegmentCommitInfo, Long> sizeInBytes;
|
||||
|
||||
SegmentByteSizeDescending(IndexWriter writer) {
|
||||
this.writer = writer;
|
||||
SegmentByteSizeDescending(Map<SegmentCommitInfo, Long> sizeInBytes) {
|
||||
this.sizeInBytes = sizeInBytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(SegmentCommitInfo o1, SegmentCommitInfo o2) {
|
||||
try {
|
||||
final long sz1 = size(o1, writer);
|
||||
final long sz2 = size(o2, writer);
|
||||
if (sz1 > sz2) {
|
||||
return -1;
|
||||
} else if (sz2 > sz1) {
|
||||
return 1;
|
||||
} else {
|
||||
return o1.info.name.compareTo(o2.info.name);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new RuntimeException(ioe);
|
||||
// Sort by largest size:
|
||||
int cmp = Long.compare(sizeInBytes.get(o2), sizeInBytes.get(o1));
|
||||
if (cmp == 0) {
|
||||
cmp = o1.info.name.compareTo(o2.info.name);
|
||||
}
|
||||
return cmp;
|
||||
}
|
||||
}
|
||||
|
||||
@ -277,6 +272,14 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
abstract String getExplanation();
|
||||
}
|
||||
|
||||
private Map<SegmentCommitInfo,Long> getSegmentSizes(IndexWriter writer, Collection<SegmentCommitInfo> infos) throws IOException {
|
||||
Map<SegmentCommitInfo,Long> sizeInBytes = new HashMap<>();
|
||||
for (SegmentCommitInfo info : infos) {
|
||||
sizeInBytes.put(info, size(info, writer));
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findMerges(MergeTrigger mergeTrigger, SegmentInfos infos, IndexWriter writer) throws IOException {
|
||||
if (verbose(writer)) {
|
||||
@ -289,13 +292,19 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
final Collection<SegmentCommitInfo> toBeMerged = new HashSet<>();
|
||||
|
||||
final List<SegmentCommitInfo> infosSorted = new ArrayList<>(infos.asList());
|
||||
Collections.sort(infosSorted, new SegmentByteSizeDescending(writer));
|
||||
|
||||
// The size can change concurrently while we are running here, because deletes
|
||||
// are now applied concurrently, and this can piss off TimSort! So we
|
||||
// call size() once per segment and sort by that:
|
||||
Map<SegmentCommitInfo,Long> sizeInBytes = getSegmentSizes(writer, infos.asList());
|
||||
|
||||
Collections.sort(infosSorted, new SegmentByteSizeDescending(sizeInBytes));
|
||||
|
||||
// Compute total index bytes & print details about the index
|
||||
long totIndexBytes = 0;
|
||||
long minSegmentBytes = Long.MAX_VALUE;
|
||||
for(SegmentCommitInfo info : infosSorted) {
|
||||
final long segBytes = size(info, writer);
|
||||
final long segBytes = sizeInBytes.get(info);
|
||||
if (verbose(writer)) {
|
||||
String extra = merging.contains(info) ? " [merging]" : "";
|
||||
if (segBytes >= maxMergedSegmentBytes/2.0) {
|
||||
@ -315,7 +324,7 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
// of the maxSegmentCount:
|
||||
int tooBigCount = 0;
|
||||
while (tooBigCount < infosSorted.size()) {
|
||||
long segBytes = size(infosSorted.get(tooBigCount), writer);
|
||||
long segBytes = sizeInBytes.get(infosSorted.get(tooBigCount));
|
||||
if (segBytes < maxMergedSegmentBytes/2.0) {
|
||||
break;
|
||||
}
|
||||
@ -355,7 +364,7 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
for(int idx = tooBigCount; idx<infosSorted.size(); idx++) {
|
||||
final SegmentCommitInfo info = infosSorted.get(idx);
|
||||
if (merging.contains(info)) {
|
||||
mergingBytes += size(info, writer);
|
||||
mergingBytes += sizeInBytes.get(info);
|
||||
} else if (!toBeMerged.contains(info)) {
|
||||
eligible.add(info);
|
||||
}
|
||||
@ -388,7 +397,7 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
boolean hitTooLarge = false;
|
||||
for(int idx = startIdx;idx<eligible.size() && candidate.size() < maxMergeAtOnce;idx++) {
|
||||
final SegmentCommitInfo info = eligible.get(idx);
|
||||
final long segBytes = size(info, writer);
|
||||
final long segBytes = sizeInBytes.get(info);
|
||||
|
||||
if (totAfterMergeBytes + segBytes > maxMergedSegmentBytes) {
|
||||
hitTooLarge = true;
|
||||
@ -408,7 +417,7 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
// segments, and already pre-excluded the too-large segments:
|
||||
assert candidate.size() > 0;
|
||||
|
||||
final MergeScore score = score(candidate, hitTooLarge, mergingBytes, writer);
|
||||
final MergeScore score = score(candidate, hitTooLarge, mergingBytes, writer, sizeInBytes);
|
||||
if (verbose(writer)) {
|
||||
message(" maybe=" + writer.segString(candidate) + " score=" + score.getScore() + " " + score.getExplanation() + " tooLarge=" + hitTooLarge + " size=" + String.format(Locale.ROOT, "%.3f MB", totAfterMergeBytes/1024./1024.), writer);
|
||||
}
|
||||
@ -447,12 +456,12 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
}
|
||||
|
||||
/** Expert: scores one merge; subclasses can override. */
|
||||
protected MergeScore score(List<SegmentCommitInfo> candidate, boolean hitTooLarge, long mergingBytes, IndexWriter writer) throws IOException {
|
||||
protected MergeScore score(List<SegmentCommitInfo> candidate, boolean hitTooLarge, long mergingBytes, IndexWriter writer, Map<SegmentCommitInfo, Long> sizeInBytes) throws IOException {
|
||||
long totBeforeMergeBytes = 0;
|
||||
long totAfterMergeBytes = 0;
|
||||
long totAfterMergeBytesFloored = 0;
|
||||
for(SegmentCommitInfo info : candidate) {
|
||||
final long segBytes = size(info, writer);
|
||||
final long segBytes = sizeInBytes.get(info);
|
||||
totAfterMergeBytes += segBytes;
|
||||
totAfterMergeBytesFloored += floorSize(segBytes);
|
||||
totBeforeMergeBytes += info.sizeInBytes();
|
||||
@ -472,7 +481,7 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
// over time:
|
||||
skew = 1.0/maxMergeAtOnce;
|
||||
} else {
|
||||
skew = ((double) floorSize(size(candidate.get(0), writer)))/totAfterMergeBytesFloored;
|
||||
skew = ((double) floorSize(sizeInBytes.get(candidate.get(0))))/totAfterMergeBytesFloored;
|
||||
}
|
||||
|
||||
// Strongly favor merges with less skew (smaller
|
||||
@ -519,7 +528,7 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
final Boolean isOriginal = segmentsToMerge.get(info);
|
||||
if (isOriginal != null) {
|
||||
segmentIsOriginal = isOriginal;
|
||||
if (!merging.contains(info)) {
|
||||
if (merging.contains(info) == false) {
|
||||
eligible.add(info);
|
||||
} else {
|
||||
forceMergeRunning = true;
|
||||
@ -531,6 +540,11 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
return null;
|
||||
}
|
||||
|
||||
// The size can change concurrently while we are running here, because deletes
|
||||
// are now applied concurrently, and this can piss off TimSort! So we
|
||||
// call size() once per segment and sort by that:
|
||||
Map<SegmentCommitInfo,Long> sizeInBytes = getSegmentSizes(writer, eligible);
|
||||
|
||||
if ((maxSegmentCount > 1 && eligible.size() <= maxSegmentCount) ||
|
||||
(maxSegmentCount == 1 && eligible.size() == 1 && (!segmentIsOriginal || isMerged(infos, eligible.get(0), writer)))) {
|
||||
if (verbose(writer)) {
|
||||
@ -539,7 +553,7 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
return null;
|
||||
}
|
||||
|
||||
Collections.sort(eligible, new SegmentByteSizeDescending(writer));
|
||||
Collections.sort(eligible, new SegmentByteSizeDescending(sizeInBytes));
|
||||
|
||||
if (verbose(writer)) {
|
||||
message("eligible=" + eligible, writer);
|
||||
@ -595,7 +609,12 @@ public class TieredMergePolicy extends MergePolicy {
|
||||
return null;
|
||||
}
|
||||
|
||||
Collections.sort(eligible, new SegmentByteSizeDescending(writer));
|
||||
// The size can change concurrently while we are running here, because deletes
|
||||
// are now applied concurrently, and this can piss off TimSort! So we
|
||||
// call size() once per segment and sort by that:
|
||||
Map<SegmentCommitInfo,Long> sizeInBytes = getSegmentSizes(writer, infos.asList());
|
||||
|
||||
Collections.sort(eligible, new SegmentByteSizeDescending(sizeInBytes));
|
||||
|
||||
if (verbose(writer)) {
|
||||
message("eligible=" + eligible, writer);
|
||||
|
@ -84,7 +84,7 @@ abstract class AbstractPagedMutable<T extends AbstractPagedMutable<T>> extends L
|
||||
|
||||
@Override
|
||||
public final long get(long index) {
|
||||
assert index >= 0 && index < size;
|
||||
assert index >= 0 && index < size: "index=" + index + " size=" + size;
|
||||
final int pageIndex = pageIndex(index);
|
||||
final int indexInPage = indexInPage(index);
|
||||
return subMutables[pageIndex].get(indexInPage);
|
||||
|
@ -17,7 +17,9 @@
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
@ -31,10 +33,13 @@ import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.NRTCachingDirectory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
@ -42,7 +47,6 @@ import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.junit.Test;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
|
||||
@ -204,12 +208,15 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
||||
writer.commit();
|
||||
reader1 = DirectoryReader.open(dir);
|
||||
}
|
||||
System.out.println("TEST: isNRT=" + isNRT + " reader1=" + reader1);
|
||||
|
||||
// update doc
|
||||
writer.updateBinaryDocValue(new Term("id", "doc-0"), "val", toBytes(10)); // update doc-0's value to 10
|
||||
if (!isNRT) {
|
||||
writer.commit();
|
||||
}
|
||||
|
||||
System.out.println("TEST: now reopen");
|
||||
|
||||
// reopen reader and assert only it sees the update
|
||||
final DirectoryReader reader2 = DirectoryReader.openIfChanged(reader1);
|
||||
@ -545,7 +552,7 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
||||
long value = rnd + 1;
|
||||
writer.updateBinaryDocValue(new Term("key", "doc"), "bdv", toBytes(value));
|
||||
|
||||
if (random.nextDouble() < 0.2) { // randomly delete some docs
|
||||
if (random.nextDouble() < 0.2) { // randomly delete one doc
|
||||
writer.deleteDocuments(new Term("id", Integer.toString(random.nextInt(docid))));
|
||||
}
|
||||
|
||||
@ -623,6 +630,140 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
static class OneSortDoc implements Comparable<OneSortDoc> {
|
||||
public BytesRef value;
|
||||
public final long sortValue;
|
||||
public final int id;
|
||||
public boolean deleted;
|
||||
|
||||
public OneSortDoc(int id, BytesRef value, long sortValue) {
|
||||
this.value = value;
|
||||
this.sortValue = sortValue;
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(OneSortDoc other) {
|
||||
int cmp = Long.compare(sortValue, other.sortValue);
|
||||
if (cmp == 0) {
|
||||
cmp = Integer.compare(id, other.id);
|
||||
assert cmp != 0;
|
||||
}
|
||||
return cmp;
|
||||
}
|
||||
}
|
||||
|
||||
public void testSortedIndex() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
iwc.setIndexSort(new Sort(new SortField("sort", SortField.Type.LONG)));
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
|
||||
int valueRange = TestUtil.nextInt(random(), 1, 1000);
|
||||
int sortValueRange = TestUtil.nextInt(random(), 1, 1000);
|
||||
|
||||
int refreshChance = TestUtil.nextInt(random(), 5, 200);
|
||||
int deleteChance = TestUtil.nextInt(random(), 2, 100);
|
||||
|
||||
int idUpto = 0;
|
||||
int deletedCount = 0;
|
||||
|
||||
List<OneSortDoc> docs = new ArrayList<>();
|
||||
DirectoryReader r = w.getReader();
|
||||
|
||||
int numIters = atLeast(1000);
|
||||
for(int iter=0;iter<numIters;iter++) {
|
||||
BytesRef value = toBytes((long) random().nextInt(valueRange));
|
||||
if (docs.isEmpty() || random().nextInt(3) == 1) {
|
||||
int id = docs.size();
|
||||
// add new doc
|
||||
Document doc = new Document();
|
||||
doc.add(newStringField("id", Integer.toString(id), Field.Store.YES));
|
||||
doc.add(new BinaryDocValuesField("number", value));
|
||||
int sortValue = random().nextInt(sortValueRange);
|
||||
doc.add(new NumericDocValuesField("sort", sortValue));
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: iter=" + iter + " add doc id=" + id + " sortValue=" + sortValue + " value=" + value);
|
||||
}
|
||||
w.addDocument(doc);
|
||||
|
||||
docs.add(new OneSortDoc(id, value, sortValue));
|
||||
} else {
|
||||
// update existing doc value
|
||||
int idToUpdate = random().nextInt(docs.size());
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: iter=" + iter + " update doc id=" + idToUpdate + " new value=" + value);
|
||||
}
|
||||
w.updateBinaryDocValue(new Term("id", Integer.toString(idToUpdate)), "number", value);
|
||||
|
||||
docs.get(idToUpdate).value = value;
|
||||
}
|
||||
|
||||
if (random().nextInt(deleteChance) == 0) {
|
||||
int idToDelete = random().nextInt(docs.size());
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: delete doc id=" + idToDelete);
|
||||
}
|
||||
w.deleteDocuments(new Term("id", Integer.toString(idToDelete)));
|
||||
if (docs.get(idToDelete).deleted == false) {
|
||||
docs.get(idToDelete).deleted = true;
|
||||
deletedCount++;
|
||||
}
|
||||
}
|
||||
|
||||
if (random().nextInt(refreshChance) == 0) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now get reader; old reader=" + r);
|
||||
}
|
||||
DirectoryReader r2 = w.getReader();
|
||||
r.close();
|
||||
r = r2;
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: got reader=" + r);
|
||||
}
|
||||
|
||||
int liveCount = 0;
|
||||
|
||||
for (LeafReaderContext ctx : r.leaves()) {
|
||||
LeafReader leafReader = ctx.reader();
|
||||
BinaryDocValues values = leafReader.getBinaryDocValues("number");
|
||||
NumericDocValues sortValues = leafReader.getNumericDocValues("sort");
|
||||
Bits liveDocs = leafReader.getLiveDocs();
|
||||
|
||||
long lastSortValue = Long.MIN_VALUE;
|
||||
for (int i=0;i<leafReader.maxDoc();i++) {
|
||||
|
||||
Document doc = leafReader.document(i);
|
||||
OneSortDoc sortDoc = docs.get(Integer.parseInt(doc.get("id")));
|
||||
|
||||
assertEquals(i, values.nextDoc());
|
||||
assertEquals(i, sortValues.nextDoc());
|
||||
|
||||
if (liveDocs != null && liveDocs.get(i) == false) {
|
||||
assertTrue(sortDoc.deleted);
|
||||
continue;
|
||||
}
|
||||
assertFalse(sortDoc.deleted);
|
||||
|
||||
assertEquals(sortDoc.value, values.binaryValue());
|
||||
|
||||
long sortValue = sortValues.longValue();
|
||||
assertEquals(sortDoc.sortValue, sortValue);
|
||||
|
||||
assertTrue(sortValue >= lastSortValue);
|
||||
lastSortValue = sortValue;
|
||||
liveCount++;
|
||||
}
|
||||
}
|
||||
|
||||
assertEquals(docs.size() - deletedCount, liveCount);
|
||||
}
|
||||
}
|
||||
|
||||
IOUtils.close(r, w, dir);
|
||||
}
|
||||
|
||||
public void testManyReopensAndFields() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
final Random random = random();
|
||||
@ -1283,7 +1424,6 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIOContext() throws Exception {
|
||||
// LUCENE-5591: make sure we pass an IOContext with an approximate
|
||||
// segmentSize in FlushInfo
|
||||
|
@ -34,8 +34,8 @@ import org.apache.lucene.util.ThreadInterruptedException;
|
||||
*/
|
||||
public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
||||
|
||||
public void testUpdateDelteSlices() {
|
||||
DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue();
|
||||
public void testUpdateDelteSlices() throws Exception {
|
||||
DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue(null);
|
||||
final int size = 200 + random().nextInt(500) * RANDOM_MULTIPLIER;
|
||||
Integer[] ids = new Integer[size];
|
||||
for (int i = 0; i < ids.length; i++) {
|
||||
@ -70,11 +70,11 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
||||
}
|
||||
assertEquals(j+1, queue.numGlobalTermDeletes());
|
||||
}
|
||||
assertEquals(uniqueValues, bd1.terms.keySet());
|
||||
assertEquals(uniqueValues, bd2.terms.keySet());
|
||||
assertEquals(uniqueValues, bd1.deleteTerms.keySet());
|
||||
assertEquals(uniqueValues, bd2.deleteTerms.keySet());
|
||||
HashSet<Term> frozenSet = new HashSet<>();
|
||||
BytesRefBuilder bytesRef = new BytesRefBuilder();
|
||||
TermIterator iter = queue.freezeGlobalBuffer(null).termIterator();
|
||||
TermIterator iter = queue.freezeGlobalBuffer(null).deleteTerms.iterator();
|
||||
while (iter.next() != null) {
|
||||
bytesRef.copyBytes(iter.bytes);
|
||||
frozenSet.add(new Term(iter.field(), bytesRef.toBytesRef()));
|
||||
@ -87,13 +87,12 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
||||
private void assertAllBetween(int start, int end, BufferedUpdates deletes,
|
||||
Integer[] ids) {
|
||||
for (int i = start; i <= end; i++) {
|
||||
assertEquals(Integer.valueOf(end), deletes.terms.get(new Term("id",
|
||||
ids[i].toString())));
|
||||
assertEquals(Integer.valueOf(end), deletes.deleteTerms.get(new Term("id", ids[i].toString())));
|
||||
}
|
||||
}
|
||||
|
||||
public void testClear() {
|
||||
DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue();
|
||||
DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue(null);
|
||||
assertFalse(queue.anyChanges());
|
||||
queue.clear();
|
||||
assertFalse(queue.anyChanges());
|
||||
@ -115,8 +114,8 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
||||
|
||||
}
|
||||
|
||||
public void testAnyChanges() {
|
||||
DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue();
|
||||
public void testAnyChanges() throws Exception {
|
||||
DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue(null);
|
||||
final int size = 200 + random().nextInt(500) * RANDOM_MULTIPLIER;
|
||||
int termsSinceFreeze = 0;
|
||||
int queriesSinceFreeze = 0;
|
||||
@ -132,8 +131,8 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
||||
assertTrue(queue.anyChanges());
|
||||
if (random().nextInt(5) == 0) {
|
||||
FrozenBufferedUpdates freezeGlobalBuffer = queue.freezeGlobalBuffer(null);
|
||||
assertEquals(termsSinceFreeze, freezeGlobalBuffer.terms.size());
|
||||
assertEquals(queriesSinceFreeze, freezeGlobalBuffer.queries.length);
|
||||
assertEquals(termsSinceFreeze, freezeGlobalBuffer.deleteTerms.size());
|
||||
assertEquals(queriesSinceFreeze, freezeGlobalBuffer.deleteQueries.length);
|
||||
queriesSinceFreeze = 0;
|
||||
termsSinceFreeze = 0;
|
||||
assertFalse(queue.anyChanges());
|
||||
@ -141,10 +140,8 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
||||
}
|
||||
}
|
||||
|
||||
public void testPartiallyAppliedGlobalSlice() throws SecurityException,
|
||||
NoSuchFieldException, IllegalArgumentException, IllegalAccessException,
|
||||
InterruptedException {
|
||||
final DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue();
|
||||
public void testPartiallyAppliedGlobalSlice() throws Exception {
|
||||
final DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue(null);
|
||||
ReentrantLock lock = queue.globalBufferLock;
|
||||
lock.lock();
|
||||
Thread t = new Thread() {
|
||||
@ -161,12 +158,12 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
||||
assertTrue("changes in global buffer", queue.anyChanges());
|
||||
FrozenBufferedUpdates freezeGlobalBuffer = queue.freezeGlobalBuffer(null);
|
||||
assertTrue(freezeGlobalBuffer.any());
|
||||
assertEquals(1, freezeGlobalBuffer.terms.size());
|
||||
assertEquals(1, freezeGlobalBuffer.deleteTerms.size());
|
||||
assertFalse("all changes applied", queue.anyChanges());
|
||||
}
|
||||
|
||||
public void testStressDeleteQueue() throws InterruptedException {
|
||||
DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue();
|
||||
public void testStressDeleteQueue() throws Exception {
|
||||
DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue(null);
|
||||
Set<Term> uniqueValues = new HashSet<>();
|
||||
final int size = 10000 + random().nextInt(500) * RANDOM_MULTIPLIER;
|
||||
Integer[] ids = new Integer[size];
|
||||
@ -192,13 +189,13 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
||||
queue.updateSlice(slice);
|
||||
BufferedUpdates deletes = updateThread.deletes;
|
||||
slice.apply(deletes, BufferedUpdates.MAX_INT);
|
||||
assertEquals(uniqueValues, deletes.terms.keySet());
|
||||
assertEquals(uniqueValues, deletes.deleteTerms.keySet());
|
||||
}
|
||||
queue.tryApplyGlobalSlice();
|
||||
Set<Term> frozenSet = new HashSet<>();
|
||||
BytesRefBuilder builder = new BytesRefBuilder();
|
||||
|
||||
TermIterator iter = queue.freezeGlobalBuffer(null).termIterator();
|
||||
TermIterator iter = queue.freezeGlobalBuffer(null).deleteTerms.iterator();
|
||||
while (iter.next() != null) {
|
||||
builder.copyBytes(iter.bytes);
|
||||
frozenSet.add(new Term(iter.field(), builder.toBytesRef()));
|
||||
|
@ -74,11 +74,9 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
|
||||
iwc.setIndexerThreadPool(threadPool);
|
||||
iwc.setRAMBufferSizeMB(maxRamMB);
|
||||
iwc.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
iwc.setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
IndexWriter writer = new IndexWriter(dir, iwc);
|
||||
flushPolicy = (MockDefaultFlushPolicy) writer.getConfig().getFlushPolicy();
|
||||
assertFalse(flushPolicy.flushOnDocCount());
|
||||
assertFalse(flushPolicy.flushOnDeleteTerms());
|
||||
assertTrue(flushPolicy.flushOnRAM());
|
||||
DocumentsWriter docsWriter = writer.getDocsWriter();
|
||||
assertNotNull(docsWriter);
|
||||
@ -131,11 +129,9 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
|
||||
iwc.setIndexerThreadPool(threadPool);
|
||||
iwc.setMaxBufferedDocs(2 + atLeast(10));
|
||||
iwc.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
iwc.setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
IndexWriter writer = new IndexWriter(dir, iwc);
|
||||
flushPolicy = (MockDefaultFlushPolicy) writer.getConfig().getFlushPolicy();
|
||||
assertTrue(flushPolicy.flushOnDocCount());
|
||||
assertFalse(flushPolicy.flushOnDeleteTerms());
|
||||
assertFalse(flushPolicy.flushOnRAM());
|
||||
DocumentsWriter docsWriter = writer.getDocsWriter();
|
||||
assertNotNull(docsWriter);
|
||||
@ -201,8 +197,7 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
|
||||
assertEquals(" all flushes must be due", 0, flushControl.flushBytes());
|
||||
assertEquals(numDocumentsToIndex, writer.numDocs());
|
||||
assertEquals(numDocumentsToIndex, writer.maxDoc());
|
||||
if (flushPolicy.flushOnRAM() && !flushPolicy.flushOnDocCount()
|
||||
&& !flushPolicy.flushOnDeleteTerms()) {
|
||||
if (flushPolicy.flushOnRAM() && !flushPolicy.flushOnDocCount()) {
|
||||
final long maxRAMBytes = (long) (iwc.getRAMBufferSizeMB() * 1024. * 1024.);
|
||||
assertTrue("peak bytes without flush exceeded watermark",
|
||||
flushPolicy.peakBytesWithoutFlush <= maxRAMBytes);
|
||||
@ -239,7 +234,6 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
|
||||
dir.setThrottling(MockDirectoryWrapper.Throttling.SOMETIMES);
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
|
||||
iwc.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
iwc.setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
FlushPolicy flushPolicy = new FlushByRamOrCountsPolicy();
|
||||
iwc.setFlushPolicy(flushPolicy);
|
||||
|
||||
@ -346,10 +340,6 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
|
||||
final ThreadState toFlush;
|
||||
if (state.flushPending) {
|
||||
toFlush = state;
|
||||
} else if (flushOnDeleteTerms()
|
||||
&& state.dwpt.pendingUpdates.numTermDeletes.get() >= indexWriterConfig
|
||||
.getMaxBufferedDeleteTerms()) {
|
||||
toFlush = state;
|
||||
} else {
|
||||
toFlush = null;
|
||||
}
|
||||
|
@ -56,8 +56,11 @@ public class TestForceMergeForever extends LuceneTestCase {
|
||||
final Directory d = newDirectory();
|
||||
MockAnalyzer analyzer = new MockAnalyzer(random());
|
||||
analyzer.setMaxTokenLength(TestUtil.nextInt(random(), 1, IndexWriter.MAX_TERM_LENGTH));
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
|
||||
// SMS can cause this test to run indefinitely long:
|
||||
iwc.setMergeScheduler(new ConcurrentMergeScheduler());
|
||||
|
||||
final MyIndexWriter w = new MyIndexWriter(d, newIndexWriterConfig(analyzer));
|
||||
final MyIndexWriter w = new MyIndexWriter(d, iwc);
|
||||
|
||||
// Try to make an index that requires merging:
|
||||
w.getConfig().setMaxBufferedDocs(TestUtil.nextInt(random(), 2, 11));
|
||||
@ -85,7 +88,7 @@ public class TestForceMergeForever extends LuceneTestCase {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
while (!doStop.get()) {
|
||||
while (doStop.get() == false) {
|
||||
w.updateDocument(new Term("docid", "" + random().nextInt(numStartDocs)),
|
||||
docs.nextDoc());
|
||||
// Force deletes to apply
|
||||
|
@ -357,67 +357,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testChangingRAMBuffer2() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
writer.getConfig().setMaxBufferedDocs(10);
|
||||
writer.getConfig().setMaxBufferedDeleteTerms(10);
|
||||
writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
|
||||
for(int j=1;j<52;j++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("field", "aaa" + j, storedTextType));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
|
||||
int lastFlushCount = -1;
|
||||
for(int j=1;j<52;j++) {
|
||||
writer.deleteDocuments(new Term("field", "aaa" + j));
|
||||
TestUtil.syncConcurrentMerges(writer);
|
||||
int flushCount = writer.getFlushCount();
|
||||
|
||||
if (j == 1)
|
||||
lastFlushCount = flushCount;
|
||||
else if (j < 10) {
|
||||
// No new files should be created
|
||||
assertEquals(flushCount, lastFlushCount);
|
||||
} else if (10 == j) {
|
||||
assertTrue("" + j, flushCount > lastFlushCount);
|
||||
lastFlushCount = flushCount;
|
||||
writer.getConfig().setRAMBufferSizeMB(0.000001);
|
||||
writer.getConfig().setMaxBufferedDeleteTerms(1);
|
||||
} else if (j < 20) {
|
||||
assertTrue(flushCount > lastFlushCount);
|
||||
lastFlushCount = flushCount;
|
||||
} else if (20 == j) {
|
||||
writer.getConfig().setRAMBufferSizeMB(16);
|
||||
writer.getConfig().setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
lastFlushCount = flushCount;
|
||||
} else if (j < 30) {
|
||||
assertEquals(flushCount, lastFlushCount);
|
||||
} else if (30 == j) {
|
||||
writer.getConfig().setRAMBufferSizeMB(0.000001);
|
||||
writer.getConfig().setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
writer.getConfig().setMaxBufferedDeleteTerms(1);
|
||||
} else if (j < 40) {
|
||||
assertTrue(flushCount> lastFlushCount);
|
||||
lastFlushCount = flushCount;
|
||||
} else if (40 == j) {
|
||||
writer.getConfig().setMaxBufferedDeleteTerms(10);
|
||||
writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
lastFlushCount = flushCount;
|
||||
} else if (j < 50) {
|
||||
assertEquals(flushCount, lastFlushCount);
|
||||
writer.getConfig().setMaxBufferedDeleteTerms(10);
|
||||
writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
} else if (50 == j) {
|
||||
assertTrue(flushCount > lastFlushCount);
|
||||
}
|
||||
}
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testEnablingNorms() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
|
||||
|
@ -62,7 +62,6 @@ public class TestIndexWriterConfig extends LuceneTestCase {
|
||||
assertEquals(OpenMode.CREATE_OR_APPEND, conf.getOpenMode());
|
||||
// we don't need to assert this, it should be unspecified
|
||||
assertTrue(IndexSearcher.getDefaultSimilarity() == conf.getSimilarity());
|
||||
assertEquals(IndexWriterConfig.DEFAULT_MAX_BUFFERED_DELETE_TERMS, conf.getMaxBufferedDeleteTerms());
|
||||
assertEquals(IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB, conf.getRAMBufferSizeMB(), 0.0);
|
||||
assertEquals(IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS, conf.getMaxBufferedDocs());
|
||||
assertEquals(IndexWriterConfig.DEFAULT_READER_POOLING, conf.getReaderPooling());
|
||||
@ -176,7 +175,7 @@ public class TestIndexWriterConfig extends LuceneTestCase {
|
||||
assertEquals(IndexWriterConfig.DISABLE_AUTO_FLUSH, IndexWriterConfig.DEFAULT_MAX_BUFFERED_DELETE_TERMS);
|
||||
assertEquals(IndexWriterConfig.DISABLE_AUTO_FLUSH, IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS);
|
||||
assertEquals(16.0, IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB, 0.0);
|
||||
assertEquals(false, IndexWriterConfig.DEFAULT_READER_POOLING);
|
||||
assertEquals(true, IndexWriterConfig.DEFAULT_READER_POOLING);
|
||||
assertEquals(true, IndexWriterConfig.DEFAULT_USE_COMPOUND_FILE_SYSTEM);
|
||||
}
|
||||
|
||||
@ -232,10 +231,6 @@ public class TestIndexWriterConfig extends LuceneTestCase {
|
||||
// Test IndexingChain
|
||||
assertTrue(DocumentsWriterPerThread.defaultIndexingChain == conf.getIndexingChain());
|
||||
|
||||
expectThrows(IllegalArgumentException.class, () -> {
|
||||
conf.setMaxBufferedDeleteTerms(0);
|
||||
});
|
||||
|
||||
expectThrows(IllegalArgumentException.class, () -> {
|
||||
conf.setMaxBufferedDocs(1);
|
||||
});
|
||||
|
@ -43,9 +43,10 @@ import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.junit.Ignore;
|
||||
|
||||
@SuppressCodecs("SimpleText") // too slow here
|
||||
public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
@ -59,8 +60,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
String[] text = { "Amsterdam", "Venice" };
|
||||
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter modifier = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false))
|
||||
.setMaxBufferedDeleteTerms(1));
|
||||
IndexWriter modifier = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)));
|
||||
|
||||
FieldType custom1 = new FieldType();
|
||||
custom1.setStored(true);
|
||||
@ -99,8 +99,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter modifier = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false))
|
||||
.setMaxBufferedDocs(2)
|
||||
.setMaxBufferedDeleteTerms(2));
|
||||
.setMaxBufferedDocs(2));
|
||||
int id = 0;
|
||||
int value = 100;
|
||||
|
||||
@ -129,20 +128,6 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testMaxBufferedDeletes() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false))
|
||||
.setMaxBufferedDeleteTerms(1));
|
||||
|
||||
writer.addDocument(new Document());
|
||||
writer.deleteDocuments(new Term("foobar", "1"));
|
||||
writer.deleteDocuments(new Term("foobar", "1"));
|
||||
writer.deleteDocuments(new Term("foobar", "1"));
|
||||
assertEquals(3, writer.getFlushDeletesCount());
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// test when delete terms only apply to ram segments
|
||||
public void testRAMDeletes() throws IOException {
|
||||
for(int t=0;t<2;t++) {
|
||||
@ -151,8 +136,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
}
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter modifier = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false))
|
||||
.setMaxBufferedDocs(4)
|
||||
.setMaxBufferedDeleteTerms(4));
|
||||
.setMaxBufferedDocs(4));
|
||||
int id = 0;
|
||||
int value = 100;
|
||||
|
||||
@ -189,8 +173,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
public void testBothDeletes() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter modifier = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false))
|
||||
.setMaxBufferedDocs(100)
|
||||
.setMaxBufferedDeleteTerms(100));
|
||||
.setMaxBufferedDocs(100));
|
||||
|
||||
int id = 0;
|
||||
int value = 100;
|
||||
@ -223,8 +206,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
public void testBatchDeletes() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter modifier = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false))
|
||||
.setMaxBufferedDocs(2)
|
||||
.setMaxBufferedDeleteTerms(2));
|
||||
.setMaxBufferedDocs(2));
|
||||
|
||||
int id = 0;
|
||||
int value = 100;
|
||||
@ -263,11 +245,13 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
}
|
||||
|
||||
// test deleteAll()
|
||||
public void testDeleteAll() throws IOException {
|
||||
public void testDeleteAllSimple() throws IOException {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now start");
|
||||
}
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter modifier = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false))
|
||||
.setMaxBufferedDocs(2)
|
||||
.setMaxBufferedDeleteTerms(2));
|
||||
.setMaxBufferedDocs(2));
|
||||
|
||||
int id = 0;
|
||||
int value = 100;
|
||||
@ -275,6 +259,9 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
for (int i = 0; i < 7; i++) {
|
||||
addDoc(modifier, ++id, value);
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now commit");
|
||||
}
|
||||
modifier.commit();
|
||||
|
||||
IndexReader reader = DirectoryReader.open(dir);
|
||||
@ -285,6 +272,9 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
addDoc(modifier, 99, value);
|
||||
|
||||
// Delete all
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now delete all");
|
||||
}
|
||||
modifier.deleteAll();
|
||||
|
||||
// Delete all shouldn't be on disk yet
|
||||
@ -295,6 +285,9 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
// Add a doc and update a doc (after the deleteAll, before the commit)
|
||||
addDoc(modifier, 101, value);
|
||||
updateDoc(modifier, 102, value);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now 2nd commit");
|
||||
}
|
||||
|
||||
// commit the delete all
|
||||
modifier.commit();
|
||||
@ -308,7 +301,6 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
public void testDeleteAllNoDeadLock() throws IOException, InterruptedException {
|
||||
Directory dir = newDirectory();
|
||||
final RandomIndexWriter modifier = new RandomIndexWriter(random(), dir);
|
||||
@ -350,22 +342,36 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
}
|
||||
latch.countDown();
|
||||
while(!doneLatch.await(1, TimeUnit.MILLISECONDS)) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now deleteAll");
|
||||
}
|
||||
modifier.deleteAll();
|
||||
if (VERBOSE) {
|
||||
System.out.println("del all");
|
||||
}
|
||||
}
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now final deleteAll");
|
||||
}
|
||||
|
||||
modifier.deleteAll();
|
||||
for (Thread thread : threads) {
|
||||
thread.join();
|
||||
}
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now close");
|
||||
}
|
||||
modifier.close();
|
||||
|
||||
DirectoryReader reader = DirectoryReader.open(dir);
|
||||
assertEquals(reader.maxDoc(), 0);
|
||||
assertEquals(reader.numDocs(), 0);
|
||||
assertEquals(reader.numDeletedDocs(), 0);
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: got reader=" + reader);
|
||||
}
|
||||
assertEquals(0, reader.maxDoc());
|
||||
assertEquals(0, reader.numDocs());
|
||||
assertEquals(0, reader.numDeletedDocs(), 0);
|
||||
reader.close();
|
||||
|
||||
dir.close();
|
||||
@ -375,8 +381,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
public void testDeleteAllRollback() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter modifier = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false))
|
||||
.setMaxBufferedDocs(2)
|
||||
.setMaxBufferedDeleteTerms(2));
|
||||
.setMaxBufferedDocs(2));
|
||||
|
||||
int id = 0;
|
||||
int value = 100;
|
||||
@ -411,8 +416,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
public void testDeleteAllNRT() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter modifier = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false))
|
||||
.setMaxBufferedDocs(2)
|
||||
.setMaxBufferedDeleteTerms(2));
|
||||
.setMaxBufferedDocs(2));
|
||||
|
||||
int id = 0;
|
||||
int value = 100;
|
||||
@ -526,7 +530,6 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
IndexWriter modifier = new IndexWriter(dir,
|
||||
newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false))
|
||||
.setMaxBufferedDocs(1000)
|
||||
.setMaxBufferedDeleteTerms(1000)
|
||||
.setMergeScheduler(new ConcurrentMergeScheduler()));
|
||||
((ConcurrentMergeScheduler) modifier.getConfig().getMergeScheduler()).setSuppressExceptions();
|
||||
|
||||
@ -701,6 +704,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
startDir.close();
|
||||
}
|
||||
|
||||
@Ignore
|
||||
// This test tests that buffered deletes are cleared when
|
||||
// an Exception is hit during flush.
|
||||
public void testErrorAfterApplyDeletes() throws IOException {
|
||||
@ -722,6 +726,10 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
// don't fail during merging
|
||||
return;
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("FAIL EVAL:");
|
||||
}
|
||||
new Throwable().printStackTrace(System.out);
|
||||
if (sawMaybe && !failed) {
|
||||
boolean seen = false;
|
||||
StackTraceElement[] trace = new Exception().getStackTrace();
|
||||
@ -768,7 +776,6 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
|
||||
MockDirectoryWrapper dir = newMockDirectory();
|
||||
IndexWriter modifier = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false))
|
||||
.setMaxBufferedDeleteTerms(2)
|
||||
.setReaderPooling(false)
|
||||
.setMergePolicy(newLogMergePolicy()));
|
||||
|
||||
@ -966,8 +973,14 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
final int inc = Math.min(left, TestUtil.nextInt(random(), 1, 20));
|
||||
final int limit = upto + inc;
|
||||
while(upto < limit) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: delete id=" + ids.get(upto));
|
||||
}
|
||||
w.deleteDocuments(new Term("id", ""+ids.get(upto++)));
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now open reader");
|
||||
}
|
||||
final IndexReader r = w.getReader();
|
||||
assertEquals(NUM_DOCS - upto, r.numDocs());
|
||||
r.close();
|
||||
@ -994,9 +1007,8 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
}
|
||||
};
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(analyzer)
|
||||
.setRAMBufferSizeMB(1.0)
|
||||
.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH)
|
||||
.setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH));
|
||||
.setRAMBufferSizeMB(4.0)
|
||||
.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH));
|
||||
Document doc = new Document();
|
||||
doc.add(newTextField("field", "go 1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20", Field.Store.NO));
|
||||
int num = atLeast(3);
|
||||
@ -1077,49 +1089,6 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// LUCENE-3340: make sure deletes that we don't apply
|
||||
// during flush (ie are just pushed into the stream) are
|
||||
// in fact later flushed due to their RAM usage:
|
||||
public void testFlushPushedDeletesByCount() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
// Cannot use RandomIndexWriter because we don't want to
|
||||
// ever call commit() for this test:
|
||||
final int flushAtDelCount = atLeast(1020);
|
||||
IndexWriter w = new IndexWriter(dir,
|
||||
newIndexWriterConfig(new MockAnalyzer(random()))
|
||||
.setMaxBufferedDeleteTerms(flushAtDelCount)
|
||||
.setMaxBufferedDocs(1000)
|
||||
.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH)
|
||||
.setMergePolicy(NoMergePolicy.INSTANCE)
|
||||
.setReaderPooling(false));
|
||||
int count = 0;
|
||||
while(true) {
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", count+"", Field.Store.NO));
|
||||
final Term delTerm;
|
||||
if (count == 1010) {
|
||||
// This is the only delete that applies
|
||||
delTerm = new Term("id", ""+0);
|
||||
} else {
|
||||
// These get buffered, taking up RAM, but delete
|
||||
// nothing when applied:
|
||||
delTerm = new Term("id", "x" + count);
|
||||
}
|
||||
w.updateDocument(delTerm, doc);
|
||||
// Eventually segment 0 should get a del docs:
|
||||
// TODO: fix this test
|
||||
if (slowFileExists(dir, "_0_1.del") || slowFileExists(dir, "_0_1.liv")) {
|
||||
break;
|
||||
}
|
||||
count++;
|
||||
if (count > flushAtDelCount) {
|
||||
fail("delete's were not applied at count=" + flushAtDelCount);
|
||||
}
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// Make sure buffered (pushed) deletes don't use up so
|
||||
// much RAM that it forces long tail of tiny segments:
|
||||
@Nightly
|
||||
@ -1298,6 +1267,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
}
|
||||
|
||||
// First one triggers, but does not reflect, the merge:
|
||||
System.out.println("TEST: now get reader");
|
||||
DirectoryReader.open(w).close();
|
||||
IndexReader r = DirectoryReader.open(w);
|
||||
assertEquals(1, r.leaves().size());
|
||||
@ -1315,25 +1285,33 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
mp.setMinMergeDocs(1);
|
||||
iwc.setMergePolicy(mp);
|
||||
iwc.setMergeScheduler(new SerialMergeScheduler());
|
||||
iwc.setMaxBufferedDeleteTerms(18);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
for(int i=0;i<38;i++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: add doc " + i);
|
||||
}
|
||||
Document doc = new Document();
|
||||
doc.add(newStringField("id", ""+i, Field.Store.NO));
|
||||
w.addDocument(doc);
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: commit1");
|
||||
}
|
||||
w.commit();
|
||||
|
||||
// Deleting 18 out of the 20 docs in the first segment make it the same "level" as the other 9 which should cause a merge to kick off:
|
||||
for(int i=0;i<18;i++) {
|
||||
w.deleteDocuments(new Term("id", ""+i));
|
||||
}
|
||||
w.commit();
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: commit2");
|
||||
}
|
||||
w.close();
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(dir);
|
||||
assertEquals(1, r.leaves().size());
|
||||
r.close();
|
||||
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@ -1345,7 +1323,6 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||
mp.setMinMergeDocs(1);
|
||||
iwc.setMergePolicy(mp);
|
||||
iwc.setMergeScheduler(new SerialMergeScheduler());
|
||||
iwc.setMaxBufferedDeleteTerms(18);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
for(int i=0;i<38;i++) {
|
||||
Document doc = new Document();
|
||||
|
@ -1758,259 +1758,6 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// Make sure if we hit a transient IOException (e.g., disk
|
||||
// full), and then the exception stops (e.g., disk frees
|
||||
// up), so we successfully close IW or open an NRT
|
||||
// reader, we don't lose any deletes or updates:
|
||||
public void testNoLostDeletesOrUpdates() throws Throwable {
|
||||
int deleteCount = 0;
|
||||
int docBase = 0;
|
||||
int docCount = 0;
|
||||
|
||||
MockDirectoryWrapper dir = newMockDirectory();
|
||||
final AtomicBoolean shouldFail = new AtomicBoolean();
|
||||
dir.failOn(new MockDirectoryWrapper.Failure() {
|
||||
|
||||
@Override
|
||||
public void eval(MockDirectoryWrapper dir) throws IOException {
|
||||
if (shouldFail.get() == false) {
|
||||
// Only sometimes throw the exc, so we get
|
||||
// it sometimes on creating the file, on
|
||||
// flushing buffer, on closing the file:
|
||||
return;
|
||||
}
|
||||
|
||||
if (random().nextInt(3) != 2) {
|
||||
return;
|
||||
}
|
||||
|
||||
StackTraceElement[] trace = Thread.currentThread().getStackTrace();
|
||||
|
||||
boolean sawSeal = false;
|
||||
boolean sawWrite = false;
|
||||
for (int i = 0; i < trace.length; i++) {
|
||||
if ("sealFlushedSegment".equals(trace[i].getMethodName())) {
|
||||
sawSeal = true;
|
||||
break;
|
||||
}
|
||||
if ("writeLiveDocs".equals(trace[i].getMethodName()) || "writeFieldUpdates".equals(trace[i].getMethodName())) {
|
||||
sawWrite = true;
|
||||
}
|
||||
}
|
||||
|
||||
// Don't throw exc if we are "flushing", else
|
||||
// the segment is aborted and docs are lost:
|
||||
if (sawWrite && sawSeal == false) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now fail; thread=" + Thread.currentThread().getName() + " exc:");
|
||||
new Throwable().printStackTrace(System.out);
|
||||
}
|
||||
shouldFail.set(false);
|
||||
throw new FakeIOException();
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
RandomIndexWriter w = null;
|
||||
|
||||
boolean tragic = false;
|
||||
|
||||
for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
|
||||
int numDocs = atLeast(100);
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: iter=" + iter + " numDocs=" + numDocs + " docBase=" + docBase + " delCount=" + deleteCount);
|
||||
}
|
||||
if (w == null) {
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
w = new RandomIndexWriter(random(), dir, iwc);
|
||||
// Since we hit exc during merging, a partial
|
||||
// forceMerge can easily return when there are still
|
||||
// too many segments in the index:
|
||||
w.setDoRandomForceMergeAssert(false);
|
||||
}
|
||||
for(int i=0;i<numDocs;i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", ""+(docBase+i), Field.Store.NO));
|
||||
doc.add(new NumericDocValuesField("f", 1L));
|
||||
doc.add(new NumericDocValuesField("cf", 2L));
|
||||
doc.add(new BinaryDocValuesField("bf", TestBinaryDocValuesUpdates.toBytes(1L)));
|
||||
doc.add(new BinaryDocValuesField("bcf", TestBinaryDocValuesUpdates.toBytes(2L)));
|
||||
w.addDocument(doc);
|
||||
}
|
||||
docCount += numDocs;
|
||||
|
||||
// TODO: we could make the test more evil, by letting
|
||||
// it throw more than one exc, randomly, before "recovering"
|
||||
|
||||
// TODO: we could also install an infoStream and try
|
||||
// to fail in "more evil" places inside BDS
|
||||
|
||||
shouldFail.set(true);
|
||||
boolean doClose = false;
|
||||
try {
|
||||
for(int i=0;i<numDocs;i++) {
|
||||
if (random().nextInt(10) == 7) {
|
||||
boolean fieldUpdate = random().nextBoolean();
|
||||
int docid = docBase + i;
|
||||
if (fieldUpdate) {
|
||||
long value = iter;
|
||||
if (VERBOSE) {
|
||||
System.out.println(" update id=" + docid + " to value " + value);
|
||||
}
|
||||
Term idTerm = new Term("id", Integer.toString(docid));
|
||||
if (random().nextBoolean()) { // update only numeric field
|
||||
w.updateDocValues(idTerm, new NumericDocValuesField("f", value), new NumericDocValuesField("cf", value*2));
|
||||
} else if (random().nextBoolean()) {
|
||||
w.updateDocValues(idTerm, new BinaryDocValuesField("bf", TestBinaryDocValuesUpdates.toBytes(value)),
|
||||
new BinaryDocValuesField("bcf", TestBinaryDocValuesUpdates.toBytes(value*2)));
|
||||
} else {
|
||||
w.updateDocValues(idTerm,
|
||||
new NumericDocValuesField("f", value),
|
||||
new NumericDocValuesField("cf", value*2),
|
||||
new BinaryDocValuesField("bf", TestBinaryDocValuesUpdates.toBytes(value)),
|
||||
new BinaryDocValuesField("bcf", TestBinaryDocValuesUpdates.toBytes(value*2)));
|
||||
}
|
||||
}
|
||||
|
||||
// sometimes do both deletes and updates
|
||||
if (!fieldUpdate || random().nextBoolean()) {
|
||||
if (VERBOSE) {
|
||||
System.out.println(" delete id=" + docid);
|
||||
}
|
||||
deleteCount++;
|
||||
w.deleteDocuments(new Term("id", ""+docid));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Trigger writeLiveDocs + writeFieldUpdates so we hit fake exc:
|
||||
IndexReader r = w.getReader();
|
||||
|
||||
// Sometimes we will make it here (we only randomly
|
||||
// throw the exc):
|
||||
assertEquals(docCount-deleteCount, r.numDocs());
|
||||
r.close();
|
||||
|
||||
// Sometimes close, so the disk full happens on close:
|
||||
if (random().nextBoolean()) {
|
||||
if (VERBOSE) {
|
||||
System.out.println(" now close writer");
|
||||
}
|
||||
doClose = true;
|
||||
w.commit();
|
||||
w.close();
|
||||
w = null;
|
||||
}
|
||||
|
||||
} catch (Throwable t) {
|
||||
// FakeIOException can be thrown from mergeMiddle, in which case IW
|
||||
// registers it before our CMS gets to suppress it. IW.forceMerge later
|
||||
// throws it as a wrapped IOE, so don't fail in this case.
|
||||
if (t instanceof FakeIOException || (t.getCause() instanceof FakeIOException)) {
|
||||
// expected
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: hit expected IOE");
|
||||
}
|
||||
if (t instanceof AlreadyClosedException) {
|
||||
// FakeIOExc struck during merge and writer is now closed:
|
||||
w = null;
|
||||
tragic = true;
|
||||
}
|
||||
} else {
|
||||
throw t;
|
||||
}
|
||||
}
|
||||
shouldFail.set(false);
|
||||
|
||||
if (w != null) {
|
||||
MergeScheduler ms = w.w.getConfig().getMergeScheduler();
|
||||
if (ms instanceof ConcurrentMergeScheduler) {
|
||||
((ConcurrentMergeScheduler) ms).sync();
|
||||
}
|
||||
|
||||
if (w.w.getTragicException() != null) {
|
||||
// Tragic exc in CMS closed the writer
|
||||
w = null;
|
||||
}
|
||||
}
|
||||
|
||||
IndexReader r;
|
||||
|
||||
if (doClose && w != null) {
|
||||
if (VERBOSE) {
|
||||
System.out.println(" now 2nd close writer");
|
||||
}
|
||||
w.close();
|
||||
w = null;
|
||||
}
|
||||
|
||||
if (w == null || random().nextBoolean()) {
|
||||
// Open non-NRT reader, to make sure the "on
|
||||
// disk" bits are good:
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: verify against non-NRT reader");
|
||||
}
|
||||
if (w != null) {
|
||||
w.commit();
|
||||
}
|
||||
r = DirectoryReader.open(dir);
|
||||
} else {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: verify against NRT reader");
|
||||
}
|
||||
r = w.getReader();
|
||||
}
|
||||
if (tragic == false) {
|
||||
assertEquals(docCount-deleteCount, r.numDocs());
|
||||
}
|
||||
BytesRef scratch = new BytesRef();
|
||||
for (LeafReaderContext context : r.leaves()) {
|
||||
LeafReader reader = context.reader();
|
||||
Bits liveDocs = reader.getLiveDocs();
|
||||
NumericDocValues f = reader.getNumericDocValues("f");
|
||||
NumericDocValues cf = reader.getNumericDocValues("cf");
|
||||
BinaryDocValues bf = reader.getBinaryDocValues("bf");
|
||||
BinaryDocValues bcf = reader.getBinaryDocValues("bcf");
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
if (liveDocs == null || liveDocs.get(i)) {
|
||||
assertEquals(i, f.advance(i));
|
||||
assertEquals(i, cf.advance(i));
|
||||
assertEquals(i, bf.advance(i));
|
||||
assertEquals(i, bcf.advance(i));
|
||||
assertEquals("doc=" + (docBase + i), cf.longValue(), f.longValue() * 2);
|
||||
assertEquals("doc=" + (docBase + i), TestBinaryDocValuesUpdates.getValue(bcf), TestBinaryDocValuesUpdates.getValue(bf) * 2);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
r.close();
|
||||
|
||||
// Sometimes re-use RIW, other times open new one:
|
||||
if (w != null && random().nextBoolean()) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: close writer");
|
||||
}
|
||||
w.close();
|
||||
w = null;
|
||||
}
|
||||
|
||||
docBase += numDocs;
|
||||
}
|
||||
|
||||
if (w != null) {
|
||||
w.close();
|
||||
}
|
||||
|
||||
// Final verify:
|
||||
if (tragic == false) {
|
||||
IndexReader r = DirectoryReader.open(dir);
|
||||
assertEquals(docCount-deleteCount, r.numDocs());
|
||||
r.close();
|
||||
}
|
||||
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// kind of slow, but omits positions, so just CPU
|
||||
@Nightly
|
||||
public void testTooManyTokens() throws Exception {
|
||||
|
@ -158,6 +158,7 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
||||
writer.updateDocument(new Term("id", id10), newDoc);
|
||||
assertFalse(r1.isCurrent());
|
||||
|
||||
System.out.println("TEST: now get reader");
|
||||
DirectoryReader r2 = writer.getReader();
|
||||
assertTrue(r2.isCurrent());
|
||||
assertEquals(0, count(new Term("id", id10), r2));
|
||||
|
@ -53,16 +53,16 @@ public class TestNRTReaderWithThreads extends LuceneTestCase {
|
||||
while ((System.currentTimeMillis() - startTime) < duration) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
int delCount = 0;
|
||||
int addCount = 0;
|
||||
for (int x=0; x < indexThreads.length; x++) {
|
||||
indexThreads[x].run = false;
|
||||
assertNull("Exception thrown: "+indexThreads[x].ex, indexThreads[x].ex);
|
||||
addCount += indexThreads[x].addCount;
|
||||
delCount += indexThreads[x].delCount;
|
||||
}
|
||||
int delCount = 0;
|
||||
int addCount = 0;
|
||||
for (int x=0; x < indexThreads.length; x++) {
|
||||
indexThreads[x].join();
|
||||
addCount += indexThreads[x].addCount;
|
||||
delCount += indexThreads[x].delCount;
|
||||
}
|
||||
for (int x=0; x < indexThreads.length; x++) {
|
||||
assertNull("Exception thrown: "+indexThreads[x].ex, indexThreads[x].ex);
|
||||
|
@ -17,8 +17,10 @@
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
@ -33,6 +35,7 @@ import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||
@ -50,7 +53,6 @@ import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.junit.Test;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
|
||||
@ -83,9 +85,14 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
writer.updateDocument (new Term("id","doc-2"), doc(2, 2000000000L ));
|
||||
writer.updateDocument (new Term("id","doc-2"), doc(2, 2222222222L ));
|
||||
writer.updateNumericDocValue(new Term("id","doc-1"), "val", 1111111111L );
|
||||
writer.commit();
|
||||
|
||||
final DirectoryReader reader = DirectoryReader.open(dir);
|
||||
|
||||
final DirectoryReader reader;
|
||||
if (random().nextBoolean()) {
|
||||
writer.commit();
|
||||
reader = DirectoryReader.open(dir);
|
||||
} else {
|
||||
reader = DirectoryReader.open(writer);
|
||||
}
|
||||
final IndexSearcher searcher = new IndexSearcher(reader);
|
||||
TopFieldDocs td;
|
||||
|
||||
@ -128,14 +135,11 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
int numDocUpdates = 0;
|
||||
int numValueUpdates = 0;
|
||||
|
||||
//System.out.println("TEST: numOperations=" + numOperations + " ADD_CUTOFF=" + ADD_CUTOFF + " UPD_CUTOFF=" + UPD_CUTOFF);
|
||||
|
||||
for (int i = 0; i < numOperations; i++) {
|
||||
final int op = TestUtil.nextInt(random(), 1, 100);
|
||||
final long val = random().nextLong();
|
||||
if (op <= ADD_CUTOFF) {
|
||||
final int id = expected.size();
|
||||
//System.out.println("TEST i=" + i + ": addDocument id=" + id + " val=" + val);
|
||||
expected.put(id, val);
|
||||
writer.addDocument(doc(id, val));
|
||||
} else {
|
||||
@ -143,11 +147,9 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
expected.put(id, val);
|
||||
if (op <= UPD_CUTOFF) {
|
||||
numDocUpdates++;
|
||||
//System.out.println("TEST i=" + i + ": updateDocument id=" + id + " val=" + val);
|
||||
writer.updateDocument(new Term("id","doc-" + id), doc(id, val));
|
||||
} else {
|
||||
numValueUpdates++;
|
||||
//System.out.println("TEST i=" + i + ": updateDV id=" + id + " val=" + val);
|
||||
writer.updateNumericDocValue(new Term("id","doc-" + id), "val", val);
|
||||
}
|
||||
}
|
||||
@ -171,7 +173,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testUpdatesAreFlushed() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false))
|
||||
@ -194,7 +195,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimple() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -230,7 +230,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateFewSegments() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -279,7 +278,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReopen() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -295,6 +293,9 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
writer.commit();
|
||||
reader1 = DirectoryReader.open(dir);
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: isNRT=" + isNRT);
|
||||
}
|
||||
|
||||
// update doc
|
||||
writer.updateNumericDocValue(new Term("id", "doc-0"), "val", 10L); // update doc-0's value to 10
|
||||
@ -303,6 +304,9 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
}
|
||||
|
||||
// reopen reader and assert only it sees the update
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: openIfChanged");
|
||||
}
|
||||
final DirectoryReader reader2 = DirectoryReader.openIfChanged(reader1);
|
||||
assertNotNull(reader2);
|
||||
assertTrue(reader1 != reader2);
|
||||
@ -318,7 +322,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
IOUtils.close(reader1, reader2, dir);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdatesAndDeletes() throws Exception {
|
||||
// create an index with a segment with only deletes, a segment with both
|
||||
// deletes and updates and a segment with only updates
|
||||
@ -368,7 +371,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdatesWithDeletes() throws Exception {
|
||||
// update and delete different documents in the same commit session
|
||||
Directory dir = newDirectory();
|
||||
@ -405,7 +407,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultipleDocValuesTypes() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -460,7 +461,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultipleNumericDocValues() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -495,7 +495,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDocumentWithNoValue() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -510,24 +509,35 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
writer.commit();
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: first commit");
|
||||
}
|
||||
|
||||
// update all docs' ndv field
|
||||
writer.updateNumericDocValue(new Term("dvUpdateKey", "dv"), "ndv", 17L);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: first close");
|
||||
}
|
||||
writer.close();
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: done close");
|
||||
}
|
||||
|
||||
final DirectoryReader reader = DirectoryReader.open(dir);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: got reader=reader");
|
||||
}
|
||||
LeafReader r = reader.leaves().get(0).reader();
|
||||
NumericDocValues ndv = r.getNumericDocValues("ndv");
|
||||
for (int i = 0; i < r.maxDoc(); i++) {
|
||||
assertEquals(i, ndv.nextDoc());
|
||||
assertEquals(17, ndv.longValue());
|
||||
assertEquals("doc=" + i + " has wrong numeric doc value", 17, ndv.longValue());
|
||||
}
|
||||
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateNonNumericDocValuesField() throws Exception {
|
||||
// we don't support adding new fields or updating existing non-numeric-dv
|
||||
// fields through numeric updates
|
||||
@ -554,7 +564,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDifferentDVFormatPerField() throws Exception {
|
||||
// test relies on separate instances of the "same thing"
|
||||
assert TestUtil.getDefaultDocValuesFormat() != TestUtil.getDefaultDocValuesFormat();
|
||||
@ -595,7 +604,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateSameDocMultipleTimes() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -622,7 +630,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSegmentMerges() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
Random random = random();
|
||||
@ -631,28 +638,54 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
|
||||
int docid = 0;
|
||||
int numRounds = atLeast(10);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: " + numRounds + " rounds");
|
||||
}
|
||||
for (int rnd = 0; rnd < numRounds; rnd++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: round=" + rnd);
|
||||
}
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("key", "doc", Store.NO));
|
||||
doc.add(new NumericDocValuesField("ndv", -1));
|
||||
int numDocs = atLeast(30);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: " + numDocs + " docs");
|
||||
}
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
doc.removeField("id");
|
||||
doc.add(new StringField("id", Integer.toString(docid++), Store.NO));
|
||||
doc.add(new StringField("id", Integer.toString(docid), Store.YES));
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: add doc id=" + docid);
|
||||
}
|
||||
writer.addDocument(doc);
|
||||
docid++;
|
||||
}
|
||||
|
||||
long value = rnd + 1;
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: update all ndv values to " + value);
|
||||
}
|
||||
writer.updateNumericDocValue(new Term("key", "doc"), "ndv", value);
|
||||
|
||||
if (random.nextDouble() < 0.2) { // randomly delete some docs
|
||||
writer.deleteDocuments(new Term("id", Integer.toString(random.nextInt(docid))));
|
||||
if (random.nextDouble() < 0.2) { // randomly delete one doc
|
||||
int delID = random.nextInt(docid);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: delete random doc id=" + delID);
|
||||
}
|
||||
writer.deleteDocuments(new Term("id", Integer.toString(delID)));
|
||||
}
|
||||
|
||||
// randomly commit or reopen-IW (or nothing), before forceMerge
|
||||
if (random.nextDouble() < 0.4) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: commit writer");
|
||||
}
|
||||
writer.commit();
|
||||
} else if (random.nextDouble() < 0.1) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: close writer");
|
||||
}
|
||||
writer.close();
|
||||
conf = newIndexWriterConfig(new MockAnalyzer(random));
|
||||
writer = new IndexWriter(dir, conf);
|
||||
@ -665,28 +698,49 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
// and some MPs might now merge it, thereby invalidating test's
|
||||
// assumption that the reader has no deletes).
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", Integer.toString(docid++), Store.NO));
|
||||
doc.add(new StringField("id", Integer.toString(docid), Store.YES));
|
||||
doc.add(new StringField("key", "doc", Store.NO));
|
||||
doc.add(new NumericDocValuesField("ndv", value));
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: add one more doc id=" + docid);
|
||||
}
|
||||
writer.addDocument(doc);
|
||||
docid++;
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: force merge");
|
||||
}
|
||||
writer.forceMerge(1, true);
|
||||
|
||||
final DirectoryReader reader;
|
||||
if (random.nextBoolean()) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: commit and open non-NRT reader");
|
||||
}
|
||||
writer.commit();
|
||||
reader = DirectoryReader.open(dir);
|
||||
} else {
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: open NRT reader");
|
||||
}
|
||||
reader = DirectoryReader.open(writer);
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: got reader=" + reader);
|
||||
}
|
||||
|
||||
assertEquals(1, reader.leaves().size());
|
||||
final LeafReader r = reader.leaves().get(0).reader();
|
||||
assertNull("index should have no deletes after forceMerge", r.getLiveDocs());
|
||||
NumericDocValues ndv = r.getNumericDocValues("ndv");
|
||||
assertNotNull(ndv);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: maxDoc=" + r.maxDoc());
|
||||
}
|
||||
for (int i = 0; i < r.maxDoc(); i++) {
|
||||
Document rdoc = r.document(i);
|
||||
assertEquals(i, ndv.nextDoc());
|
||||
assertEquals(value, ndv.longValue());
|
||||
assertEquals("docid=" + i + " has wrong ndv value; doc=" + rdoc, value, ndv.longValue());
|
||||
}
|
||||
reader.close();
|
||||
}
|
||||
@ -695,7 +749,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateDocumentByMultipleTerms() throws Exception {
|
||||
// make sure the order of updates is respected, even when multiple terms affect same document
|
||||
Directory dir = newDirectory();
|
||||
@ -723,8 +776,141 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
static class OneSortDoc implements Comparable<OneSortDoc> {
|
||||
public long value;
|
||||
public final long sortValue;
|
||||
public final int id;
|
||||
public boolean deleted;
|
||||
|
||||
public OneSortDoc(int id, long value, long sortValue) {
|
||||
this.value = value;
|
||||
this.sortValue = sortValue;
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(OneSortDoc other) {
|
||||
int cmp = Long.compare(sortValue, other.sortValue);
|
||||
if (cmp == 0) {
|
||||
cmp = Integer.compare(id, other.id);
|
||||
assert cmp != 0;
|
||||
}
|
||||
return cmp;
|
||||
}
|
||||
}
|
||||
|
||||
public void testSortedIndex() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
iwc.setIndexSort(new Sort(new SortField("sort", SortField.Type.LONG)));
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
|
||||
int valueRange = TestUtil.nextInt(random(), 1, 1000);
|
||||
int sortValueRange = TestUtil.nextInt(random(), 1, 1000);
|
||||
|
||||
int refreshChance = TestUtil.nextInt(random(), 5, 200);
|
||||
int deleteChance = TestUtil.nextInt(random(), 2, 100);
|
||||
|
||||
int idUpto = 0;
|
||||
int deletedCount = 0;
|
||||
|
||||
List<OneSortDoc> docs = new ArrayList<>();
|
||||
DirectoryReader r = w.getReader();
|
||||
|
||||
int numIters = atLeast(1000);
|
||||
for(int iter=0;iter<numIters;iter++) {
|
||||
int value = random().nextInt(valueRange);
|
||||
if (docs.isEmpty() || random().nextInt(3) == 1) {
|
||||
int id = docs.size();
|
||||
// add new doc
|
||||
Document doc = new Document();
|
||||
doc.add(newStringField("id", Integer.toString(id), Field.Store.YES));
|
||||
doc.add(new NumericDocValuesField("number", value));
|
||||
int sortValue = random().nextInt(sortValueRange);
|
||||
doc.add(new NumericDocValuesField("sort", sortValue));
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: iter=" + iter + " add doc id=" + id + " sortValue=" + sortValue + " value=" + value);
|
||||
}
|
||||
w.addDocument(doc);
|
||||
|
||||
docs.add(new OneSortDoc(id, value, sortValue));
|
||||
} else {
|
||||
// update existing doc value
|
||||
int idToUpdate = random().nextInt(docs.size());
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: iter=" + iter + " update doc id=" + idToUpdate + " new value=" + value);
|
||||
}
|
||||
w.updateNumericDocValue(new Term("id", Integer.toString(idToUpdate)), "number", (long) value);
|
||||
|
||||
docs.get(idToUpdate).value = value;
|
||||
}
|
||||
|
||||
if (random().nextInt(deleteChance) == 0) {
|
||||
int idToDelete = random().nextInt(docs.size());
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: delete doc id=" + idToDelete);
|
||||
}
|
||||
w.deleteDocuments(new Term("id", Integer.toString(idToDelete)));
|
||||
if (docs.get(idToDelete).deleted == false) {
|
||||
docs.get(idToDelete).deleted = true;
|
||||
deletedCount++;
|
||||
}
|
||||
}
|
||||
|
||||
if (random().nextInt(refreshChance) == 0) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now get reader; old reader=" + r);
|
||||
}
|
||||
DirectoryReader r2 = w.getReader();
|
||||
r.close();
|
||||
r = r2;
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: got reader=" + r);
|
||||
}
|
||||
|
||||
int liveCount = 0;
|
||||
|
||||
for (LeafReaderContext ctx : r.leaves()) {
|
||||
LeafReader leafReader = ctx.reader();
|
||||
NumericDocValues values = leafReader.getNumericDocValues("number");
|
||||
NumericDocValues sortValues = leafReader.getNumericDocValues("sort");
|
||||
Bits liveDocs = leafReader.getLiveDocs();
|
||||
|
||||
long lastSortValue = Long.MIN_VALUE;
|
||||
for (int i=0;i<leafReader.maxDoc();i++) {
|
||||
|
||||
Document doc = leafReader.document(i);
|
||||
OneSortDoc sortDoc = docs.get(Integer.parseInt(doc.get("id")));
|
||||
|
||||
assertEquals(i, values.nextDoc());
|
||||
assertEquals(i, sortValues.nextDoc());
|
||||
|
||||
if (liveDocs != null && liveDocs.get(i) == false) {
|
||||
assertTrue(sortDoc.deleted);
|
||||
continue;
|
||||
}
|
||||
assertFalse(sortDoc.deleted);
|
||||
|
||||
assertEquals(sortDoc.value, values.longValue());
|
||||
|
||||
long sortValue = sortValues.longValue();
|
||||
assertEquals(sortDoc.sortValue, sortValue);
|
||||
|
||||
assertTrue(sortValue >= lastSortValue);
|
||||
lastSortValue = sortValue;
|
||||
liveCount++;
|
||||
}
|
||||
}
|
||||
|
||||
assertEquals(docs.size() - deletedCount, liveCount);
|
||||
}
|
||||
}
|
||||
|
||||
IOUtils.close(r, w, dir);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testManyReopensAndFields() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
final Random random = random();
|
||||
@ -735,6 +921,9 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
|
||||
final boolean isNRT = random.nextBoolean();
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: isNRT=" + isNRT);
|
||||
}
|
||||
DirectoryReader reader;
|
||||
if (isNRT) {
|
||||
reader = DirectoryReader.open(writer);
|
||||
@ -753,45 +942,58 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
int docID = 0;
|
||||
for (int i = 0; i < numRounds; i++) {
|
||||
int numDocs = atLeast(5);
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "]: round=" + i + ", numDocs=" + numDocs);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: round=" + i + ", numDocs=" + numDocs);
|
||||
}
|
||||
for (int j = 0; j < numDocs; j++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "doc-" + docID, Store.NO));
|
||||
doc.add(new StringField("id", "doc-" + docID, Store.YES));
|
||||
doc.add(new StringField("key", "all", Store.NO)); // update key
|
||||
// add all fields with their current value
|
||||
for (int f = 0; f < fieldValues.length; f++) {
|
||||
doc.add(new NumericDocValuesField("f" + f, fieldValues[f]));
|
||||
}
|
||||
writer.addDocument(doc);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST add doc id=" + docID);
|
||||
}
|
||||
++docID;
|
||||
}
|
||||
|
||||
int fieldIdx = random.nextInt(fieldValues.length);
|
||||
|
||||
String updateField = "f" + fieldIdx;
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: update field=" + updateField + " for all docs to value=" + (fieldValues[fieldIdx]+1));
|
||||
}
|
||||
writer.updateNumericDocValue(new Term("key", "all"), updateField, ++fieldValues[fieldIdx]);
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "]: updated field '" + updateField + "' to value " + fieldValues[fieldIdx]);
|
||||
|
||||
if (random.nextDouble() < 0.2) {
|
||||
int deleteDoc = random.nextInt(docID); // might also delete an already deleted document, ok!
|
||||
int deleteDoc = random.nextInt(numDocs); // might also delete an already deleted document, ok!
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: delete doc id=" + deleteDoc);
|
||||
}
|
||||
writer.deleteDocuments(new Term("id", "doc-" + deleteDoc));
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "]: deleted document: doc-" + deleteDoc);
|
||||
}
|
||||
|
||||
// verify reader
|
||||
if (!isNRT) {
|
||||
if (isNRT == false) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now commit");
|
||||
}
|
||||
writer.commit();
|
||||
}
|
||||
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "]: reopen reader: " + reader);
|
||||
DirectoryReader newReader = DirectoryReader.openIfChanged(reader);
|
||||
assertNotNull(newReader);
|
||||
reader.close();
|
||||
reader = newReader;
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "]: reopened reader: " + reader);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: got reader maxDoc=" + reader.maxDoc() + " " + reader);
|
||||
}
|
||||
assertTrue(reader.numDocs() > 0); // we delete at most one document per round
|
||||
for (LeafReaderContext context : reader.leaves()) {
|
||||
LeafReader r = context.reader();
|
||||
// System.out.println(((SegmentReader) r).getSegmentName());
|
||||
Bits liveDocs = r.getLiveDocs();
|
||||
for (int field = 0; field < fieldValues.length; field++) {
|
||||
String f = "f" + field;
|
||||
@ -800,21 +1002,18 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
int maxDoc = r.maxDoc();
|
||||
for (int doc = 0; doc < maxDoc; doc++) {
|
||||
if (liveDocs == null || liveDocs.get(doc)) {
|
||||
// System.out.println("doc=" + (doc + context.docBase) + " f='" + f + "' vslue=" + ndv.get(doc));
|
||||
assertEquals(doc, ndv.advance(doc));
|
||||
assertEquals("invalid value for doc=" + doc + ", field=" + f + ", reader=" + r, fieldValues[field], ndv.longValue());
|
||||
assertEquals("advanced to wrong doc in seg=" + r, doc, ndv.advance(doc));
|
||||
assertEquals("invalid value for docID=" + doc + " id=" + r.document(doc).get("id") + ", field=" + f + ", reader=" + r + " doc=" + r.document(doc), fieldValues[field], ndv.longValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
// System.out.println();
|
||||
}
|
||||
|
||||
writer.close();
|
||||
IOUtils.close(reader, dir);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateSegmentWithNoDocValues() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -866,7 +1065,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateSegmentWithNoDocValues2() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -949,7 +1147,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateSegmentWithPostingButNoDocValues() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -992,7 +1189,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateNumericDVFieldWithSameNameAsPostingField() throws Exception {
|
||||
// this used to fail because FieldInfos.Builder neglected to update
|
||||
// globalFieldMaps.docValuesTypes map
|
||||
@ -1017,7 +1213,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStressMultiThreading() throws Exception {
|
||||
final Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -1069,28 +1264,23 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
final int field = random().nextInt(numFields);
|
||||
final String f = "f" + field;
|
||||
final String cf = "cf" + field;
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] numUpdates=" + numUpdates + " updateTerm=" + t + " field=" + field);
|
||||
long updValue = random.nextInt();
|
||||
writer.updateDocValues(t, new NumericDocValuesField(f, updValue), new NumericDocValuesField(cf, updValue*2));
|
||||
|
||||
if (random.nextDouble() < 0.2) {
|
||||
// delete a random document
|
||||
int doc = random.nextInt(numDocs);
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] deleteDoc=doc" + doc);
|
||||
writer.deleteDocuments(new Term("id", "doc" + doc));
|
||||
}
|
||||
|
||||
if (random.nextDouble() < 0.05) { // commit every 20 updates on average
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] commit");
|
||||
writer.commit();
|
||||
}
|
||||
|
||||
if (random.nextDouble() < 0.1) { // reopen NRT reader (apply updates), on average once every 10 updates
|
||||
if (reader == null) {
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] open NRT");
|
||||
reader = DirectoryReader.open(writer);
|
||||
} else {
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] reopen NRT");
|
||||
DirectoryReader r2 = DirectoryReader.openIfChanged(reader, writer);
|
||||
if (r2 != null) {
|
||||
reader.close();
|
||||
@ -1099,7 +1289,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
}
|
||||
}
|
||||
}
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] DONE");
|
||||
success = true;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
@ -1144,7 +1333,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateDifferentDocsInDifferentGens() throws Exception {
|
||||
// update same document multiple times across generations
|
||||
Directory dir = newDirectory();
|
||||
@ -1184,7 +1372,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChangeCodec() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -1236,7 +1423,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddIndexes() throws Exception {
|
||||
Directory dir1 = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -1296,7 +1482,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
IOUtils.close(dir1, dir2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeleteUnusedUpdatesFiles() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -1326,7 +1511,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test @Nightly
|
||||
public void testTonsOfUpdates() throws Exception {
|
||||
// LUCENE-5248: make sure that when there are many updates, we don't use too much RAM
|
||||
Directory dir = newDirectory();
|
||||
@ -1345,8 +1529,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
updateTerms.add(TestUtil.randomSimpleString(random));
|
||||
}
|
||||
|
||||
// System.out.println("numDocs=" + numDocs + " numNumericFields=" + numNumericFields + " numTerms=" + numTerms);
|
||||
|
||||
// build a large index with many NDV fields and update terms
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
Document doc = new Document();
|
||||
@ -1368,7 +1550,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
// many flushes during numeric updates
|
||||
writer.getConfig().setRAMBufferSizeMB(2048.0 / 1024 / 1024);
|
||||
final int numUpdates = atLeast(100);
|
||||
// System.out.println("numUpdates=" + numUpdates);
|
||||
for (int i = 0; i < numUpdates; i++) {
|
||||
int field = random.nextInt(numNumericFields);
|
||||
Term updateTerm = new Term("upd", RandomPicks.randomFrom(random, updateTerms));
|
||||
@ -1396,7 +1577,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdatesOrder() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -1413,6 +1593,9 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
writer.updateNumericDocValue(new Term("upd", "t2"), "f1", 3L); // update f1 to 3
|
||||
writer.updateNumericDocValue(new Term("upd", "t2"), "f2", 3L); // update f2 to 3
|
||||
writer.updateNumericDocValue(new Term("upd", "t1"), "f1", 4L); // update f1 to 4 (but not f2)
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now close");
|
||||
}
|
||||
writer.close();
|
||||
|
||||
DirectoryReader reader = DirectoryReader.open(dir);
|
||||
@ -1427,7 +1610,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateAllDeletedSegment() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -1455,7 +1637,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateTwoNonexistingTerms() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
@ -1480,7 +1661,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIOContext() throws Exception {
|
||||
// LUCENE-5591: make sure we pass an IOContext with an approximate
|
||||
// segmentSize in FlushInfo
|
||||
|
@ -68,10 +68,11 @@ public class TestPerSegmentDeletes extends LuceneTestCase {
|
||||
|
||||
writer.deleteDocuments(new Term("id", "11"));
|
||||
|
||||
// flushing without applying deletes means
|
||||
// there will still be deletes in the segment infos
|
||||
writer.flush(false, false);
|
||||
assertTrue(writer.bufferedUpdatesStream.any());
|
||||
|
||||
// deletes are now resolved on flush, so there shouldn't be
|
||||
// any deletes after flush
|
||||
assertFalse(writer.bufferedUpdatesStream.any());
|
||||
|
||||
// get reader flushes pending deletes
|
||||
// so there should not be anymore
|
||||
|
@ -16,14 +16,11 @@
|
||||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
@ -73,77 +70,4 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
|
||||
|
||||
assertFalse(expected.hasNext());
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testMergeOne() {
|
||||
Term t1 = new Term("foo", "a");
|
||||
PrefixCodedTerms.Builder b1 = new PrefixCodedTerms.Builder();
|
||||
b1.add(t1);
|
||||
PrefixCodedTerms pb1 = b1.finish();
|
||||
|
||||
Term t2 = new Term("foo", "b");
|
||||
PrefixCodedTerms.Builder b2 = new PrefixCodedTerms.Builder();
|
||||
b2.add(t2);
|
||||
PrefixCodedTerms pb2 = b2.finish();
|
||||
|
||||
MergedPrefixCodedTermsIterator merged = new MergedPrefixCodedTermsIterator(Arrays.asList(new PrefixCodedTerms[] {pb1, pb2}));
|
||||
BytesRef term = merged.next();
|
||||
assertNotNull(term);
|
||||
assertEquals("foo", merged.field());
|
||||
assertEquals("a", term.utf8ToString());
|
||||
term = merged.next();
|
||||
assertNotNull(term);
|
||||
assertEquals("b", term.utf8ToString());
|
||||
assertNull(merged.next());
|
||||
}
|
||||
|
||||
@SuppressWarnings({"unchecked","rawtypes"})
|
||||
public void testMergeRandom() {
|
||||
PrefixCodedTerms pb[] = new PrefixCodedTerms[TestUtil.nextInt(random(), 2, 10)];
|
||||
Set<Term> superSet = new TreeSet<>();
|
||||
|
||||
for (int i = 0; i < pb.length; i++) {
|
||||
Set<Term> terms = new TreeSet<>();
|
||||
int nterms = TestUtil.nextInt(random(), 0, 10000);
|
||||
for (int j = 0; j < nterms; j++) {
|
||||
String field = TestUtil.randomUnicodeString(random(), 2);
|
||||
//String field = TestUtil.randomSimpleString(random(), 2);
|
||||
Term term = new Term(field, TestUtil.randomUnicodeString(random(), 4));
|
||||
terms.add(term);
|
||||
}
|
||||
superSet.addAll(terms);
|
||||
|
||||
PrefixCodedTerms.Builder b = new PrefixCodedTerms.Builder();
|
||||
//System.out.println("TEST: sub " + i + " has " + terms.size() + " terms");
|
||||
for (Term ref: terms) {
|
||||
//System.out.println(" add " + ref.field() + " " + ref.bytes());
|
||||
b.add(ref);
|
||||
}
|
||||
pb[i] = b.finish();
|
||||
}
|
||||
|
||||
Iterator<Term> expected = superSet.iterator();
|
||||
|
||||
MergedPrefixCodedTermsIterator actual = new MergedPrefixCodedTermsIterator(Arrays.asList(pb));
|
||||
String field = "";
|
||||
|
||||
BytesRef lastTerm = null;
|
||||
BytesRef term;
|
||||
while ((term = actual.next()) != null) {
|
||||
if (field != actual.field()) {
|
||||
field = actual.field();
|
||||
lastTerm = null;
|
||||
}
|
||||
if (lastTerm != null && lastTerm.equals(term)) {
|
||||
continue;
|
||||
}
|
||||
lastTerm = BytesRef.deepCopyOf(term);
|
||||
assertTrue(expected.hasNext());
|
||||
|
||||
Term expectedTerm = expected.next();
|
||||
assertEquals(expectedTerm, new Term(field, term));
|
||||
}
|
||||
|
||||
assertFalse(expected.hasNext());
|
||||
}
|
||||
}
|
||||
|
@ -333,9 +333,11 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
|
||||
};
|
||||
t.start();
|
||||
writer.waitAfterUpdate = true; // wait in addDocument to let some reopens go through
|
||||
|
||||
final long lastGen = writer.updateDocument(new Term("foo", "bar"), doc); // once this returns the doc is already reflected in the last reopen
|
||||
|
||||
assertFalse(manager.isSearcherCurrent()); // false since there is a delete in the queue
|
||||
// We now eagerly resolve deletes so the manager should see it after update:
|
||||
assertTrue(manager.isSearcherCurrent());
|
||||
|
||||
IndexSearcher searcher = manager.acquire();
|
||||
try {
|
||||
|
@ -1188,13 +1188,16 @@ public class TestJoinUtil extends LuceneTestCase {
|
||||
private void executeRandomJoin(boolean multipleValuesPerDocument, int maxIndexIter, int maxSearchIter, int numberOfDocumentsToIndex) throws Exception {
|
||||
for (int indexIter = 1; indexIter <= maxIndexIter; indexIter++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("indexIter=" + indexIter);
|
||||
System.out.println("TEST: indexIter=" + indexIter + " numDocs=" + numberOfDocumentsToIndex);
|
||||
}
|
||||
IndexIterationContext context = createContext(numberOfDocumentsToIndex, multipleValuesPerDocument, false);
|
||||
IndexSearcher indexSearcher = context.searcher;
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: got searcher=" + indexSearcher);
|
||||
}
|
||||
for (int searchIter = 1; searchIter <= maxSearchIter; searchIter++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("searchIter=" + searchIter);
|
||||
System.out.println("TEST: searchIter=" + searchIter);
|
||||
}
|
||||
|
||||
int r = random().nextInt(context.randomUniqueValues.length);
|
||||
@ -1360,9 +1363,9 @@ public class TestJoinUtil extends LuceneTestCase {
|
||||
}
|
||||
final List<String> subValues;
|
||||
{
|
||||
int start = randomUniqueValuesReplica.size()==numberOfLinkValues? 0 : random.nextInt(randomUniqueValuesReplica.size()-numberOfLinkValues);
|
||||
subValues = randomUniqueValuesReplica.subList(start, start+numberOfLinkValues);
|
||||
Collections.shuffle(subValues, random);
|
||||
int start = randomUniqueValuesReplica.size()==numberOfLinkValues? 0 : random.nextInt(randomUniqueValuesReplica.size()-numberOfLinkValues);
|
||||
subValues = randomUniqueValuesReplica.subList(start, start+numberOfLinkValues);
|
||||
Collections.shuffle(subValues, random);
|
||||
}
|
||||
for (String linkValue : subValues) {
|
||||
|
||||
@ -1404,6 +1407,9 @@ public class TestJoinUtil extends LuceneTestCase {
|
||||
}
|
||||
|
||||
if (random.nextBoolean()) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now force merge");
|
||||
}
|
||||
w.forceMerge(1);
|
||||
}
|
||||
w.close();
|
||||
|
@ -45,6 +45,7 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase {
|
||||
private long lastVersion;
|
||||
|
||||
private final Bits liveDocs;
|
||||
private String segment;
|
||||
|
||||
public IDVersionPostingsWriter(Bits liveDocs) {
|
||||
this.liveDocs = liveDocs;
|
||||
@ -58,6 +59,7 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase {
|
||||
@Override
|
||||
public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException {
|
||||
CodecUtil.writeIndexHeader(termsOut, TERMS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
|
||||
segment = state.segmentInfo.name;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -87,7 +89,7 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase {
|
||||
return;
|
||||
}
|
||||
if (lastDocID != -1) {
|
||||
throw new IllegalArgumentException("term appears in more than one document");
|
||||
throw new IllegalArgumentException("term appears in more than one document: " + lastDocID + " and " + docID);
|
||||
}
|
||||
if (termDocFreq != 1) {
|
||||
throw new IllegalArgumentException("term appears more than once in the document");
|
||||
|
@ -161,7 +161,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
|
||||
}
|
||||
|
||||
private final List<FieldMetaData> fields = new ArrayList<>();
|
||||
// private final String segment;
|
||||
private final String segment;
|
||||
|
||||
/** Create a new writer. The number of items (terms or
|
||||
* sub-blocks) per block will aim to be between
|
||||
@ -175,6 +175,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
|
||||
throws IOException
|
||||
{
|
||||
BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock);
|
||||
segment = state.segmentInfo.name;
|
||||
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
|
||||
@ -729,7 +730,6 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
|
||||
|
||||
/** Writes one term's worth of postings. */
|
||||
public void write(BytesRef text, TermsEnum termsEnum) throws IOException {
|
||||
|
||||
BlockTermState state = postingsWriter.writeTerm(text, termsEnum, docsSeen);
|
||||
// TODO: LUCENE-5693: we don't need this check if we fix IW to not send deleted docs to us on flush:
|
||||
if (state != null && ((IDVersionPostingsWriter) postingsWriter).lastDocID != -1) {
|
||||
|
@ -2106,6 +2106,9 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
||||
String[] uniqueValues = valueSet.toArray(new String[0]);
|
||||
|
||||
// index some docs
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now add numDocs=" + numDocs);
|
||||
}
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
Document doc = new Document();
|
||||
Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
|
||||
@ -2137,12 +2140,18 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
||||
|
||||
// delete some docs
|
||||
int numDeletions = random().nextInt(numDocs/10);
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now delete " + numDeletions + " docs");
|
||||
}
|
||||
for (int i = 0; i < numDeletions; i++) {
|
||||
int id = random().nextInt(numDocs);
|
||||
writer.deleteDocuments(new Term("id", Integer.toString(id)));
|
||||
}
|
||||
|
||||
// compare
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now get reader");
|
||||
}
|
||||
DirectoryReader ir = writer.getReader();
|
||||
TestUtil.checkReader(ir);
|
||||
for (LeafReaderContext context : ir.leaves()) {
|
||||
@ -2168,7 +2177,13 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
||||
}
|
||||
}
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now close reader");
|
||||
}
|
||||
ir.close();
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: force merge");
|
||||
}
|
||||
writer.forceMerge(1);
|
||||
|
||||
// compare again
|
||||
@ -2195,8 +2210,17 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
|
||||
}
|
||||
}
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: close reader");
|
||||
}
|
||||
ir.close();
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: close writer");
|
||||
}
|
||||
writer.close();
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: close dir");
|
||||
}
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
@ -564,7 +564,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
|
||||
handleFakeIOException(e, exceptionStream);
|
||||
allowAlreadyClosed = true;
|
||||
}
|
||||
|
||||
|
||||
if (random().nextInt(10) == 0) {
|
||||
// trigger flush:
|
||||
try {
|
||||
|
@ -1182,17 +1182,6 @@ public abstract class LuceneTestCase extends Assert {
|
||||
didChange = true;
|
||||
}
|
||||
|
||||
if (rarely(r)) {
|
||||
// change buffered deletes parameters
|
||||
boolean limitBufferedDeletes = r.nextBoolean();
|
||||
if (limitBufferedDeletes) {
|
||||
c.setMaxBufferedDeleteTerms(TestUtil.nextInt(r, 1, 1000));
|
||||
} else {
|
||||
c.setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
}
|
||||
didChange = true;
|
||||
}
|
||||
|
||||
if (rarely(r)) {
|
||||
IndexWriter.IndexReaderWarmer curWarmer = c.getMergedSegmentWarmer();
|
||||
if (curWarmer == null || curWarmer instanceof SimpleMergedSegmentWarmer) {
|
||||
|
Loading…
x
Reference in New Issue
Block a user