From 58105a203a19d18a56e09cf69dc0083c1b890315 Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Wed, 21 Jun 2017 13:47:15 -0400 Subject: [PATCH] LUCENE-7868: use multiple threads to concurrently resolve deletes and DV udpates --- lucene/CHANGES.txt | 7 + .../index/BinaryDocValuesFieldUpdates.java | 101 ++- .../apache/lucene/index/BufferedUpdates.java | 53 +- .../lucene/index/BufferedUpdatesStream.java | 740 +++++---------- .../apache/lucene/index/CoalescedUpdates.java | 109 --- .../lucene/index/DocValuesFieldUpdates.java | 192 ++-- .../apache/lucene/index/DocValuesUpdate.java | 3 - .../apache/lucene/index/DocumentsWriter.java | 66 +- .../index/DocumentsWriterDeleteQueue.java | 29 +- .../index/DocumentsWriterFlushControl.java | 21 +- .../index/DocumentsWriterFlushQueue.java | 33 +- .../index/DocumentsWriterPerThread.java | 35 +- .../index/DocumentsWriterPerThreadPool.java | 11 +- .../index/FlushByRamOrCountsPolicy.java | 29 +- .../org/apache/lucene/index/FlushPolicy.java | 3 - .../lucene/index/FreqProxTermsWriter.java | 5 +- .../lucene/index/FrozenBufferedUpdates.java | 855 ++++++++++++++++-- .../apache/lucene/index/IndexFileDeleter.java | 7 +- .../org/apache/lucene/index/IndexWriter.java | 778 ++++++++-------- .../lucene/index/IndexWriterConfig.java | 19 +- .../lucene/index/LiveIndexWriterConfig.java | 48 +- .../index/MergedPrefixCodedTermsIterator.java | 132 --- .../index/NumericDocValuesFieldUpdates.java | 101 ++- .../apache/lucene/index/PrefixCodedTerms.java | 2 +- .../lucene/index/ReadersAndUpdates.java | 415 ++++++--- .../lucene/index/SegmentCommitInfo.java | 16 +- .../lucene/index/SegmentCoreReaders.java | 2 - .../org/apache/lucene/index/SegmentInfo.java | 2 +- .../apache/lucene/index/SegmentReader.java | 19 +- .../lucene/index/SerialMergeScheduler.java | 4 +- .../lucene/index/TieredMergePolicy.java | 73 +- .../util/packed/AbstractPagedMutable.java | 2 +- .../index/TestBinaryDocValuesUpdates.java | 146 ++- .../index/TestDocumentsWriterDeleteQueue.java | 39 +- .../index/TestFlushByRamOrCountsPolicy.java | 12 +- .../lucene/index/TestForceMergeForever.java | 7 +- .../apache/lucene/index/TestIndexWriter.java | 61 -- .../lucene/index/TestIndexWriterConfig.java | 7 +- .../lucene/index/TestIndexWriterDelete.java | 153 ++-- .../index/TestIndexWriterExceptions.java | 253 ------ .../lucene/index/TestIndexWriterReader.java | 1 + .../index/TestNRTReaderWithThreads.java | 8 +- .../index/TestNumericDocValuesUpdates.java | 314 +++++-- .../lucene/index/TestPerSegmentDeletes.java | 7 +- .../lucene/index/TestPrefixCodedTerms.java | 76 -- .../TestControlledRealTimeReopenThread.java | 4 +- .../lucene/search/join/TestJoinUtil.java | 16 +- .../idversion/IDVersionPostingsWriter.java | 4 +- .../VersionBlockTreeTermsWriter.java | 4 +- .../index/BaseDocValuesFormatTestCase.java | 24 + .../index/BaseIndexFileFormatTestCase.java | 2 +- .../apache/lucene/util/LuceneTestCase.java | 11 - 52 files changed, 2649 insertions(+), 2412 deletions(-) delete mode 100644 lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java delete mode 100644 lucene/core/src/java/org/apache/lucene/index/MergedPrefixCodedTermsIterator.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index d695310e047..56da726c03b 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -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 diff --git a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java index f8cece9b5d3..e2700eabb94 100644 --- a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java @@ -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; } - } diff --git a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java index 1c3494fc5cd..a5a86e6774f 100644 --- a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java @@ -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 terms = new HashMap<>(); - final Map queries = new HashMap<>(); - final List docIDs = new ArrayList<>(); + final Map deleteTerms = new HashMap<>(); + final Map deleteQueries = new HashMap<>(); + final List deleteDocIDs = new ArrayList<>(); // Map> // 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; } } diff --git a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java index 9955626d594..8c0a2bea48c 100644 --- a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java +++ b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java @@ -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 updates = new ArrayList<>(); + private final Set 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 allDeleted; - ApplyDeletesResult(boolean anyDeletes, long gen, List allDeleted) { + ApplyDeletesResult(boolean anyDeletes, List allDeleted) { this.anyDeletes = anyDeletes; - this.gen = gen; this.allDeleted = allDeleted; } } - // Sorts SegmentInfos from smallest to biggest bufferedDelGen: - private static final Comparator sortSegInfoByDelGen = new Comparator() { - @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 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 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 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 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 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 sortByDelGen(List 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= 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= 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 { - 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 infos) throws IOException { - int numReaders = infos.size(); - SegmentState[] segStates = new SegmentState[numReaders]; + public SegmentState[] openSegmentStates(IndexWriter.ReaderPool pool, List infos, + Set alreadySeenSegments, long delGen) throws IOException { + ensureOpen(); + + List segStates = new ArrayList<>(); boolean success = false; try { - for(int i=0;i allDeleted = null; long totDelCount = 0; - for (int j=0;j> 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 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 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 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); + } + } + } } diff --git a/lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java b/lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java deleted file mode 100644 index bf92ac1cb9e..00000000000 --- a/lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java +++ /dev/null @@ -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 queries = new HashMap<>(); - final List terms = new ArrayList<>(); - final List> numericDVUpdates = new ArrayList<>(); - final List> 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 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 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 queriesIterable() { - return new Iterable() { - - @Override - public Iterator iterator() { - return new Iterator() { - private final Iterator> iter = queries.entrySet().iterator(); - - @Override - public boolean hasNext() { - return iter.hasNext(); - } - - @Override - public QueryAndLimit next() { - final Map.Entry ent = iter.next(); - return new QueryAndLimit(ent.getKey(), ent.getValue()); - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }; - } - }; - } -} diff --git a/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java b/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java index 528d4bfb604..a54bbe9b920 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java @@ -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 numericDVUpdates = new HashMap<>(); - final Map 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 queue = new PriorityQueue(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(); } diff --git a/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java b/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java index 1c85f335311..a66f9300850 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java @@ -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; } - } - } diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java index 2807517ef58..27e28c08734 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java @@ -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; diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java index db0e571d380..c4a084558cb 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java @@ -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(); } diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java index a5b4b7cf9d5..047fb9cc3a6 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java @@ -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; } - - } diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java index 2c62487a38b..df1b38c45fc 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java @@ -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; } } -} \ No newline at end of file +} diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java index c929ba239c2..94ffba77e2c 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java @@ -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(); diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java index cc723424853..de5b3fe88dc 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java @@ -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(); } } diff --git a/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java b/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java index a85c98b7c9c..ffd9501057c 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java @@ -29,8 +29,7 @@ import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState; *
  • * {@link #onDelete(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)} * - applies pending delete operations based on the global number of buffered - * delete terms iff {@link IndexWriterConfig#getMaxBufferedDeleteTerms()} is - * enabled
  • + * delete terms if the consumed memory is greater than {@link IndexWriterConfig#getRAMBufferSizeMB()}. *
  • * {@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 true if this {@link FlushPolicy} flushes on - * {@link IndexWriterConfig#getMaxBufferedDeleteTerms()}, otherwise - * false. - */ - protected boolean flushOnDeleteTerms() { - return indexWriterConfig.getMaxBufferedDeleteTerms() != IndexWriterConfig.DISABLE_AUTO_FLUSH; - } - /** * Returns true if this {@link FlushPolicy} flushes on * {@link IndexWriterConfig#getRAMBufferSizeMB()}, otherwise diff --git a/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java b/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java index e70959f3cce..cad07b4c712 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java @@ -33,9 +33,6 @@ import org.apache.lucene.util.InfoStream; *
  • Number of RAM resident documents - configured via * {@link IndexWriterConfig#setMaxBufferedDocs(int)}
  • * - * The policy also applies pending delete operations (by term and/or query), - * given the threshold set in - * {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)}. *

    * {@link IndexWriter} consults the provided {@link FlushPolicy} to control the * flushing process. The policy is informed for each added or updated document diff --git a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java index 1ca2830edcc..d953f8db55d 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java @@ -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 segDeletes = state.segUpdates.terms; + if (state.segUpdates != null && state.segUpdates.deleteTerms.size() > 0) { + Map segDeletes = state.segUpdates.deleteTerms; List deleteTerms = new ArrayList<>(segDeletes.keySet()); Collections.sort(deleteTerms); String lastField = null; diff --git a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java index 4f482ad4d10..d5d1de08afa 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java @@ -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 ent : deletes.queries.entrySet()) { - queries[upto] = ent.getKey(); - queryLimits[upto] = ent.getValue(); + for(Map.Entry 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 allNumericUpdates = new ArrayList<>(); - int numericUpdatesSize = 0; - for (LinkedHashMap 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 allBinaryUpdates = new ArrayList<>(); - int binaryUpdatesSize = 0; - for (LinkedHashMap 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> 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 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> 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 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 getInfosToApply(IndexWriter writer) { + assert Thread.holdsLock(writer); + List 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 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 delFiles = new HashSet<>(); + BufferedUpdatesStream.SegmentState[] segStates; + + synchronized (writer) { + List 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 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 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 queriesIterable() { - return new Iterable() { - @Override - public Iterator iterator() { - return new Iterator() { - 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; } } diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java index f7f196d7e77..e2b822b6a79 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java @@ -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 diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java index 14fbbae23ae..487028227af 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -29,10 +29,11 @@ import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Locale; -import java.util.Map.Entry; import java.util.Map; +import java.util.PriorityQueue; import java.util.Queue; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; @@ -71,6 +72,8 @@ import org.apache.lucene.util.ThreadInterruptedException; import org.apache.lucene.util.UnicodeUtil; import org.apache.lucene.util.Version; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + /** An IndexWriter creates and maintains an index. @@ -110,9 +113,7 @@ import org.apache.lucene.util.Version; The default is to flush when RAM usage hits {@link IndexWriterConfig#DEFAULT_RAM_BUFFER_SIZE_MB} MB. For best indexing speed you should flush by RAM usage with a - large RAM buffer. Additionally, if IndexWriter reaches the configured number of - buffered deletes (see {@link IndexWriterConfig#setMaxBufferedDeleteTerms}) - the deleted terms and queries are flushed and applied to existing segments. + large RAM buffer. In contrast to the other flush options {@link IndexWriterConfig#setRAMBufferSizeMB} and {@link IndexWriterConfig#setMaxBufferedDocs(int)}, deleted terms won't trigger a segment flush. Note that flushing just moves the @@ -237,7 +238,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { /** Used only for testing. */ boolean enableTestPoints = false; - private static final int UNBOUNDED_MAX_MERGE_SEGMENTS = -1; + static final int UNBOUNDED_MAX_MERGE_SEGMENTS = -1; /** * Name of the write lock in the index. @@ -272,7 +273,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { volatile Throwable tragedy; private final Directory directoryOrig; // original user directory - private final Directory directory; // wrapped with additional checks + final Directory directory; // wrapped with additional checks private final Analyzer analyzer; // how to analyze text private final AtomicLong changeCount = new AtomicLong(); // increments every time a change is completed @@ -289,7 +290,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { final SegmentInfos segmentInfos; // the segments final FieldNumbers globalFieldNumberMap; - private final DocumentsWriter docWriter; + final DocumentsWriter docWriter; private final Queue eventQueue; final IndexFileDeleter deleter; @@ -302,11 +303,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { private volatile boolean closed; private volatile boolean closing; + final AtomicBoolean maybeMerge = new AtomicBoolean(); + private Iterable> commitUserData; // Holds all SegmentInfo instances currently involved in // merges - private HashSet mergingSegments = new HashSet<>(); + HashSet mergingSegments = new HashSet<>(); private final MergeScheduler mergeScheduler; private LinkedList pendingMerges = new LinkedList<>(); @@ -317,11 +320,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { private boolean didMessageState; final AtomicInteger flushCount = new AtomicInteger(); + final AtomicInteger flushDeletesCount = new AtomicInteger(); final ReaderPool readerPool = new ReaderPool(); final BufferedUpdatesStream bufferedUpdatesStream; + /** Counts how many merges have completed; this is used by {@link FrozenBufferedUpdates#apply} + * to handle concurrently apply deletes/updates with merges completing. */ + final AtomicLong mergeFinishedGen = new AtomicLong(); + // This is a "write once" variable (like the organic dye // on a DVD-R that may or may not be heated by a laser and // then cooled to permanently record the event): it's @@ -449,21 +457,36 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } else { anyChanges = false; } - if (!anyChanges) { + if (anyChanges == false) { // prevent double increment since docWriter#doFlush increments the flushcount // if we flushed anything. flushCount.incrementAndGet(); } - // Prevent segmentInfos from changing while opening the - // reader; in theory we could instead do similar retry logic, - // just like we do when loading segments_N + + processEvents(false, true); + + if (applyAllDeletes) { + applyAllDeletesAndUpdates(); + } + synchronized(this) { - anyChanges |= maybeApplyDeletes(applyAllDeletes); + + // NOTE: we cannot carry doc values updates in memory yet, so we always must write them through to disk and re-open each + // SegmentReader: + + // TODO: we could instead just clone SIS and pull/incref readers in sync'd block, and then do this w/o IW's lock? + // Must do this sync'd on IW to prevent a merge from completing at the last second and failing to write its DV updates: + readerPool.writeAllDocValuesUpdates(); + if (writeAllDeletes) { // Must move the deletes to disk: readerPool.commit(segmentInfos); } + // Prevent segmentInfos from changing while opening the + // reader; in theory we could instead do similar retry logic, + // just like we do when loading segments_N + r = StandardDirectoryReader.open(this, segmentInfos, applyAllDeletes, writeAllDeletes); if (infoStream.isEnabled("IW")) { infoStream.message("IW", "return reader version=" + r.getVersion() + " reader=" + r); @@ -483,6 +506,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } } } + anyChanges |= maybeMerge.getAndSet(false); if (anyChanges) { maybeMerge(config.getMergePolicy(), MergeTrigger.FULL_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS); } @@ -509,7 +533,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } /** Holds shared SegmentReader instances. IndexWriter uses - * SegmentReaders for 1) applying deletes, 2) doing + * SegmentReaders for 1) applying deletes/DV updates, 2) doing * merges, 3) handing out a real-time reader. This pool * reuses instances of the SegmentReaders in all these * places if it is in "near real-time mode" (getReader() @@ -519,8 +543,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { private final Map readerMap = new HashMap<>(); - // used only by asserts - public synchronized boolean infoIsLive(SegmentCommitInfo info) { + /** Asserts this info still exists in IW's segment infos */ + public synchronized boolean assertInfoIsLive(SegmentCommitInfo info) { int idx = segmentInfos.indexOf(info); assert idx != -1: "info=" + info + " isn't live"; assert segmentInfos.info(idx) == info: "info=" + info + " doesn't match live info in segmentInfos"; @@ -531,12 +555,19 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { final ReadersAndUpdates rld = readerMap.get(info); if (rld != null) { assert info == rld.info; -// System.out.println("[" + Thread.currentThread().getName() + "] ReaderPool.drop: " + info); readerMap.remove(info); rld.dropReaders(); } } + public synchronized long ramBytesUsed() { + long bytes = 0; + for (ReadersAndUpdates rld : readerMap.values()) { + bytes += rld.ramBytesUsed.get(); + } + return bytes; + } + public synchronized boolean anyPendingDeletes() { for(ReadersAndUpdates rld : readerMap.values()) { if (rld.getPendingDeleteCount() != 0) { @@ -556,30 +587,39 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // Matches incRef in get: rld.decRef(); - // Pool still holds a ref: - assert rld.refCount() >= 1; + if (rld.refCount() == 0) { + // This happens if the segment was just merged away, while a buffered deletes packet was still applying deletes/updates to it. + assert readerMap.containsKey(rld.info) == false: "seg=" + rld.info + " has refCount 0 but still unexpectedly exists in the reader pool"; + } else { - if (!poolReaders && rld.refCount() == 1) { - // This is the last ref to this RLD, and we're not - // pooling, so remove it: -// System.out.println("[" + Thread.currentThread().getName() + "] ReaderPool.release: " + rld.info); - if (rld.writeLiveDocs(directory)) { - // Make sure we only write del docs for a live segment: - assert assertInfoLive == false || infoIsLive(rld.info); - // Must checkpoint because we just - // created new _X_N.del and field updates files; - // don't call IW.checkpoint because that also - // increments SIS.version, which we do not want to - // do here: it was done previously (after we - // invoked BDS.applyDeletes), whereas here all we - // did was move the state to disk: - checkpointNoSIS(); + // Pool still holds a ref: + assert rld.refCount() > 0: "refCount=" + rld.refCount() + " reader=" + rld.info; + + if (!poolReaders && rld.refCount() == 1 && readerMap.containsKey(rld.info)) { + // This is the last ref to this RLD, and we're not + // pooling, so remove it: + if (rld.writeLiveDocs(directory)) { + // Make sure we only write del docs for a live segment: + assert assertInfoLive == false || assertInfoIsLive(rld.info); + // Must checkpoint because we just + // created new _X_N.del and field updates files; + // don't call IW.checkpoint because that also + // increments SIS.version, which we do not want to + // do here: it was done previously (after we + // invoked BDS.applyDeletes), whereas here all we + // did was move the state to disk: + checkpointNoSIS(); + } + + rld.writeFieldUpdates(directory, bufferedUpdatesStream.getCompletedDelGen(), infoStream); + + if (rld.getNumDVUpdates() == 0) { + rld.dropReaders(); + readerMap.remove(rld.info); + } else { + // We are forced to pool this segment until its deletes fully apply (no delGen gaps) + } } - //System.out.println("IW: done writeLiveDocs for info=" + rld.info); - -// System.out.println("[" + Thread.currentThread().getName() + "] ReaderPool.release: drop readers " + rld.info); - rld.dropReaders(); - readerMap.remove(rld.info); } } @@ -588,6 +628,96 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { dropAll(false); } + void writeAllDocValuesUpdates() throws IOException { + Collection copy; + synchronized (this) { + copy = new HashSet<>(readerMap.values()); + } + boolean any = false; + for (ReadersAndUpdates rld : copy) { + any |= rld.writeFieldUpdates(directory, bufferedUpdatesStream.getCompletedDelGen(), infoStream); + } + if (any) { + checkpoint(); + } + } + + void writeDocValuesUpdates(List infos) throws IOException { + boolean any = false; + for (SegmentCommitInfo info : infos) { + ReadersAndUpdates rld = get(info, false); + if (rld != null) { + any |= rld.writeFieldUpdates(directory, bufferedUpdatesStream.getCompletedDelGen(), infoStream); + } + } + if (any) { + checkpoint(); + } + } + + private final AtomicBoolean writeDocValuesLock = new AtomicBoolean(); + + void writeSomeDocValuesUpdates() throws IOException { + + assert Thread.holdsLock(IndexWriter.this) == false; + + if (writeDocValuesLock.compareAndSet(false, true)) { + try { + + LiveIndexWriterConfig config = getConfig(); + double mb = config.getRAMBufferSizeMB(); + // If the reader pool is > 50% of our IW buffer, then write the updates: + if (mb != IndexWriterConfig.DISABLE_AUTO_FLUSH) { + long startNS = System.nanoTime(); + + long ramBytesUsed = ramBytesUsed(); + if (ramBytesUsed > 0.5 * mb * 1024 * 1024) { + if (infoStream.isEnabled("BD")) { + infoStream.message("BD", String.format(Locale.ROOT, "now write some pending DV updates: %.2f MB used vs IWC Buffer %.2f MB", + ramBytesUsed/1024./1024., mb)); + } + + // Sort by largest ramBytesUsed: + PriorityQueue queue = new PriorityQueue<>(readerMap.size(), (a, b) -> Long.compare(b.ramBytesUsed.get(), a.ramBytesUsed.get())); + synchronized (this) { + for (ReadersAndUpdates rld : readerMap.values()) { + queue.add(rld); + } + } + + int count = 0; + while (ramBytesUsed > 0.5 * mb * 1024 * 1024) { + ReadersAndUpdates rld = queue.poll(); + if (rld == null) { + break; + } + + // We need to do before/after because not all RAM in this RAU is used by DV updates, and + // not all of those bytes can be written here: + long bytesUsedBefore = rld.ramBytesUsed.get(); + + // Only acquire IW lock on each write, since this is a time consuming operation. This way + // other threads get a chance to run in between our writes. + synchronized (IndexWriter.this) { + rld.writeFieldUpdates(directory, bufferedUpdatesStream.getCompletedDelGen(), infoStream); + } + long bytesUsedAfter = rld.ramBytesUsed.get(); + ramBytesUsed -= bytesUsedBefore - bytesUsedAfter; + count++; + } + + if (infoStream.isEnabled("BD")) { + infoStream.message("BD", String.format(Locale.ROOT, "done write some DV updates for %d segments: now %.2f MB used vs IWC Buffer %.2f MB; took %.2f sec", + count, ramBytesUsed()/1024./1024., mb, ((System.nanoTime() - startNS)/1000000000.))); + } + } + } + } finally { + writeDocValuesLock.set(false); + } + } + } + /** Remove all our references to readers, and commits * any pending changes. */ synchronized void dropAll(boolean doSave) throws IOException { @@ -599,7 +729,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { try { if (doSave && rld.writeLiveDocs(directory)) { // Make sure we only write del docs and field updates for a live segment: - assert infoIsLive(rld.info); + assert assertInfoIsLive(rld.info); // Must checkpoint because we just // created new _X_N.del and field updates files; // don't call IW.checkpoint because that also @@ -654,9 +784,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { final ReadersAndUpdates rld = readerMap.get(info); if (rld != null) { assert rld.info == info; - if (rld.writeLiveDocs(directory)) { + boolean changed = rld.writeLiveDocs(directory); + + changed |= rld.writeFieldUpdates(directory, bufferedUpdatesStream.getCompletedDelGen(), infoStream); + + if (changed) { // Make sure we only write del docs for a live segment: - assert infoIsLive(info); + assert assertInfoIsLive(info); // Must checkpoint because we just // created new _X_N.del and field updates files; @@ -667,10 +801,22 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // did was move the state to disk: checkpointNoSIS(); } + } } } + public synchronized boolean anyChanges() { + for (ReadersAndUpdates rld : readerMap.values()) { + // NOTE: we don't check for pending deletes because deletes carry over in RAM to NRT readers + if (rld.getNumDVUpdates() != 0) { + return true; + } + } + + return false; + } + /** * Obtain a ReadersAndLiveDocs instance from the * readerPool. If create is true, you must later call @@ -685,14 +831,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { ReadersAndUpdates rld = readerMap.get(info); if (rld == null) { - if (!create) { + if (create == false) { return null; } rld = new ReadersAndUpdates(IndexWriter.this, info); // Steal initial reference: readerMap.put(info, rld); } else { - assert rld.info == info: "rld.info=" + rld.info + " info=" + info + " isLive?=" + infoIsLive(rld.info) + " vs " + infoIsLive(info); + assert rld.info == info: "rld.info=" + rld.info + " info=" + info + " isLive?=" + assertInfoIsLive(rld.info) + " vs " + assertInfoIsLive(info); } if (create) { @@ -809,7 +955,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { mergeScheduler.setInfoStream(infoStream); codec = config.getCodec(); - bufferedUpdatesStream = new BufferedUpdatesStream(infoStream); + bufferedUpdatesStream = new BufferedUpdatesStream(this); poolReaders = config.getReaderPooling(); OpenMode mode = config.getOpenMode(); @@ -1446,38 +1592,38 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { ReadersAndUpdates rld = readerPool.get(info, false); if (rld != null) { synchronized(bufferedUpdatesStream) { - rld.initWritableLiveDocs(); if (rld.delete(docID)) { final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount(); if (fullDelCount == rld.info.info.maxDoc()) { - // If a merge has already registered for this - // segment, we leave it in the readerPool; the - // merge will skip merging it and will then drop - // it once it's done: - if (!mergingSegments.contains(rld.info)) { - segmentInfos.remove(rld.info); - readerPool.drop(rld.info); - checkpoint(); - } + dropDeletedSegment(rld.info); + checkpoint(); } // Must bump changeCount so if no other changes // happened, we still commit this change: changed(); } - //System.out.println(" yes " + info.info.name + " " + docID); return docWriter.deleteQueue.getNextSequenceNumber(); } - } else { - //System.out.println(" no rld " + info.info.name + " " + docID); } - } else { - //System.out.println(" no seg " + info.info.name + " " + docID); } return -1; } + /** Drops a segment that has 100% deleted documents. */ + synchronized void dropDeletedSegment(SegmentCommitInfo info) throws IOException { + // If a merge has already registered for this + // segment, we leave it in the readerPool; the + // merge will skip merging it and will then drop + // it once it's done: + if (mergingSegments.contains(info) == false) { + segmentInfos.remove(info); + pendingNumDocs.addAndGet(-info.info.maxDoc()); + readerPool.drop(info); + } + } + /** * Deletes the document(s) containing any of the * terms. All given deletes are applied and flushed atomically @@ -1881,20 +2027,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { public void forceMerge(int maxNumSegments, boolean doWait) throws IOException { ensureOpen(); - if (maxNumSegments < 1) + if (maxNumSegments < 1) { throw new IllegalArgumentException("maxNumSegments must be >= 1; got " + maxNumSegments); + } if (infoStream.isEnabled("IW")) { infoStream.message("IW", "forceMerge: index now " + segString()); infoStream.message("IW", "now flush at forceMerge"); } - flush(true, true); - synchronized(this) { resetMergeExceptions(); segmentsToMerge.clear(); for(SegmentCommitInfo info : segmentInfos) { + assert info != null; segmentsToMerge.put(info, Boolean.TRUE); } mergeMaxNumSegments = maxNumSegments; @@ -1903,12 +2049,18 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // merge: for(final MergePolicy.OneMerge merge : pendingMerges) { merge.maxNumSegments = maxNumSegments; - segmentsToMerge.put(merge.info, Boolean.TRUE); + if (merge.info != null) { + // TODO: explain why this is sometimes still null + segmentsToMerge.put(merge.info, Boolean.TRUE); + } } for (final MergePolicy.OneMerge merge: runningMerges) { merge.maxNumSegments = maxNumSegments; - segmentsToMerge.put(merge.info, Boolean.TRUE); + if (merge.info != null) { + // TODO: explain why this is sometimes still null + segmentsToMerge.put(merge.info, Boolean.TRUE); + } } } @@ -2076,7 +2228,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { maybeMerge(config.getMergePolicy(), MergeTrigger.EXPLICIT, UNBOUNDED_MAX_MERGE_SEGMENTS); } - private final void maybeMerge(MergePolicy mergePolicy, MergeTrigger trigger, int maxNumSegments) throws IOException { + final void maybeMerge(MergePolicy mergePolicy, MergeTrigger trigger, int maxNumSegments) throws IOException { ensureOpen(false); boolean newMergesFound = updatePendingMerges(mergePolicy, trigger, maxNumSegments); mergeScheduler.merge(this, trigger, newMergesFound); @@ -2103,7 +2255,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { final MergePolicy.MergeSpecification spec; if (maxNumSegments != UNBOUNDED_MAX_MERGE_SEGMENTS) { assert trigger == MergeTrigger.EXPLICIT || trigger == MergeTrigger.MERGE_FINISHED : - "Expected EXPLICT or MERGE_FINISHED as trigger even with maxNumSegments set but was: " + trigger.name(); + "Expected EXPLICT or MERGE_FINISHED as trigger even with maxNumSegments set but was: " + trigger.name(); + spec = mergePolicy.findForcedMerges(segmentInfos, maxNumSegments, Collections.unmodifiableMap(segmentsToMerge), this); newMergesFound = spec != null; if (newMergesFound) { @@ -2212,6 +2365,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { mergeScheduler.close(); bufferedUpdatesStream.clear(); + docWriter.close(); // mark it as closed first to prevent subsequent indexing actions/flushes docWriter.abort(this); // don't sync on IW here synchronized(this) { @@ -2496,49 +2650,63 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { segmentInfos.changed(); } - synchronized void publishFrozenUpdates(FrozenBufferedUpdates packet) { + synchronized void publishFrozenUpdates(FrozenBufferedUpdates packet) throws IOException { assert packet != null && packet.any(); - synchronized (bufferedUpdatesStream) { - bufferedUpdatesStream.push(packet); - } + bufferedUpdatesStream.push(packet); + docWriter.putEvent(new DocumentsWriter.ResolveUpdatesEvent(packet)); } - + /** * Atomically adds the segment private delete packet and publishes the flushed * segments SegmentInfo to the index writer. */ - void publishFlushedSegment(SegmentCommitInfo newSegment, - FrozenBufferedUpdates packet, FrozenBufferedUpdates globalPacket) throws IOException { + synchronized void publishFlushedSegment(SegmentCommitInfo newSegment, + FrozenBufferedUpdates packet, FrozenBufferedUpdates globalPacket, + Sorter.DocMap sortMap) throws IOException { try { - synchronized (this) { - // Lock order IW -> BDS - ensureOpen(false); - synchronized (bufferedUpdatesStream) { - if (infoStream.isEnabled("IW")) { - infoStream.message("IW", "publishFlushedSegment"); - } - - if (globalPacket != null && globalPacket.any()) { - bufferedUpdatesStream.push(globalPacket); - } - // Publishing the segment must be synched on IW -> BDS to make the sure - // that no merge prunes away the seg. private delete packet - final long nextGen; - if (packet != null && packet.any()) { - nextGen = bufferedUpdatesStream.push(packet); - } else { - // Since we don't have a delete packet to apply we can get a new - // generation right away - nextGen = bufferedUpdatesStream.getNextGen(); - } - if (infoStream.isEnabled("IW")) { - infoStream.message("IW", "publish sets newSegment delGen=" + nextGen + " seg=" + segString(newSegment)); - } - newSegment.setBufferedDeletesGen(nextGen); - segmentInfos.add(newSegment); - checkpoint(); - } + // Lock order IW -> BDS + ensureOpen(false); + + if (infoStream.isEnabled("IW")) { + infoStream.message("IW", "publishFlushedSegment " + newSegment); } + + if (globalPacket != null && globalPacket.any()) { + // Do this as an event so it applies higher in the stack when we are not holding DocumentsWriterFlushQueue.purgeLock: + bufferedUpdatesStream.push(globalPacket); + docWriter.putEvent(new DocumentsWriter.ResolveUpdatesEvent(globalPacket)); + } + + // Publishing the segment must be sync'd on IW -> BDS to make the sure + // that no merge prunes away the seg. private delete packet + final long nextGen; + if (packet != null && packet.any()) { + nextGen = bufferedUpdatesStream.push(packet); + + // Do this as an event so it applies higher in the stack when we are not holding DocumentsWriterFlushQueue.purgeLock: + docWriter.putEvent(new DocumentsWriter.ResolveUpdatesEvent(packet)); + + } else { + // Since we don't have a delete packet to apply we can get a new + // generation right away + nextGen = bufferedUpdatesStream.getNextGen(); + // No deletes/updates here, so marked finished immediately: + bufferedUpdatesStream.finishedSegment(nextGen); + } + if (infoStream.isEnabled("IW")) { + infoStream.message("IW", "publish sets newSegment delGen=" + nextGen + " seg=" + segString(newSegment)); + } + newSegment.setBufferedDeletesGen(nextGen); + segmentInfos.add(newSegment); + checkpoint(); + + if (packet != null && packet.any() && sortMap != null) { + // TODO: not great we do this heavyish op while holding IW's monitor lock, + // but it only applies if you are using sorted indices and updating doc values: + ReadersAndUpdates rld = readerPool.get(newSegment, true); + rld.sortMap = sortMap; + } + } finally { flushCount.incrementAndGet(); doAfterFlush(); @@ -2924,7 +3092,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { /** Copies the segment files as-is into the IndexWriter's directory. */ private SegmentCommitInfo copySegmentAsIs(SegmentCommitInfo info, String segName, IOContext context) throws IOException { - //System.out.println("copy seg=" + info.info.name + " version=" + info.info.getVersion()); // Same SI as before but we change directory and name SegmentInfo newInfo = new SegmentInfo(directoryOrig, info.info.getVersion(), info.info.getMinVersion(), segName, info.info.maxDoc(), info.info.getUseCompoundFile(), info.info.getCodec(), @@ -2991,16 +3158,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { @Override public final long prepareCommit() throws IOException { ensureOpen(); - boolean[] doMaybeMerge = new boolean[1]; - pendingSeqNo = prepareCommitInternal(doMaybeMerge); + pendingSeqNo = prepareCommitInternal(); // we must do this outside of the commitLock else we can deadlock: - if (doMaybeMerge[0]) { + if (maybeMerge.getAndSet(false)) { maybeMerge(config.getMergePolicy(), MergeTrigger.FULL_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS); } return pendingSeqNo; } - private long prepareCommitInternal(boolean[] doMaybeMerge) throws IOException { + private long prepareCommitInternal() throws IOException { startCommitTime = System.nanoTime(); synchronized(commitLock) { ensureOpen(false); @@ -3020,7 +3186,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { doBeforeFlush(); testPoint("startDoFlush"); SegmentInfos toCommit = null; - boolean anySegmentsFlushed = false; + boolean anyChanges = false; long seqNo; // This is copied from doFlush, except it's modified to @@ -3035,19 +3201,23 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { try { seqNo = docWriter.flushAllThreads(); if (seqNo < 0) { - anySegmentsFlushed = true; + anyChanges = true; seqNo = -seqNo; } - if (!anySegmentsFlushed) { + if (anyChanges == false) { // prevent double increment since docWriter#doFlush increments the flushcount // if we flushed anything. flushCount.incrementAndGet(); } - processEvents(false, true); + + // cannot pass triggerMerges=true here else it can lead to deadlock: + processEvents(false, false); + flushSuccess = true; + applyAllDeletesAndUpdates(); + synchronized(this) { - maybeApplyDeletes(true); readerPool.commit(segmentInfos); @@ -3106,8 +3276,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { boolean success = false; try { - if (anySegmentsFlushed) { - doMaybeMerge[0] = true; + if (anyChanges) { + maybeMerge.set(true); } startCommit(toCommit); success = true; @@ -3228,8 +3398,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { infoStream.message("IW", "commit: start"); } - boolean[] doMaybeMerge = new boolean[1]; - long seqNo; synchronized(commitLock) { @@ -3243,7 +3411,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { if (infoStream.isEnabled("IW")) { infoStream.message("IW", "commit: now prepare"); } - seqNo = prepareCommitInternal(doMaybeMerge); + seqNo = prepareCommitInternal(); } else { if (infoStream.isEnabled("IW")) { infoStream.message("IW", "commit: already prepared"); @@ -3255,7 +3423,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } // we must do this outside of the commitLock else we can deadlock: - if (doMaybeMerge[0]) { + if (maybeMerge.getAndSet(false)) { maybeMerge(mergePolicy, MergeTrigger.FULL_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS); } @@ -3417,8 +3585,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { processEvents(false, true); } } + + if (applyAllDeletes) { + applyAllDeletesAndUpdates(); + } + + anyChanges |= maybeMerge.getAndSet(false); + synchronized(this) { - anyChanges |= maybeApplyDeletes(applyAllDeletes); doAfterFlush(); success = true; return anyChanges; @@ -3436,48 +3610,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } } - final synchronized boolean maybeApplyDeletes(boolean applyAllDeletes) throws IOException { - if (applyAllDeletes) { - if (infoStream.isEnabled("IW")) { - infoStream.message("IW", "apply all deletes during flush"); - } - return applyAllDeletesAndUpdates(); - } else if (infoStream.isEnabled("IW")) { - infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedUpdatesStream.numTerms() + " bytesUsed=" + bufferedUpdatesStream.ramBytesUsed()); - } - - return false; - } - - final synchronized boolean applyAllDeletesAndUpdates() throws IOException { + final void applyAllDeletesAndUpdates() throws IOException { + assert Thread.holdsLock(this) == false; flushDeletesCount.incrementAndGet(); - final BufferedUpdatesStream.ApplyDeletesResult result; if (infoStream.isEnabled("IW")) { - infoStream.message("IW", "now apply all deletes for all segments maxDoc=" + (docWriter.getNumDocs() + segmentInfos.totalMaxDoc())); + infoStream.message("IW", "now apply all deletes for all segments buffered updates bytesUsed=" + bufferedUpdatesStream.ramBytesUsed() + " reader pool bytesUsed=" + readerPool.ramBytesUsed()); } - result = bufferedUpdatesStream.applyDeletesAndUpdates(readerPool, segmentInfos.asList()); - if (result.anyDeletes) { - checkpoint(); - } - if (!keepFullyDeletedSegments && result.allDeleted != null) { - if (infoStream.isEnabled("IW")) { - infoStream.message("IW", "drop 100% deleted segments: " + segString(result.allDeleted)); - } - for (SegmentCommitInfo info : result.allDeleted) { - // If a merge has already registered for this - // segment, we leave it in the readerPool; the - // merge will skip merging it and will then drop - // it once it's done: - if (!mergingSegments.contains(info)) { - segmentInfos.remove(info); - pendingNumDocs.addAndGet(-info.info.maxDoc()); - readerPool.drop(info); - } - } - checkpoint(); - } - bufferedUpdatesStream.prune(segmentInfos); - return result.anyDeletes; + bufferedUpdatesStream.waitApplyAll(); } // for testing only @@ -3514,41 +3653,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { private static class MergedDeletesAndUpdates { ReadersAndUpdates mergedDeletesAndUpdates = null; - boolean initializedWritableLiveDocs = false; MergedDeletesAndUpdates() {} - final void init(ReaderPool readerPool, MergePolicy.OneMerge merge, boolean initWritableLiveDocs) throws IOException { + final void init(ReaderPool readerPool, MergePolicy.OneMerge merge) throws IOException { if (mergedDeletesAndUpdates == null) { mergedDeletesAndUpdates = readerPool.get(merge.info, true); } - if (initWritableLiveDocs && !initializedWritableLiveDocs) { - mergedDeletesAndUpdates.initWritableLiveDocs(); - this.initializedWritableLiveDocs = true; - } - } - - } - - private void maybeApplyMergedDVUpdates(MergePolicy.OneMerge merge, MergeState mergeState, - MergedDeletesAndUpdates holder, String[] mergingFields, DocValuesFieldUpdates[] dvFieldUpdates, - DocValuesFieldUpdates.Iterator[] updatesIters, int segment, int curDoc) throws IOException { - int newDoc = -1; - for (int idx = 0; idx < mergingFields.length; idx++) { - DocValuesFieldUpdates.Iterator updatesIter = updatesIters[idx]; - if (updatesIter.doc() == curDoc) { // document has an update - if (holder.mergedDeletesAndUpdates == null) { - holder.init(readerPool, merge, false); - } - if (newDoc == -1) { // map once per all field updates, but only if there are any updates - newDoc = mergeState.docMaps[segment].get(curDoc); - } - DocValuesFieldUpdates dvUpdates = dvFieldUpdates[idx]; - dvUpdates.add(newDoc, updatesIter.value()); - updatesIter.nextDoc(); // advance to next document - } else { - assert updatesIter.doc() > curDoc : "field=" + mergingFields[idx] + " updateDoc=" + updatesIter.doc() + " curDoc=" + curDoc; - } } } @@ -3564,6 +3675,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { */ synchronized private ReadersAndUpdates commitMergedDeletesAndUpdates(MergePolicy.OneMerge merge, MergeState mergeState) throws IOException { + mergeFinishedGen.incrementAndGet(); + testPoint("startCommitMergeDeletes"); final List sourceSegments = merge.segments; @@ -3576,9 +3689,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // started merging: long minGen = Long.MAX_VALUE; - // Lazy init (only when we find a delete to carry over): - final MergedDeletesAndUpdates holder = new MergedDeletesAndUpdates(); - final DocValuesFieldUpdates.Container mergedDVUpdates = new DocValuesFieldUpdates.Container(); + // Lazy init (only when we find a delete or update to carry over): + final ReadersAndUpdates mergedDeletesAndUpdates = readerPool.get(merge.info, true); + + // field -> delGen -> dv field updates + Map> mappedDVUpdates = new HashMap<>(); + + boolean anyDVUpdates = false; assert sourceSegments.size() == mergeState.docMaps.length; for (int i = 0; i < sourceSegments.size(); i++) { @@ -3587,36 +3704,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { final int maxDoc = info.info.maxDoc(); final Bits prevLiveDocs = merge.readers.get(i).getLiveDocs(); final ReadersAndUpdates rld = readerPool.get(info, false); - // We hold a ref so it should still be in the pool: + // We hold a ref, from when we opened the readers during mergeInit, so it better still be in the pool: assert rld != null: "seg=" + info.info.name; final Bits currentLiveDocs = rld.getLiveDocs(); - final Map mergingFieldUpdates = rld.getMergingFieldUpdates(); - final String[] mergingFields; - final DocValuesFieldUpdates[] dvFieldUpdates; - final DocValuesFieldUpdates.Iterator[] updatesIters; - if (mergingFieldUpdates.isEmpty()) { - mergingFields = null; - updatesIters = null; - dvFieldUpdates = null; - } else { - mergingFields = new String[mergingFieldUpdates.size()]; - dvFieldUpdates = new DocValuesFieldUpdates[mergingFieldUpdates.size()]; - updatesIters = new DocValuesFieldUpdates.Iterator[mergingFieldUpdates.size()]; - int idx = 0; - for (Entry e : mergingFieldUpdates.entrySet()) { - String field = e.getKey(); - DocValuesFieldUpdates updates = e.getValue(); - mergingFields[idx] = field; - dvFieldUpdates[idx] = mergedDVUpdates.getUpdates(field, updates.type); - if (dvFieldUpdates[idx] == null) { - dvFieldUpdates[idx] = mergedDVUpdates.newUpdates(field, updates.type, mergeState.segmentInfo.maxDoc()); - } - updatesIters[idx] = updates.iterator(); - updatesIters[idx].nextDoc(); // advance to first update doc - ++idx; - } - } -// System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMergedDeletes: info=" + info + ", mergingUpdates=" + mergingUpdates); + + MergeState.DocMap segDocMap = mergeState.docMaps[i]; + MergeState.DocMap segLeafDocMap = mergeState.leafDocMaps[i]; if (prevLiveDocs != null) { @@ -3648,26 +3741,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { assert currentLiveDocs.get(j) == false; } else if (currentLiveDocs.get(j) == false) { // the document was deleted while we were merging: - if (holder.mergedDeletesAndUpdates == null || holder.initializedWritableLiveDocs == false) { - holder.init(readerPool, merge, true); - } - holder.mergedDeletesAndUpdates.delete(mergeState.docMaps[i].get(mergeState.leafDocMaps[i].get(j))); - if (mergingFields != null) { // advance all iters beyond the deleted document - skipDeletedDoc(updatesIters, j); - } - } else if (mergingFields != null) { - maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j); - } - } - } else if (mergingFields != null) { - // need to check each non-deleted document if it has any updates - for (int j = 0; j < maxDoc; j++) { - if (prevLiveDocs.get(j)) { - // document isn't deleted, check if any of the fields have an update to it - maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j); - } else { - // advance all iters beyond the deleted document - skipDeletedDoc(updatesIters, j); + mergedDeletesAndUpdates.delete(segDocMap.get(segLeafDocMap.get(j))); } } } @@ -3677,52 +3751,83 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // does: for (int j = 0; j < maxDoc; j++) { if (currentLiveDocs.get(j) == false) { - if (holder.mergedDeletesAndUpdates == null || !holder.initializedWritableLiveDocs) { - holder.init(readerPool, merge, true); - } - holder.mergedDeletesAndUpdates.delete(mergeState.docMaps[i].get(mergeState.leafDocMaps[i].get(j))); - if (mergingFields != null) { // advance all iters beyond the deleted document - skipDeletedDoc(updatesIters, j); - } - } else if (mergingFields != null) { - maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j); + mergedDeletesAndUpdates.delete(segDocMap.get(segLeafDocMap.get(j))); } } - } else if (mergingFields != null) { - // no deletions before or after, but there were updates - for (int j = 0; j < maxDoc; j++) { - maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j); + } + + // Now carry over all doc values updates that were resolved while we were merging, remapping the docIDs to the newly merged docIDs. + // We only carry over packets that finished resolving; if any are still running (concurrently) they will detect that our merge completed + // and re-resolve against the newly merged segment: + + Map> mergingDVUpdates = rld.getMergingDVUpdates(); + + for (Map.Entry> ent : mergingDVUpdates.entrySet()) { + + String field = ent.getKey(); + + Map mappedField = mappedDVUpdates.get(field); + if (mappedField == null) { + mappedField = new HashMap<>(); + mappedDVUpdates.put(field, mappedField); + } + + for (DocValuesFieldUpdates updates : ent.getValue()) { + + if (bufferedUpdatesStream.stillRunning(updates.delGen)) { + continue; + } + + // sanity check: + assert field.equals(updates.field); + + DocValuesFieldUpdates mappedUpdates = mappedField.get(updates.delGen); + if (mappedUpdates == null) { + switch (updates.type) { + case NUMERIC: + mappedUpdates = new NumericDocValuesFieldUpdates(updates.delGen, updates.field, merge.info.info.maxDoc()); + break; + case BINARY: + mappedUpdates = new BinaryDocValuesFieldUpdates(updates.delGen, updates.field, merge.info.info.maxDoc()); + break; + default: + throw new AssertionError(); + } + mappedField.put(updates.delGen, mappedUpdates); + } + + DocValuesFieldUpdates.Iterator it = updates.iterator(); + int doc; + while ((doc = it.nextDoc()) != NO_MORE_DOCS) { + int mappedDoc = segDocMap.get(segLeafDocMap.get(doc)); + if (mappedDoc != -1) { + // not deleted + mappedUpdates.add(mappedDoc, it.value()); + anyDVUpdates = true; + } + } } } } - if (mergedDVUpdates.any()) { -// System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMergedDeletes: mergedDeletes.info=" + mergedDeletes.info + ", mergedFieldUpdates=" + mergedFieldUpdates); - boolean success = false; - try { - // if any error occurs while writing the field updates we should release - // the info, otherwise it stays in the pool but is considered not "live" - // which later causes false exceptions in pool.dropAll(). - // NOTE: currently this is the only place which throws a true - // IOException. If this ever changes, we need to extend that try/finally - // block to the rest of the method too. - holder.mergedDeletesAndUpdates.writeFieldUpdates(directory, mergedDVUpdates); - success = true; - } finally { - if (!success) { - holder.mergedDeletesAndUpdates.dropChanges(); - readerPool.drop(merge.info); + if (anyDVUpdates) { + // Persist the merged DV updates onto the RAU for the merged segment: + for(Map d : mappedDVUpdates.values()) { + for (DocValuesFieldUpdates updates : d.values()) { + updates.finish(); + mergedDeletesAndUpdates.addDVUpdate(updates); } } } - + if (infoStream.isEnabled("IW")) { - if (holder.mergedDeletesAndUpdates == null) { + if (mergedDeletesAndUpdates == null) { infoStream.message("IW", "no new deletes or field updates since merge started"); } else { - String msg = holder.mergedDeletesAndUpdates.getPendingDeleteCount() + " new deletes"; - if (mergedDVUpdates.any()) { - msg += " and " + mergedDVUpdates.size() + " new field updates"; + String msg = mergedDeletesAndUpdates.getPendingDeleteCount() + " new deletes"; + if (anyDVUpdates) { + msg += " and " + mergedDeletesAndUpdates.getNumDVUpdates() + " new field updates"; + msg += " (" + mergedDeletesAndUpdates.ramBytesUsed.get() + ") bytes"; } msg += " since merge started"; infoStream.message("IW", msg); @@ -3731,7 +3836,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { merge.info.setBufferedDeletesGen(minGen); - return holder.mergedDeletesAndUpdates; + return mergedDeletesAndUpdates; } synchronized private boolean commitMerge(MergePolicy.OneMerge merge, MergeState mergeState) throws IOException { @@ -3775,7 +3880,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } final ReadersAndUpdates mergedUpdates = merge.info.info.maxDoc() == 0 ? null : commitMergedDeletesAndUpdates(merge, mergeState); -// System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMerge: mergedDeletes=" + mergedDeletes); // If the doc store we are using has been closed and // is in now compound format (but wasn't when we @@ -3922,9 +4026,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { try { try { mergeInit(merge); - //if (merge.info != null) { - //System.out.println("MERGE: " + merge.info.info.name); - //} if (infoStream.isEnabled("IW")) { infoStream.message("IW", "now merge\n merge=" + segString(merge.segments) + "\n index=" + segString()); @@ -4064,7 +4165,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { /** Does initial setup for a merge, which is fast but holds * the synchronized lock on IndexWriter instance. */ - final synchronized void mergeInit(MergePolicy.OneMerge merge) throws IOException { + final void mergeInit(MergePolicy.OneMerge merge) throws IOException { + + // Make sure any deletes that must be resolved before we commit the merge are complete: + bufferedUpdatesStream.waitApplyForMerge(merge.segments); + boolean success = false; try { _mergeInit(merge); @@ -4110,29 +4215,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { infoStream.message("IW", "now apply deletes for " + merge.segments.size() + " merging segments"); } - // Lock order: IW -> BD - final BufferedUpdatesStream.ApplyDeletesResult result = bufferedUpdatesStream.applyDeletesAndUpdates(readerPool, merge.segments); + // Must move the pending doc values updates to disk now, else the newly merged segment will not see them: + // TODO: we could fix merging to pull the merged DV iterator so we don't have to move these updates to disk first, i.e. just carry them + // in memory: + readerPool.writeDocValuesUpdates(merge.segments); - if (result.anyDeletes) { - checkpoint(); - } - - if (!keepFullyDeletedSegments && result.allDeleted != null) { - if (infoStream.isEnabled("IW")) { - infoStream.message("IW", "drop 100% deleted segments: " + result.allDeleted); - } - for(SegmentCommitInfo info : result.allDeleted) { - segmentInfos.remove(info); - pendingNumDocs.addAndGet(-info.info.maxDoc()); - if (merge.segments.contains(info)) { - mergingSegments.remove(info); - merge.segments.remove(info); - } - readerPool.drop(info); - } - checkpoint(); - } - // Bind a new segment name here so even with // ConcurrentMergePolicy we keep deterministic segment // names. @@ -4145,11 +4232,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { setDiagnostics(si, SOURCE_MERGE, details); merge.setMergeInfo(new SegmentCommitInfo(si, 0, -1L, -1L, -1L)); -// System.out.println("[" + Thread.currentThread().getName() + "] IW._mergeInit: " + segString(merge.segments) + " into " + si); - - // Lock order: IW -> BD - bufferedUpdatesStream.prune(segmentInfos); - if (infoStream.isEnabled("IW")) { infoStream.message("IW", "merge seg=" + merge.info.info.name + " " + segString(merge.segments)); } @@ -4204,7 +4286,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { final int numSegments = merge.readers.size(); Throwable th = null; - boolean drop = !suppressExceptions; + boolean drop = suppressExceptions == false; for (int i = 0; i < numSegments; i++) { final SegmentReader sr = merge.readers.get(i); @@ -4278,59 +4360,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // commit merged deletes final ReadersAndUpdates rld = readerPool.get(info, true); - // Carefully pull the most recent live docs and reader - SegmentReader reader; - final Bits liveDocs; - final int delCount; + SegmentReader reader = rld.getReaderForMerge(context); + int delCount = reader.numDeletedDocs(); - synchronized (this) { - // Must sync to ensure BufferedDeletesStream cannot change liveDocs, - // pendingDeleteCount and field updates while we pull a copy: - reader = rld.getReaderForMerge(context); - liveDocs = rld.getReadOnlyLiveDocs(); - delCount = rld.getPendingDeleteCount() + info.getDelCount(); - - assert reader != null; - assert rld.verifyDocCounts(); - - if (infoStream.isEnabled("IW")) { - if (rld.getPendingDeleteCount() != 0) { - infoStream.message("IW", "seg=" + segString(info) + " delCount=" + info.getDelCount() + " pendingDelCount=" + rld.getPendingDeleteCount()); - } else if (info.getDelCount() != 0) { - infoStream.message("IW", "seg=" + segString(info) + " delCount=" + info.getDelCount()); - } else { - infoStream.message("IW", "seg=" + segString(info) + " no deletes"); - } - } - } - - // Deletes might have happened after we pulled the merge reader and - // before we got a read-only copy of the segment's actual live docs - // (taking pending deletes into account). In that case we need to - // make a new reader with updated live docs and del count. - if (reader.numDeletedDocs() != delCount) { - // fix the reader's live docs and del count - assert delCount > reader.numDeletedDocs(); // beware of zombies - - SegmentReader newReader; - - synchronized (this) { - // We must also sync on IW here, because another thread could be writing - // new DV updates / remove old gen field infos files causing FNFE: - newReader = new SegmentReader(info, reader, liveDocs, info.info.maxDoc() - delCount); - } - - boolean released = false; - try { - rld.release(reader); - released = true; - } finally { - if (!released) { - newReader.decRef(); - } - } - - reader = newReader; + if (infoStream.isEnabled("IW")) { + infoStream.message("IW", "seg=" + segString(info) + " reader=" + reader); } merge.readers.add(reader); @@ -4338,8 +4372,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { segUpto++; } -// System.out.println("[" + Thread.currentThread().getName() + "] IW.mergeMiddle: merging " + merge.getMergeReaders()); - // Let the merge wrap readers List mergeReaders = new ArrayList<>(); for (SegmentReader reader : merge.readers) { @@ -4411,7 +4443,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // Very important to do this before opening the reader // because codec must know if prox was written for // this segment: - //System.out.println("merger set hasProx=" + merger.hasProx() + " seg=" + merge.info.name); boolean useCompoundFile; synchronized (this) { // Guard segmentInfos useCompoundFile = mergePolicy.useCompoundFile(segmentInfos, merge.info, this); @@ -4601,7 +4632,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } } - private boolean keepFullyDeletedSegments; + boolean keepFullyDeletedSegments; /** Only for testing. * @@ -4714,8 +4745,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { infoStream.message("IW", "startCommit: wrote pending segments file \"" + IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS, "", toSync.getGeneration()) + "\""); } - //System.out.println("DONE prepareCommit"); - pendingCommitSet = true; pendingCommit = toSync; } @@ -4863,9 +4892,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } synchronized boolean nrtIsCurrent(SegmentInfos infos) { - //System.out.println("IW.nrtIsCurrent " + (infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any())); ensureOpen(); - boolean isCurrent = infos.getVersion() == segmentInfos.getVersion() && !docWriter.anyChanges() && !bufferedUpdatesStream.any(); + boolean isCurrent = infos.getVersion() == segmentInfos.getVersion() + && docWriter.anyChanges() == false + && bufferedUpdatesStream.any() == false + && readerPool.anyChanges() == false; if (infoStream.isEnabled("IW")) { if (isCurrent == false) { infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.getVersion() == segmentInfos.getVersion()) + "; DW changes: " + docWriter.anyChanges() + "; BD changes: "+ bufferedUpdatesStream.any()); @@ -4977,9 +5008,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { try { purge(forcePurge); } finally { - if (applyAllDeletesAndUpdates()) { - maybeMerge(config.getMergePolicy(), MergeTrigger.SEGMENT_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS); - } flushCount.incrementAndGet(); } } @@ -5017,20 +5045,22 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } } - private boolean processEvents(boolean triggerMerge, boolean forcePurge) throws IOException { - return processEvents(eventQueue, triggerMerge, forcePurge); + private void processEvents(boolean triggerMerge, boolean forcePurge) throws IOException { + processEvents(eventQueue, triggerMerge, forcePurge); + if (triggerMerge) { + maybeMerge(getConfig().getMergePolicy(), MergeTrigger.SEGMENT_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS); + } } - private boolean processEvents(Queue queue, boolean triggerMerge, boolean forcePurge) throws IOException { + private void processEvents(Queue queue, boolean triggerMerge, boolean forcePurge) throws IOException { boolean processed = false; if (tragedy == null) { Event event; - while((event = queue.poll()) != null) { + while ((event = queue.poll()) != null) { processed = true; event.process(this, triggerMerge, forcePurge); } } - return processed; } /** diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java index 0fdbc3e8652..1377a95e1cf 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java @@ -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); diff --git a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java index d9e1bc7bebb..cff1074f963 100644 --- a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java +++ b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java @@ -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. - *

    - * Disabled by default (writer flushes by RAM usage). - *

    - * NOTE: This setting won't trigger a segment flush. - * - *

    - * 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 { * NOTE: 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. *

    * NOTE: 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(); diff --git a/lucene/core/src/java/org/apache/lucene/index/MergedPrefixCodedTermsIterator.java b/lucene/core/src/java/org/apache/lucene/index/MergedPrefixCodedTermsIterator.java deleted file mode 100644 index cd14eecc2c8..00000000000 --- a/lucene/core/src/java/org/apache/lucene/index/MergedPrefixCodedTermsIterator.java +++ /dev/null @@ -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 { - 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 { - 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 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(); - } -} - diff --git a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java index 4dd3cd0b8fb..a42754df4af 100644 --- a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java @@ -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; } - } diff --git a/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java b/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java index df1653bcd4d..ba56c2a4636 100644 --- a/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java +++ b/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java @@ -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); } diff --git a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java index d4dd4a44a64..630131ef5e7 100644 --- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java @@ -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 mergingDVUpdates = new HashMap<>(); + + // Holds resolved (to docIDs) doc values updates that have not yet been + // written to the index + private final Map> 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> 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 fieldUpdates, DocValuesFieldUpdates update) { + for (int i=0;i 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 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 updates, - Directory dir, DocValuesFormat dvFormat, final SegmentReader reader, Map> fieldFiles) throws IOException { - for (Entry 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> fieldFiles, long maxDelGen, InfoStream infoStream) throws IOException { + + for (Entry> ent : pendingDVUpdates.entrySet()) { + final String field = ent.getKey(); + final List updates = ent.getValue(); + if (updates.get(0).type != DocValuesType.NUMERIC) { + continue; + } + + final List 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 updates, - TrackingDirectoryWrapper dir, DocValuesFormat dvFormat, final SegmentReader reader, Map> fieldFiles) throws IOException { - for (Entry 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> fieldFiles, long maxDelGen, InfoStream infoStream) throws IOException { + for (Entry> ent : pendingDVUpdates.entrySet()) { + final String field = ent.getKey(); + final List updates = ent.getValue(); + if (updates.get(0).type != DocValuesType.BINARY) { + continue; + } + + final List 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 writeFieldInfosGen(FieldInfos fieldInfos, Directory dir, DocValuesFormat dvFormat, + private synchronized Set 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> newDVFiles = new HashMap<>(); Set fieldInfosFiles = null; FieldInfos fieldInfos = null; + + boolean any = false; + int count = 0; + for (List 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 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 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>> it = pendingDVUpdates.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry> ent = it.next(); + int upto = 0; + List 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 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> 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> ent : pendingDVUpdates.entrySet()) { + List 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 getMergingFieldUpdates() { + + public synchronized Map> 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(); } - } diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java index b1084a6b2df..661283b4203 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java @@ -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 diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java index ce2d44890c0..2d2e786b181 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java @@ -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, diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java index 1c024417cb3..5e6d773cc95 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java @@ -172,7 +172,7 @@ public final class SegmentInfo { /** Return all files referenced by this SegmentInfo. */ public Set 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); } diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java b/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java index c8235d5cba8..bd821562a17 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java @@ -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); + } } } diff --git a/lucene/core/src/java/org/apache/lucene/index/SerialMergeScheduler.java b/lucene/core/src/java/org/apache/lucene/index/SerialMergeScheduler.java index 5a8f98b016b..ce68247593b 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SerialMergeScheduler.java +++ b/lucene/core/src/java/org/apache/lucene/index/SerialMergeScheduler.java @@ -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); } } diff --git a/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java index 668f1ec8c1c..c020a6de062 100644 --- a/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java @@ -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 { - private final IndexWriter writer; + private final Map sizeInBytes; - SegmentByteSizeDescending(IndexWriter writer) { - this.writer = writer; + SegmentByteSizeDescending(Map 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 getSegmentSizes(IndexWriter writer, Collection infos) throws IOException { + Map 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 toBeMerged = new HashSet<>(); final List 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 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 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 candidate, boolean hitTooLarge, long mergingBytes, IndexWriter writer) throws IOException { + protected MergeScore score(List candidate, boolean hitTooLarge, long mergingBytes, IndexWriter writer, Map 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 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 sizeInBytes = getSegmentSizes(writer, infos.asList()); + + Collections.sort(eligible, new SegmentByteSizeDescending(sizeInBytes)); if (verbose(writer)) { message("eligible=" + eligible, writer); diff --git a/lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java b/lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java index c5fac1e9912..e73863ba861 100644 --- a/lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java +++ b/lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java @@ -84,7 +84,7 @@ abstract class AbstractPagedMutable> 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); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java index ed2b66f8a46..80ae80423e7 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java @@ -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 { + 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 docs = new ArrayList<>(); + DirectoryReader r = w.getReader(); + + int numIters = atLeast(1000); + for(int iter=0;iter= 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 diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java b/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java index c60f54d72a6..8991aea0588 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java @@ -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 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 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 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())); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java index aa2901cabbe..ca5aba8d948 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java @@ -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; } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestForceMergeForever.java b/lucene/core/src/test/org/apache/lucene/index/TestForceMergeForever.java index 037939534f1..8626fe5f8ae 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestForceMergeForever.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestForceMergeForever.java @@ -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 diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java index 6897f06ab08..9538e03d5a0 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java @@ -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())) diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java index 2014c16237e..464966adf1c 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java @@ -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); }); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java index 975f5f072f8..8bc3f42f338 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java @@ -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(); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java index c0907a5e682..ad35f32e1d7 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java @@ -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 docs = new ArrayList<>(); + DirectoryReader r = w.getReader(); + + int numIters = atLeast(1000); + for(int iter=0;iter= 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 diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java index 112a1084917..fc56614ba6d 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java @@ -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 diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java b/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java index 89d4ad17d45..3ec0b568268 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java @@ -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 superSet = new TreeSet<>(); - - for (int i = 0; i < pb.length; i++) { - Set 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 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()); - } } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java index a1b2a5c2917..7a7b0ac6cb6 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java @@ -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 { diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java index 1503de8cc3b..0bd47849c7b 100644 --- a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java +++ b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java @@ -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 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(); diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java index 334f784c405..fc643d206d7 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java +++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java @@ -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"); diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java index d83b9155e5b..e9187af3cd5 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java +++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java @@ -161,7 +161,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer { } private final List 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) { diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java index 8cb666570eb..28ab3b6d01f 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java @@ -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(); } diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java index 959466a59f2..ab929460975 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java @@ -564,7 +564,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase { handleFakeIOException(e, exceptionStream); allowAlreadyClosed = true; } - + if (random().nextInt(10) == 0) { // trigger flush: try { diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java index 0243a56ccb6..3a87c1e24c4 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java @@ -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) {