From commits-return-101336-archive-asf-public=cust-asf.ponee.io@lucene.apache.org Thu May 31 09:24:49 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 8CB2D180632 for ; Thu, 31 May 2018 09:24:48 +0200 (CEST) Received: (qmail 28182 invoked by uid 500); 31 May 2018 07:24:47 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 28173 invoked by uid 99); 31 May 2018 07:24:47 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 31 May 2018 07:24:47 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 62B95E0A90; Thu, 31 May 2018 07:24:47 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: simonw@apache.org To: commits@lucene.apache.org Message-Id: <5b63ddeb12064cba85c1f8e5d65f3e5e@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: lucene-solr:branch_7x: LUCENE-8338: Ensure number returned for PendingDeletes are well defined Date: Thu, 31 May 2018 07:24:47 +0000 (UTC) Repository: lucene-solr Updated Branches: refs/heads/branch_7x 528b96540 -> a462c147a LUCENE-8338: Ensure number returned for PendingDeletes are well defined Today a call to PendingDeletes#numPendingDeletes might return 0 if the deletes are written to disk. This doesn't mean these values are committed or refreshed in the latest reader. Some places in IW use these numbers to make decisions if there has been deletes added since last time checked (BufferedUpdateStream) which can cause wrong (while not fatal) decision ie. to kick of new merges. Now this API is made protected and not visible outside of PendingDeletes to prevent any kind of confusion. The APIs now allow to get absolute numbers of getDelCount and numDocs which have the same name and semantics as their relatives on IndexReader/Writer and SegmentCommitInfo. Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/a462c147 Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/a462c147 Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/a462c147 Branch: refs/heads/branch_7x Commit: a462c147a0607a984ce157d9b7fae0c20c8a9ad7 Parents: 528b965 Author: Simon Willnauer Authored: Tue May 29 16:08:12 2018 +0200 Committer: Simon Willnauer Committed: Thu May 31 09:21:38 2018 +0200 ---------------------------------------------------------------------- .../lucene/index/BufferedUpdatesStream.java | 2 +- .../apache/lucene/index/FilterMergePolicy.java | 4 +- .../lucene/index/FrozenBufferedUpdates.java | 4 +- .../org/apache/lucene/index/IndexWriter.java | 16 +++---- .../org/apache/lucene/index/MergePolicy.java | 6 +-- .../org/apache/lucene/index/NoMergePolicy.java | 4 +- .../org/apache/lucene/index/PendingDeletes.java | 46 +++++++++++++++++++- .../apache/lucene/index/PendingSoftDeletes.java | 9 ++-- .../org/apache/lucene/index/ReaderPool.java | 3 +- .../apache/lucene/index/ReadersAndUpdates.java | 45 +++++-------------- .../index/SoftDeletesRetentionMergePolicy.java | 4 +- .../org/apache/lucene/index/TestReaderPool.java | 5 +-- .../org/apache/lucene/index/TestTryDelete.java | 2 +- 13 files changed, 83 insertions(+), 67 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java ---------------------------------------------------------------------- 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 c93e4b6..dcc8bbf 100644 --- a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java +++ b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java @@ -259,7 +259,7 @@ final class BufferedUpdatesStream implements Accountable { SegmentState(ReadersAndUpdates rld, IOUtils.IOConsumer onClose, SegmentCommitInfo info) throws IOException { this.rld = rld; - startDelCount = rld.getPendingDeleteCount(); + startDelCount = rld.getDelCount(); delGen = info.getBufferedDeletesGen(); this.onClose = onClose; reader = rld.getReader(IOContext.READ); http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java index cbc8b18..eb634b4 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java @@ -99,8 +99,8 @@ public class FilterMergePolicy extends MergePolicy { } @Override - public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount, + public int numDeletesToMerge(SegmentCommitInfo info, int delCount, IOSupplier readerSupplier) throws IOException { - return in.numDeletesToMerge(info, pendingDeleteCount, readerSupplier); + return in.numDeletesToMerge(info, delCount, readerSupplier); } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java ---------------------------------------------------------------------- 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 5ea9392..560d86d 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java @@ -389,8 +389,8 @@ final class FrozenBufferedUpdates { final List segmentStates = Arrays.asList(segStates); for (BufferedUpdatesStream.SegmentState segState : segmentStates) { if (success) { - totDelCount += segState.rld.getPendingDeleteCount() - segState.startDelCount; - int fullDelCount = segState.rld.info.getDelCount() + segState.rld.getPendingDeleteCount(); + totDelCount += segState.rld.getDelCount() - segState.startDelCount; + int fullDelCount = segState.rld.getDelCount(); assert fullDelCount <= segState.rld.info.info.maxDoc() : fullDelCount + " > " + segState.rld.info.info.maxDoc(); if (segState.rld.isFullyDeleted() && writer.getConfig().getMergePolicy().keepFullyDeletedSegment(() -> segState.reader) == false) { if (allDeleted == null) { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java ---------------------------------------------------------------------- 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 5c53cb7..ba30df0 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -635,14 +635,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable, public int numDeletedDocs(SegmentCommitInfo info) { ensureOpen(false); validate(info); - int delCount = info.getDelCount(); - final ReadersAndUpdates rld = getPooledInstance(info, false); if (rld != null) { - delCount += rld.getPendingDeleteCount(); + return rld.getDelCount(); // get the full count from here since SCI might change concurrently + } else { + int delCount = info.getDelCount(); + assert delCount <= info.info.maxDoc(): "delCount: " + delCount + " maxDoc: " + info.info.maxDoc(); + return delCount; } - assert delCount <= info.info.maxDoc(): "delCount: " + delCount + " maxDoc: " + info.info.maxDoc(); - return delCount; } /** @@ -3697,7 +3697,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable, // Lazy init (only when we find a delete or update to carry over): final ReadersAndUpdates mergedDeletesAndUpdates = getPooledInstance(merge.info, true); - + int numDeletesBefore = mergedDeletesAndUpdates.getDelCount(); // field -> delGen -> dv field updates Map> mappedDVUpdates = new HashMap<>(); @@ -3788,7 +3788,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable, if (mergedDeletesAndUpdates == null) { infoStream.message("IW", "no new deletes or field updates since merge started"); } else { - String msg = mergedDeletesAndUpdates.getPendingDeleteCount() + " new deletes"; + String msg = mergedDeletesAndUpdates.getDelCount() - numDeletesBefore + " new deletes"; if (anyDVUpdates) { msg += " and " + mergedDeletesAndUpdates.getNumDVUpdates() + " new field updates"; msg += " (" + mergedDeletesAndUpdates.ramBytesUsed.get() + ") bytes"; @@ -4363,7 +4363,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable, ReadersAndUpdates.MergeReader mr = rld.getReaderForMerge(context); SegmentReader reader = mr.reader; - int delCount = reader.numDeletedDocs(); if (infoStream.isEnabled("IW")) { infoStream.message("IW", "seg=" + segString(info) + " reader=" + reader); @@ -4371,7 +4370,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable, merge.hardLiveDocs.add(mr.hardLiveDocs); merge.readers.add(reader); - assert delCount <= info.info.maxDoc(): "delCount=" + delCount + " info.maxDoc=" + info.info.maxDoc() + " rld.pendingDeleteCount=" + rld.getPendingDeleteCount() + " info.getDelCount()=" + info.getDelCount(); segUpto++; } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java index 47a09e8..d552d74 100644 --- a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java @@ -638,12 +638,12 @@ public abstract class MergePolicy { * @see IndexWriter#softUpdateDocument(Term, Iterable, Field...) * @see IndexWriterConfig#setSoftDeletesField(String) * @param info the segment info that identifies the segment - * @param pendingDeleteCount the number of pending deletes for this segment + * @param delCount the number deleted documents for this segment * @param readerSupplier a supplier that allows to obtain a {@link CodecReader} for this segment */ - public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount, + public int numDeletesToMerge(SegmentCommitInfo info, int delCount, IOSupplier readerSupplier) throws IOException { - return info.getDelCount() + pendingDeleteCount; + return delCount; } /** http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java index f3449e2..1480ce4 100644 --- a/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java @@ -81,8 +81,8 @@ public final class NoMergePolicy extends MergePolicy { } @Override - public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount, IOSupplier readerSupplier) throws IOException { - return super.numDeletesToMerge(info, pendingDeleteCount, readerSupplier); + public int numDeletesToMerge(SegmentCommitInfo info, int delCount, IOSupplier readerSupplier) throws IOException { + return super.numDeletesToMerge(info, delCount, readerSupplier); } @Override http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java index 506d397..f19b053 100644 --- a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java +++ b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java @@ -118,7 +118,7 @@ class PendingDeletes { /** * Returns the number of pending deletes that are not written to disk. */ - int numPendingDeletes() { + protected int numPendingDeletes() { return pendingDeleteCount; } @@ -232,7 +232,49 @@ class PendingDeletes { } int numDeletesToMerge(MergePolicy policy, IOSupplier readerIOSupplier) throws IOException { - return policy.numDeletesToMerge(info, numPendingDeletes(), readerIOSupplier); + return policy.numDeletesToMerge(info, getDelCount(), readerIOSupplier); } + /** + * Returns true if the given reader needs to be refreshed in order to see the latest deletes + */ + final boolean needsRefresh(CodecReader reader) { + return reader.getLiveDocs() != getLiveDocs() || reader.numDeletedDocs() != getDelCount(); + } + + /** + * Returns the number of deleted docs in the segment. + */ + final int getDelCount() { + return info.getDelCount() + numPendingDeletes(); + } + + /** + * Returns the number of live documents in this segment + */ + final int numDocs() { + return info.info.maxDoc() - getDelCount(); + } + + // Call only from assert! + boolean verifyDocCounts(CodecReader reader) { + int count = 0; + Bits liveDocs = getLiveDocs(); + if (liveDocs != null) { + for(int docID = 0; docID < info.info.maxDoc(); docID++) { + if (liveDocs.get(docID)) { + count++; + } + } + } else { + count = info.info.maxDoc(); + } + assert numDocs() == count: "info.maxDoc=" + info.info.maxDoc() + " info.getDelCount()=" + info.getDelCount() + + " pendingDeletes=" + toString() + " count=" + count; + assert reader.numDocs() == numDocs() : "reader.numDocs() = " + reader.numDocs() + " numDocs() " + numDocs(); + assert reader.numDeletedDocs() <= info.info.maxDoc(): "delCount=" + reader.numDeletedDocs() + " info.maxDoc=" + + info.info.maxDoc() + " rld.pendingDeleteCount=" + numPendingDeletes() + + " info.getDelCount()=" + info.getDelCount(); + return true; + } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java index fe012f6..1c32e4f 100644 --- a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java +++ b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java @@ -58,6 +58,7 @@ final class PendingSoftDeletes extends PendingDeletes { } else { // if it was deleted subtract the delCount pendingDeleteCount--; + assert pendingDeleteCount >= 0 : " illegal pending delete count: " + pendingDeleteCount; } return true; } @@ -65,7 +66,7 @@ final class PendingSoftDeletes extends PendingDeletes { } @Override - int numPendingDeletes() { + protected int numPendingDeletes() { return super.numPendingDeletes() + hardDeletes.numPendingDeletes(); } @@ -78,11 +79,11 @@ final class PendingSoftDeletes extends PendingDeletes { if (iterator != null) { // nothing is deleted we don't have a soft deletes field in this segment assert info.info.maxDoc() > 0 : "maxDoc is 0"; pendingDeleteCount += applySoftDeletes(iterator, getMutableBits()); + assert pendingDeleteCount >= 0 : " illegal pending delete count: " + pendingDeleteCount; } dvGeneration = info.getDocValuesGen(); } - assert numPendingDeletes() + info.getDelCount() <= info.info.maxDoc() : - numPendingDeletes() + " + " + info.getDelCount() + " > " + info.info.maxDoc(); + assert getDelCount() <= info.info.maxDoc() : getDelCount() + " > " + info.info.maxDoc(); } @Override @@ -133,6 +134,7 @@ final class PendingSoftDeletes extends PendingDeletes { void onDocValuesUpdate(FieldInfo info, DocValuesFieldUpdates.Iterator iterator) throws IOException { if (this.field.equals(info.name)) { pendingDeleteCount += applySoftDeletes(iterator, getMutableBits()); + assert pendingDeleteCount >= 0 : " illegal pending delete count: " + pendingDeleteCount; assert dvGeneration < info.getDocValuesGen() : "we have seen this generation update already: " + dvGeneration + " vs. " + info.getDocValuesGen(); assert dvGeneration != -2 : "docValues generation is still uninitialized"; dvGeneration = info.getDocValuesGen(); @@ -208,5 +210,4 @@ final class PendingSoftDeletes extends PendingDeletes { Bits getHardLiveDocs() { return hardDeletes.getLiveDocs(); } - } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/ReaderPool.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/index/ReaderPool.java b/lucene/core/src/java/org/apache/lucene/index/ReaderPool.java index 861cfaf..45f58a6 100644 --- a/lucene/core/src/java/org/apache/lucene/index/ReaderPool.java +++ b/lucene/core/src/java/org/apache/lucene/index/ReaderPool.java @@ -132,7 +132,7 @@ final class ReaderPool implements Closeable { */ synchronized boolean anyPendingDeletes() { for(ReadersAndUpdates rld : readerMap.values()) { - if (rld.getPendingDeleteCount() != 0) { + if (rld.anyPendingDeletes()) { return true; } } @@ -321,7 +321,6 @@ final class ReaderPool implements Closeable { /** * Returns true iff there are any buffered doc values updates. Otherwise false. - * @see #anyPendingDeletes() */ synchronized boolean anyDocValuesChanges() { for (ReadersAndUpdates rld : readerMap.values()) { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java ---------------------------------------------------------------------- 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 5558595..710b748 100644 --- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java @@ -100,8 +100,6 @@ final class ReadersAndUpdates { *

NOTE: steals incoming ref from reader. */ ReadersAndUpdates(int indexCreatedVersionMajor, SegmentReader reader, PendingDeletes pendingDeletes) throws IOException { this(indexCreatedVersionMajor, reader.getOriginalSegmentInfo(), pendingDeletes); - assert pendingDeletes.numPendingDeletes() >= 0 - : "got " + pendingDeletes.numPendingDeletes() + " reader.numDeletedDocs()=" + reader.numDeletedDocs() + " info.getDelCount()=" + info.getDelCount() + " maxDoc=" + reader.maxDoc() + " numDocs=" + reader.numDocs(); this.reader = reader; pendingDeletes.onNewReader(reader, info); } @@ -122,10 +120,9 @@ final class ReadersAndUpdates { return rc; } - public synchronized int getPendingDeleteCount() { - return pendingDeletes.numPendingDeletes(); + public synchronized int getDelCount() { + return pendingDeletes.getDelCount(); } - private synchronized boolean assertNoDupGen(List fieldUpdates, DocValuesFieldUpdates update) { for (int i=0;i