You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/07/03 11:58:35 UTC
[55/58] [abbrv] lucene-solr:jira/solr-10879: LUCENE-7888: fix
concurrency hazards between merge completing and DV updates applying
LUCENE-7888: fix concurrency hazards between merge completing and DV updates applying
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/eaf1d45a
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/eaf1d45a
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/eaf1d45a
Branch: refs/heads/jira/solr-10879
Commit: eaf1d45a1cad74a1037c7c4178fd2379a903f8cc
Parents: ee1edd9
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Jul 2 15:58:53 2017 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Jul 2 15:58:53 2017 -0400
----------------------------------------------------------------------
.../lucene/index/FrozenBufferedUpdates.java | 19 +++++++++++++++----
.../org/apache/lucene/index/IndexWriter.java | 5 +++--
.../apache/lucene/index/ReadersAndUpdates.java | 8 +++++---
3 files changed, 23 insertions(+), 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eaf1d45a/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 8248841..c4e60db 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
@@ -262,6 +262,8 @@ class FrozenBufferedUpdates {
int totalSegmentCount = 0;
long totalDelCount = 0;
+ boolean finished = false;
+
// 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.
@@ -334,7 +336,7 @@ class FrozenBufferedUpdates {
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",
+ messagePrefix + "done inner apply del packet (%s) to %d segments; %d new deletes/updates; took %.3f sec",
this, segStates.length, delCount, (System.nanoTime() - iterStartNS) / 1000000000.));
}
@@ -352,6 +354,13 @@ class FrozenBufferedUpdates {
if (mergeGenCur == mergeGenStart) {
+ // Must do this while still holding IW lock else a merge could finish and skip carrying over our updates:
+
+ // Record that this packet is finished:
+ writer.bufferedUpdatesStream.finished(this);
+
+ finished = true;
+
// No merge finished while we were applying, so we are done!
break;
}
@@ -367,9 +376,11 @@ class FrozenBufferedUpdates {
iter++;
}
- // Record that this packet is finished:
- writer.bufferedUpdatesStream.finished(this);
-
+ if (finished == false) {
+ // 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",
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eaf1d45a/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 2ef928b..26a25a5 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -642,12 +642,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
}
- void writeDocValuesUpdates(List<SegmentCommitInfo> infos) throws IOException {
+ void writeDocValuesUpdatesForMerge(List<SegmentCommitInfo> 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);
+ rld.setIsMerging();
}
}
if (any) {
@@ -4216,7 +4217,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// 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);
+ readerPool.writeDocValuesUpdatesForMerge(merge.segments);
// Bind a new segment name here so even with
// ConcurrentMergePolicy we keep deterministic segment
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eaf1d45a/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 630131e..dd49fcb 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
@@ -808,14 +808,16 @@ class ReadersAndUpdates {
return true;
}
- /** Returns a reader for merge, with the latest doc values updates and deletions. */
- synchronized SegmentReader getReaderForMerge(IOContext context) throws IOException {
-
+ synchronized public void setIsMerging() {
// 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();
+ }
+
+ /** Returns a reader for merge, with the latest doc values updates and deletions. */
+ synchronized SegmentReader getReaderForMerge(IOContext context) throws IOException {
// 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,