You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2020/06/24 16:50:33 UTC
[lucene-solr] 04/07: LUCENE-8962: Ensure we don't include fully
deleted segments in a commit
This is an automated email from the ASF dual-hosted git repository.
simonw pushed a commit to branch jira/lucene-8962
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git
commit 189244b60a137626d5029e1e56899da3acb2f340
Author: Simon Willnauer <si...@apache.org>
AuthorDate: Mon Jun 22 19:44:06 2020 +0200
LUCENE-8962: Ensure we don't include fully deleted segments in a commit
IW might drop segments that are merged into a fully deleted segment on the floor
and deletes the newly created files right away. We should not include these segments
in a commit since we can't guarantee valid ref-counts on these files.
---
.../java/org/apache/lucene/index/IndexWriter.java | 71 ++++++++++++----------
.../java/org/apache/lucene/index/MergePolicy.java | 6 +-
.../lucene/index/TestDemoParallelLeafReader.java | 2 +-
.../org/apache/lucene/index/TestIndexWriter.java | 2 +-
.../org/apache/lucene/index/TestMergePolicy.java | 10 +--
5 files changed, 50 insertions(+), 41 deletions(-)
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 bd273a1..88dc399 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -3234,15 +3234,23 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
// corresponding add from an updateDocument) can
// sneak into the commit point:
toCommit = segmentInfos.clone();
-
+ pendingCommitChangeCount = changeCount.get();
+ // This protects the segmentInfos we are now going
+ // to commit. This is important in case, eg, while
+ // we are trying to sync all referenced files, a
+ // merge completes which would otherwise have
+ // removed the files we are now syncing.
+ deleter.incRef(toCommit.files(false));
if (anyChanges && maxCommitMergeWaitSeconds > 0) {
SegmentInfos committingSegmentInfos = toCommit;
onCommitMerges = updatePendingMerges(new OneMergeWrappingMergePolicy(config.getMergePolicy(), toWrap ->
new MergePolicy.OneMerge(toWrap.segments) {
@Override
- public void mergeFinished(boolean committed) throws IOException {
+ public void mergeFinished(boolean committed, boolean segmentDropped) throws IOException {
assert Thread.holdsLock(IndexWriter.this);
- if (committed && includeInCommit.get()) {
+ if (segmentDropped == false
+ && committed
+ && includeInCommit.get()) {
deleter.incRef(info.files());
Set<String> mergedSegmentNames = new HashSet<>();
for (SegmentCommitInfo sci : segments) {
@@ -3262,8 +3270,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
committingSegmentInfos.counter = Math.max(committingSegmentInfos.counter, segmentCounter + 1);
committingSegmentInfos.applyMergeChanges(applicableMerge, false);
}
- toWrap.mergeFinished(committed);
- super.mergeFinished(committed);
+ toWrap.mergeFinished(committed, false);
+ super.mergeFinished(committed, segmentDropped);
}
@Override
@@ -3274,14 +3282,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
), MergeTrigger.COMMIT, UNBOUNDED_MAX_MERGE_SEGMENTS);
}
- pendingCommitChangeCount = changeCount.get();
- // This protects the segmentInfos we are now going
- // to commit. This is important in case, eg, while
- // we are trying to sync all referenced files, a
- // merge completes which would otherwise have
- // removed the files we are now syncing.
- deleter.incRef(toCommit.files(false));
}
success = true;
} finally {
@@ -4019,7 +4020,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
// Must close before checkpoint, otherwise IFD won't be
// able to delete the held-open files from the merge
// readers:
- closeMergeReaders(merge, false);
+ closeMergeReaders(merge, false, dropSegment);
}
if (infoStream.isEnabled("IW")) {
@@ -4341,27 +4342,33 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
}
@SuppressWarnings("try")
- private synchronized void closeMergeReaders(MergePolicy.OneMerge merge, boolean suppressExceptions) throws IOException {
+ private synchronized void closeMergeReaders(MergePolicy.OneMerge merge, boolean suppressExceptions, boolean droppedSegment) throws IOException {
final boolean drop = suppressExceptions == false;
- try (Closeable finalizer = () -> merge.mergeFinished(suppressExceptions == false)) {
- IOUtils.applyToAll(merge.readers, sr -> {
- final ReadersAndUpdates rld = getPooledInstance(sr.getOriginalSegmentInfo(), false);
- // We still hold a ref so it should not have been removed:
- assert rld != null;
- if (drop) {
- rld.dropChanges();
- } else {
- rld.dropMergingUpdates();
- }
- rld.release(sr);
- release(rld);
- if (drop) {
- readerPool.drop(rld.info);
- }
- });
+ try {
+ // first call mergeFinished before we potentially drop the reader and the last reference.
+ merge.mergeFinished(suppressExceptions == false, droppedSegment);
} finally {
- Collections.fill(merge.readers, null);
+ try {
+ IOUtils.applyToAll(merge.readers, sr -> {
+ final ReadersAndUpdates rld = getPooledInstance(sr.getOriginalSegmentInfo(), false);
+ // We still hold a ref so it should not have been removed:
+ assert rld != null;
+ if (drop) {
+ rld.dropChanges();
+ } else {
+ rld.dropMergingUpdates();
+ }
+ rld.release(sr);
+ release(rld);
+ if (drop) {
+ readerPool.drop(rld.info);
+ }
+ });
+ } finally {
+ Collections.fill(merge.readers, null);
+ }
}
+
}
private void countSoftDeletes(CodecReader reader, Bits wrappedLiveDocs, Bits hardLiveDocs, Counter softDeleteCounter,
@@ -4661,7 +4668,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
// Readers are already closed in commitMerge if we didn't hit
// an exc:
if (success == false) {
- closeMergeReaders(merge, true);
+ closeMergeReaders(merge, true, false);
}
}
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 5a090da..80c7b0d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
@@ -254,8 +254,10 @@ public abstract class MergePolicy {
}
/** Called by {@link IndexWriter} after the merge is done and all readers have been closed.
- * @param success true iff the merge finished successfully ie. was committed */
- public void mergeFinished(boolean success) throws IOException {
+ * @param success true iff the merge finished successfully ie. was committed
+ * @param segmentDropped true iff the merged segment was dropped since it was fully deleted
+ */
+ public void mergeFinished(boolean success, boolean segmentDropped) throws IOException {
mergeCompleted.complete(success);
// https://issues.apache.org/jira/browse/LUCENE-9408
// if (mergeCompleted.complete(success) == false) {
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
index 7fdad3b..2ab2c92 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
@@ -538,7 +538,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
}
@Override
- public void mergeFinished(boolean success) throws IOException {
+ public void mergeFinished(boolean success, boolean segmentDropped) throws IOException {
Throwable th = null;
for (ParallelLeafReader r : parallelReaders) {
try {
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 5c6164a..41bd607 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -4181,7 +4181,7 @@ public class TestIndexWriter extends LuceneTestCase {
SetOnce<Boolean> onlyFinishOnce = new SetOnce<>();
return new MergePolicy.OneMerge(merge.segments) {
@Override
- public void mergeFinished(boolean success) {
+ public void mergeFinished(boolean success, boolean segmentDropped) {
onlyFinishOnce.set(true);
}
};
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestMergePolicy.java
index e5f5635..0ff32e6 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestMergePolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestMergePolicy.java
@@ -43,7 +43,7 @@ public class TestMergePolicy extends LuceneTestCase {
Thread t = new Thread(() -> {
try {
for (MergePolicy.OneMerge m : ms.merges) {
- m.mergeFinished(true);
+ m.mergeFinished(true, false);
}
} catch (IOException e) {
throw new AssertionError(e);
@@ -66,7 +66,7 @@ public class TestMergePolicy extends LuceneTestCase {
}
Thread t = new Thread(() -> {
try {
- ms.merges.get(0).mergeFinished(true);
+ ms.merges.get(0).mergeFinished(true, false);
} catch (IOException e) {
throw new AssertionError(e);
}
@@ -89,7 +89,7 @@ public class TestMergePolicy extends LuceneTestCase {
Thread t = new Thread(() -> {
while (stop.get() == false) {
try {
- ms.merges.get(i.getAndIncrement()).mergeFinished(true);
+ ms.merges.get(i.getAndIncrement()).mergeFinished(true, false);
Thread.sleep(1);
} catch (IOException | InterruptedException e) {
throw new AssertionError(e);
@@ -115,8 +115,8 @@ public class TestMergePolicy extends LuceneTestCase {
try (Directory dir = newDirectory()) {
MergePolicy.MergeSpecification spec = createRandomMergeSpecification(dir, 1);
MergePolicy.OneMerge oneMerge = spec.merges.get(0);
- oneMerge.mergeFinished(true);
- expectThrows(IllegalStateException.class, () -> oneMerge.mergeFinished(false));
+ oneMerge.mergeFinished(true, false);
+ expectThrows(IllegalStateException.class, () -> oneMerge.mergeFinished(false, false));
}
}