LUCENE-8217: Remove IndexFileDeleter#decRefWhileHandlingExceptions

This method is a duplicate of IDF#decRef(...) and hides exceptions
from the caller. This change removes this method and replaces it with
it's counterpart that escalades the exception.
This commit is contained in:
Simon Willnauer 2018-03-20 12:41:47 +01:00
parent f664896d1f
commit d4e69c5cd8
3 changed files with 34 additions and 82 deletions

View File

@ -592,26 +592,6 @@ final class IndexFileDeleter implements Closeable {
}
}
/** Decrefs all provided files, ignoring any exceptions hit; call this if
* you are already handling an exception. */
void decRefWhileHandlingException(Collection<String> files) {
assert locked();
Set<String> toDelete = new HashSet<>();
for(final String file : files) {
try {
if (decRef(file)) {
toDelete.add(file);
}
} catch (Throwable t) {
}
}
try {
deleteFiles(toDelete);
} catch (Throwable t) {
}
}
/** Returns true if the file should now be deleted. */
private boolean decRef(String fileName) {
assert locked();

View File

@ -3407,27 +3407,29 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
maybeCloseOnTragicEvent();
}
boolean success = false;
try {
if (anyChanges) {
maybeMerge.set(true);
}
startCommit(toCommit);
success = true;
if (pendingCommit == null) {
return -1;
} else {
return seqNo;
}
} finally {
if (!success) {
synchronized (this) {
if (filesToCommit != null) {
deleter.decRefWhileHandlingException(filesToCommit);
} catch (Throwable t) {
synchronized (this) {
if (filesToCommit != null) {
try {
deleter.decRef(filesToCommit);
} catch (Throwable t1) {
t.addSuppressed(t1);
} finally {
filesToCommit = null;
}
}
}
throw t;
}
}
}
@ -3566,7 +3568,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
private final void finishCommit() throws IOException {
boolean commitCompleted = false;
boolean finished = false;
String committedSegmentsFileName = null;
try {
@ -3578,7 +3579,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
if (pendingCommit != null) {
try {
final Collection<String> commitFiles = this.filesToCommit;
try (Closeable finalizer = () -> deleter.decRef(commitFiles)) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "commit: pendingCommit != null");
@ -3603,21 +3605,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
lastCommitChangeCount = pendingCommitChangeCount;
rollbackSegments = pendingCommit.createBackupSegmentInfos();
finished = true;
} finally {
notifyAll();
try {
if (finished) {
// all is good
deleter.decRef(filesToCommit);
} else if (commitCompleted == false) {
// exc happened in finishCommit: not a tragedy
deleter.decRefWhileHandlingException(filesToCommit);
}
} finally {
pendingCommit = null;
filesToCommit = null;
}
pendingCommit = null;
this.filesToCommit = null;
}
} else {
assert filesToCommit == null;
@ -4822,7 +4813,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
testPoint("midStartCommit2");
synchronized(this) {
synchronized (this) {
assert pendingCommit == null;
@ -4861,7 +4852,23 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
testPoint("midStartCommitSuccess");
} catch (Throwable t) {
synchronized(this) {
if (!pendingCommitSet) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "hit exception committing segments file");
}
try {
// Hit exception
deleter.decRef(filesToCommit);
} catch (Throwable t1) {
t.addSuppressed(t1);
} finally {
filesToCommit = null;
}
}
}
throw t;
} finally {
synchronized(this) {
// Have our master segmentInfos record the
@ -4869,16 +4876,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// on error or success so we don't
// double-write a segments_N file.
segmentInfos.updateGeneration(toSync);
if (!pendingCommitSet) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "hit exception committing segments file");
}
// Hit exception
deleter.decRefWhileHandlingException(filesToCommit);
filesToCommit = null;
}
}
}
} catch (VirtualMachineError tragedy) {

View File

@ -82,17 +82,13 @@ public final class IOUtils {
*/
public static void close(Iterable<? extends Closeable> objects) throws IOException {
Throwable th = null;
for (Closeable object : objects) {
try {
if (object != null) {
object.close();
}
} catch (Throwable t) {
addSuppressed(th, t);
if (th == null) {
th = t;
}
th = useOrSuppress(th, t);
}
}
@ -141,16 +137,6 @@ public final class IOUtils {
}
}
/** adds a Throwable to the list of suppressed Exceptions of the first Throwable
* @param exception this exception should get the suppressed one added
* @param suppressed the suppressed exception
*/
private static void addSuppressed(Throwable exception, Throwable suppressed) {
if (exception != null && suppressed != null) {
exception.addSuppressed(suppressed);
}
}
/**
* Wrapping the given {@link InputStream} in a reader using a {@link CharsetDecoder}.
* Unlike Java's defaults this reader will throw an exception if your it detects
@ -237,10 +223,7 @@ public final class IOUtils {
try {
dir.deleteFile(name);
} catch (Throwable t) {
addSuppressed(th, t);
if (th == null) {
th = t;
}
th = useOrSuppress(th, t);
}
}
}
@ -250,10 +233,6 @@ public final class IOUtils {
}
}
public static void deleteFiles(Directory dir, String... files) throws IOException {
deleteFiles(dir, Arrays.asList(files));
}
/**
* Deletes all given files, suppressing all thrown IOExceptions.
* <p>
@ -304,17 +283,13 @@ public final class IOUtils {
*/
public static void deleteFilesIfExist(Collection<? extends Path> files) throws IOException {
Throwable th = null;
for (Path file : files) {
try {
if (file != null) {
Files.deleteIfExists(file);
}
} catch (Throwable t) {
addSuppressed(th, t);
if (th == null) {
th = t;
}
th = useOrSuppress(th, t);
}
}