mirror of https://github.com/apache/lucene.git
LUCENE-6786: remove IndexFileDeleter.refresh(String), using only the global refresh() to delete newly created (unreferenced) files on rollback
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1702100 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
f538ed4e57
commit
54f63231ee
|
@ -417,36 +417,27 @@ final class IndexFileDeleter implements Closeable {
|
||||||
* is non-null, we will only delete files corresponding to
|
* is non-null, we will only delete files corresponding to
|
||||||
* that segment.
|
* that segment.
|
||||||
*/
|
*/
|
||||||
void refresh(String segmentName) throws IOException {
|
void refresh() throws IOException {
|
||||||
assert locked();
|
assert locked();
|
||||||
|
deletable.clear();
|
||||||
|
|
||||||
String[] files = directory.listAll();
|
String[] files = directory.listAll();
|
||||||
String segmentPrefix1;
|
|
||||||
String segmentPrefix2;
|
|
||||||
if (segmentName != null) {
|
|
||||||
segmentPrefix1 = segmentName + ".";
|
|
||||||
segmentPrefix2 = segmentName + "_";
|
|
||||||
} else {
|
|
||||||
segmentPrefix1 = null;
|
|
||||||
segmentPrefix2 = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
|
Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
|
||||||
|
|
||||||
for(int i=0;i<files.length;i++) {
|
for(int i=0;i<files.length;i++) {
|
||||||
String fileName = files[i];
|
String fileName = files[i];
|
||||||
m.reset(fileName);
|
m.reset(fileName);
|
||||||
if ((segmentName == null || fileName.startsWith(segmentPrefix1) || fileName.startsWith(segmentPrefix2)) &&
|
if (!fileName.endsWith("write.lock") &&
|
||||||
!fileName.endsWith("write.lock") &&
|
|
||||||
!refCounts.containsKey(fileName) &&
|
!refCounts.containsKey(fileName) &&
|
||||||
(m.matches() || fileName.startsWith(IndexFileNames.SEGMENTS)
|
(m.matches() || fileName.startsWith(IndexFileNames.SEGMENTS)
|
||||||
// we only try to clear out pending_segments_N during rollback(), because we don't ref-count it
|
// we only try to clear out pending_segments_N during rollback(), because we don't ref-count it
|
||||||
// TODO: this is sneaky, should we do this, or change TestIWExceptions? rollback closes anyway, and
|
// TODO: this is sneaky, should we do this, or change TestIWExceptions? rollback closes anyway, and
|
||||||
// any leftover file will be deleted/retried on next IW bootup anyway...
|
// any leftover file will be deleted/retried on next IW bootup anyway...
|
||||||
|| (segmentName == null && fileName.startsWith(IndexFileNames.PENDING_SEGMENTS)))) {
|
|| fileName.startsWith(IndexFileNames.PENDING_SEGMENTS))) {
|
||||||
// Unreferenced file, so remove it
|
// Unreferenced file, so remove it
|
||||||
if (infoStream.isEnabled("IFD")) {
|
if (infoStream.isEnabled("IFD")) {
|
||||||
infoStream.message("IFD", "refresh [prefix=" + segmentName + "]: removing newly created unreferenced file \"" + fileName + "\"");
|
infoStream.message("IFD", "refresh: removing newly created unreferenced file \"" + fileName + "\"");
|
||||||
}
|
}
|
||||||
deletable.add(fileName);
|
deletable.add(fileName);
|
||||||
}
|
}
|
||||||
|
@ -455,12 +446,6 @@ final class IndexFileDeleter implements Closeable {
|
||||||
deletePendingFiles();
|
deletePendingFiles();
|
||||||
}
|
}
|
||||||
|
|
||||||
void refresh() throws IOException {
|
|
||||||
assert locked();
|
|
||||||
deletable.clear();
|
|
||||||
refresh(null);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() {
|
public void close() {
|
||||||
// DecRef old files from the last checkpoint, if any:
|
// DecRef old files from the last checkpoint, if any:
|
||||||
|
|
|
@ -2606,17 +2606,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
boolean success = false;
|
merger.merge(); // merge 'em
|
||||||
try {
|
|
||||||
merger.merge(); // merge 'em
|
|
||||||
success = true;
|
|
||||||
} finally {
|
|
||||||
if (!success) {
|
|
||||||
synchronized(this) {
|
|
||||||
deleter.refresh(info.name);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
SegmentCommitInfo infoPerCommit = new SegmentCommitInfo(info, 0, -1L, -1L, -1L);
|
SegmentCommitInfo infoPerCommit = new SegmentCommitInfo(info, 0, -1L, -1L, -1L);
|
||||||
|
|
||||||
|
@ -2657,17 +2647,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
// creating CFS so that 1) .si isn't slurped into CFS,
|
// creating CFS so that 1) .si isn't slurped into CFS,
|
||||||
// and 2) .si reflects useCompoundFile=true change
|
// and 2) .si reflects useCompoundFile=true change
|
||||||
// above:
|
// above:
|
||||||
success = false;
|
codec.segmentInfoFormat().write(trackingDir, info, context);
|
||||||
try {
|
|
||||||
codec.segmentInfoFormat().write(trackingDir, info, context);
|
|
||||||
success = true;
|
|
||||||
} finally {
|
|
||||||
if (!success) {
|
|
||||||
synchronized(this) {
|
|
||||||
deleter.refresh(info.name);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
info.addFiles(trackingDir.getCreatedFiles());
|
info.addFiles(trackingDir.getCreatedFiles());
|
||||||
|
|
||||||
|
@ -3662,9 +3642,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
infoStream.message("IW", "hit exception during merge");
|
infoStream.message("IW", "hit exception during merge");
|
||||||
}
|
}
|
||||||
if (merge.info != null && !segmentInfos.contains(merge.info)) {
|
|
||||||
deleter.refresh(merge.info.info.name);
|
|
||||||
}
|
|
||||||
} else if (merge.rateLimiter.getAbort() == false && (merge.maxNumSegments != -1 || (!closed && !closing))) {
|
} else if (merge.rateLimiter.getAbort() == false && (merge.maxNumSegments != -1 || (!closed && !closing))) {
|
||||||
// This merge (and, generally, any change to the
|
// This merge (and, generally, any change to the
|
||||||
// segments) may now enable new merges, so we call
|
// segments) may now enable new merges, so we call
|
||||||
|
@ -4069,19 +4046,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
merge.mergeStartNS = System.nanoTime();
|
merge.mergeStartNS = System.nanoTime();
|
||||||
|
|
||||||
// This is where all the work happens:
|
// This is where all the work happens:
|
||||||
boolean success3 = false;
|
if (merger.shouldMerge()) {
|
||||||
try {
|
merger.merge();
|
||||||
if (merger.shouldMerge()) {
|
|
||||||
merger.merge();
|
|
||||||
}
|
|
||||||
success3 = true;
|
|
||||||
} finally {
|
|
||||||
if (!success3) {
|
|
||||||
synchronized(this) {
|
|
||||||
deleter.refresh(merge.info.info.name);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
MergeState mergeState = merger.mergeState;
|
MergeState mergeState = merger.mergeState;
|
||||||
assert mergeState.segmentInfo == merge.info.info;
|
assert mergeState.segmentInfo == merge.info.info;
|
||||||
merge.info.info.setFiles(new HashSet<>(dirWrapper.getCreatedFiles()));
|
merge.info.info.setFiles(new HashSet<>(dirWrapper.getCreatedFiles()));
|
||||||
|
@ -4688,10 +4656,19 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cleans up residuals from a segment that could not be entirely flushed due to an error
|
* Cleans up residuals from a segment that could not be entirely flushed due to an error
|
||||||
* @see IndexFileDeleter#refresh(String)
|
|
||||||
*/
|
*/
|
||||||
synchronized final void flushFailed(SegmentInfo info) throws IOException {
|
synchronized final void flushFailed(SegmentInfo info) throws IOException {
|
||||||
deleter.refresh(info.name);
|
// TODO: this really should be a tragic
|
||||||
|
Collection<String> files;
|
||||||
|
try {
|
||||||
|
files = info.files();
|
||||||
|
} catch (IllegalStateException ise) {
|
||||||
|
// OK
|
||||||
|
files = null;
|
||||||
|
}
|
||||||
|
if (files != null) {
|
||||||
|
deleter.deleteNewFiles(files);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
final int purge(boolean forced) throws IOException {
|
final int purge(boolean forced) throws IOException {
|
||||||
|
|
Loading…
Reference in New Issue