mirror of https://github.com/apache/lucene.git
LUCENE-8215: Fix several fragile exception handling places in o.a.l.index
Several places in the index package don't handle exceptions well or ignores them. This change adds some utility methods and cuts over to make use of try/with blocks to simplify exception handling.
This commit is contained in:
parent
3048e5da22
commit
2e35ef2b3d
|
@ -19,6 +19,7 @@ package org.apache.lucene.index;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
|
@ -302,7 +303,6 @@ class BufferedUpdatesStream implements Accountable {
|
||||||
ensureOpen();
|
ensureOpen();
|
||||||
|
|
||||||
List<SegmentState> segStates = new ArrayList<>();
|
List<SegmentState> segStates = new ArrayList<>();
|
||||||
boolean success = false;
|
|
||||||
try {
|
try {
|
||||||
for (SegmentCommitInfo info : infos) {
|
for (SegmentCommitInfo info : infos) {
|
||||||
if (info.getBufferedDeletesGen() <= delGen && alreadySeenSegments.contains(info) == false) {
|
if (info.getBufferedDeletesGen() <= delGen && alreadySeenSegments.contains(info) == false) {
|
||||||
|
@ -310,17 +310,15 @@ class BufferedUpdatesStream implements Accountable {
|
||||||
alreadySeenSegments.add(info);
|
alreadySeenSegments.add(info);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
success = true;
|
} catch (Throwable t) {
|
||||||
} finally {
|
|
||||||
if (success == false) {
|
|
||||||
for(SegmentState segState : segStates) {
|
for(SegmentState segState : segStates) {
|
||||||
try {
|
try {
|
||||||
segState.finish(pool);
|
segState.finish(pool);
|
||||||
} catch (Throwable th) {
|
} catch (Throwable th) {
|
||||||
// suppress so we keep throwing original exc
|
t.addSuppressed(th);
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
throw t;
|
||||||
}
|
}
|
||||||
|
|
||||||
return segStates.toArray(new SegmentState[0]);
|
return segStates.toArray(new SegmentState[0]);
|
||||||
|
@ -328,13 +326,10 @@ class BufferedUpdatesStream implements Accountable {
|
||||||
|
|
||||||
/** Close segment states previously opened with openSegmentStates. */
|
/** Close segment states previously opened with openSegmentStates. */
|
||||||
public ApplyDeletesResult closeSegmentStates(IndexWriter.ReaderPool pool, SegmentState[] segStates, boolean success) throws IOException {
|
public ApplyDeletesResult closeSegmentStates(IndexWriter.ReaderPool pool, SegmentState[] segStates, boolean success) throws IOException {
|
||||||
int count = segStates.length;
|
|
||||||
Throwable firstExc = null;
|
|
||||||
List<SegmentCommitInfo> allDeleted = null;
|
List<SegmentCommitInfo> allDeleted = null;
|
||||||
long totDelCount = 0;
|
long totDelCount = 0;
|
||||||
|
final List<SegmentState> segmentStates = Arrays.asList(segStates);
|
||||||
for (int j=0;j<count;j++) {
|
for (SegmentState segState : segmentStates) {
|
||||||
SegmentState segState = segStates[j];
|
|
||||||
if (success) {
|
if (success) {
|
||||||
totDelCount += segState.rld.getPendingDeleteCount() - segState.startDelCount;
|
totDelCount += segState.rld.getPendingDeleteCount() - segState.startDelCount;
|
||||||
int fullDelCount = segState.rld.info.getDelCount() + segState.rld.getPendingDeleteCount();
|
int fullDelCount = segState.rld.info.getDelCount() + segState.rld.getPendingDeleteCount();
|
||||||
|
@ -346,21 +341,8 @@ class BufferedUpdatesStream implements Accountable {
|
||||||
allDeleted.add(segState.reader.getSegmentInfo());
|
allDeleted.add(segState.reader.getSegmentInfo());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
try {
|
|
||||||
segStates[j].finish(pool);
|
|
||||||
} catch (Throwable th) {
|
|
||||||
if (firstExc == null) {
|
|
||||||
firstExc = th;
|
|
||||||
}
|
}
|
||||||
}
|
IOUtils.applyToAll(segmentStates, s -> s.finish(pool));
|
||||||
}
|
|
||||||
|
|
||||||
if (success) {
|
|
||||||
if (firstExc != null) {
|
|
||||||
throw IOUtils.rethrowAlways(firstExc);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (infoStream.isEnabled("BD")) {
|
if (infoStream.isEnabled("BD")) {
|
||||||
infoStream.message("BD", "closeSegmentStates: " + totDelCount + " new deleted documents; pool " + updates.size() + " packets; bytesUsed=" + pool.ramBytesUsed());
|
infoStream.message("BD", "closeSegmentStates: " + totDelCount + " new deleted documents; pool " + updates.size() + " packets; bytesUsed=" + pool.ramBytesUsed());
|
||||||
}
|
}
|
||||||
|
|
|
@ -418,7 +418,8 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
||||||
if (value != null) {
|
if (value != null) {
|
||||||
spins = Boolean.parseBoolean(value);
|
spins = Boolean.parseBoolean(value);
|
||||||
}
|
}
|
||||||
} catch (Throwable ignored) {
|
} catch (Exception ignored) {
|
||||||
|
// that's fine we might hit a SecurityException etc. here just continue
|
||||||
}
|
}
|
||||||
setDefaultMaxMergesAndThreads(spins);
|
setDefaultMaxMergesAndThreads(spins);
|
||||||
if (verbose()) {
|
if (verbose()) {
|
||||||
|
|
|
@ -636,8 +636,8 @@ final class DocumentsWriterFlushControl implements Accountable {
|
||||||
try {
|
try {
|
||||||
documentsWriter.subtractFlushedNumDocs(dwpt.getNumDocsInRAM());
|
documentsWriter.subtractFlushedNumDocs(dwpt.getNumDocsInRAM());
|
||||||
dwpt.abort();
|
dwpt.abort();
|
||||||
} catch (Throwable ex) {
|
} catch (Exception ex) {
|
||||||
// ignore - keep on aborting the flush queue
|
// that's fine we just abort everything here this is best effort
|
||||||
} finally {
|
} finally {
|
||||||
doAfterFlush(dwpt);
|
doAfterFlush(dwpt);
|
||||||
}
|
}
|
||||||
|
@ -647,8 +647,8 @@ final class DocumentsWriterFlushControl implements Accountable {
|
||||||
flushingWriters.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
|
flushingWriters.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
|
||||||
documentsWriter.subtractFlushedNumDocs(blockedFlush.dwpt.getNumDocsInRAM());
|
documentsWriter.subtractFlushedNumDocs(blockedFlush.dwpt.getNumDocsInRAM());
|
||||||
blockedFlush.dwpt.abort();
|
blockedFlush.dwpt.abort();
|
||||||
} catch (Throwable ex) {
|
} catch (Exception ex) {
|
||||||
// ignore - keep on aborting the blocked queue
|
// that's fine we just abort everything here this is best effort
|
||||||
} finally {
|
} finally {
|
||||||
doAfterFlush(blockedFlush.dwpt);
|
doAfterFlush(blockedFlush.dwpt);
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,6 +16,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.lucene.index;
|
package org.apache.lucene.index;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
@ -26,6 +27,7 @@ import java.util.Locale;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
||||||
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
||||||
|
@ -40,7 +42,6 @@ import org.apache.lucene.store.RAMOutputStream;
|
||||||
import org.apache.lucene.util.ArrayUtil;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
import org.apache.lucene.util.Bits;
|
import org.apache.lucene.util.Bits;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.IOUtils;
|
|
||||||
import org.apache.lucene.util.InfoStream;
|
import org.apache.lucene.util.InfoStream;
|
||||||
import org.apache.lucene.util.RamUsageEstimator;
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
|
|
||||||
|
@ -247,7 +248,7 @@ class FrozenBufferedUpdates {
|
||||||
/** Translates a frozen packet of delete term/query, or doc values
|
/** Translates a frozen packet of delete term/query, or doc values
|
||||||
* updates, into their actual docIDs in the index, and applies the change. This is a heavy
|
* updates, into their actual docIDs in the index, and applies the change. This is a heavy
|
||||||
* operation and is done concurrently by incoming indexing threads. */
|
* operation and is done concurrently by incoming indexing threads. */
|
||||||
|
@SuppressWarnings("try")
|
||||||
public synchronized void apply(IndexWriter writer) throws IOException {
|
public synchronized void apply(IndexWriter writer) throws IOException {
|
||||||
if (applied.getCount() == 0) {
|
if (applied.getCount() == 0) {
|
||||||
// already done
|
// already done
|
||||||
|
@ -319,14 +320,12 @@ class FrozenBufferedUpdates {
|
||||||
writer.deleter.incRef(delFiles);
|
writer.deleter.incRef(delFiles);
|
||||||
}
|
}
|
||||||
|
|
||||||
boolean success = false;
|
AtomicBoolean success = new AtomicBoolean();
|
||||||
long delCount;
|
long delCount;
|
||||||
try {
|
try (Closeable finalizer = () -> finishApply(writer, segStates, success.get(), delFiles)) {
|
||||||
// don't hold IW monitor lock here so threads are free concurrently resolve deletes/updates:
|
// don't hold IW monitor lock here so threads are free concurrently resolve deletes/updates:
|
||||||
delCount = apply(segStates);
|
delCount = apply(segStates);
|
||||||
success = true;
|
success.set(true);
|
||||||
} finally {
|
|
||||||
finishApply(writer, segStates, success, delFiles);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Since we jus resolved some more deletes/updates, now is a good time to write them:
|
// Since we jus resolved some more deletes/updates, now is a good time to write them:
|
||||||
|
@ -722,7 +721,6 @@ class FrozenBufferedUpdates {
|
||||||
// We apply segment-private deletes on flush:
|
// We apply segment-private deletes on flush:
|
||||||
assert privateSegment == null;
|
assert privateSegment == null;
|
||||||
|
|
||||||
try {
|
|
||||||
long startNS = System.nanoTime();
|
long startNS = System.nanoTime();
|
||||||
|
|
||||||
long delCount = 0;
|
long delCount = 0;
|
||||||
|
@ -815,10 +813,6 @@ class FrozenBufferedUpdates {
|
||||||
}
|
}
|
||||||
|
|
||||||
return delCount;
|
return delCount;
|
||||||
|
|
||||||
} catch (Throwable t) {
|
|
||||||
throw IOUtils.rethrowAlways(t);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setDelGen(long delGen) {
|
public void setDelGen(long delGen) {
|
||||||
|
|
|
@ -356,10 +356,6 @@ final class IndexFileDeleter implements Closeable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public SegmentInfos getLastSegmentInfos() {
|
|
||||||
return lastSegmentInfos;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Remove the CommitPoints in the commitsToDelete List by
|
* Remove the CommitPoints in the commitsToDelete List by
|
||||||
* DecRef'ing all files from each SegmentInfos.
|
* DecRef'ing all files from each SegmentInfos.
|
||||||
|
@ -381,9 +377,7 @@ final class IndexFileDeleter implements Closeable {
|
||||||
try {
|
try {
|
||||||
decRef(commit.files);
|
decRef(commit.files);
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
if (firstThrowable == null) {
|
firstThrowable = IOUtils.useOrSuppress(firstThrowable, t);
|
||||||
firstThrowable = t;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
commitsToDelete.clear();
|
commitsToDelete.clear();
|
||||||
|
@ -583,20 +577,14 @@ final class IndexFileDeleter implements Closeable {
|
||||||
toDelete.add(file);
|
toDelete.add(file);
|
||||||
}
|
}
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
if (firstThrowable == null) {
|
firstThrowable = IOUtils.useOrSuppress(firstThrowable, t);
|
||||||
// Save first exception and throw it in the end, but be sure to finish decRef all files
|
|
||||||
firstThrowable = t;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
deleteFiles(toDelete);
|
deleteFiles(toDelete);
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
if (firstThrowable == null) {
|
firstThrowable = IOUtils.useOrSuppress(firstThrowable, t);
|
||||||
// Save first exception and throw it in the end, but be sure to finish decRef all files
|
|
||||||
firstThrowable = t;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (firstThrowable != null) {
|
if (firstThrowable != null) {
|
||||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.lucene.document.Document;
|
||||||
import org.apache.lucene.document.DocumentStoredFieldVisitor;
|
import org.apache.lucene.document.DocumentStoredFieldVisitor;
|
||||||
import org.apache.lucene.store.AlreadyClosedException;
|
import org.apache.lucene.store.AlreadyClosedException;
|
||||||
import org.apache.lucene.util.Bits; // javadocs
|
import org.apache.lucene.util.Bits; // javadocs
|
||||||
import org.apache.lucene.util.IOUtils;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
IndexReader is an abstract class, providing an interface for accessing a
|
IndexReader is an abstract class, providing an interface for accessing a
|
||||||
|
@ -142,14 +141,11 @@ public abstract class IndexReader implements Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
// overridden by StandardDirectoryReader and SegmentReader
|
// overridden by StandardDirectoryReader and SegmentReader
|
||||||
void notifyReaderClosedListeners(Throwable th) throws IOException {
|
void notifyReaderClosedListeners() throws IOException {
|
||||||
// nothing to notify in the base impl, just rethrow
|
// nothing to notify in the base impl
|
||||||
if (th != null) {
|
|
||||||
throw IOUtils.rethrowAlways(th);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void reportCloseToParentReaders() {
|
private void reportCloseToParentReaders() throws IOException {
|
||||||
synchronized (parentReaders) {
|
synchronized (parentReaders) {
|
||||||
for (IndexReader parent : parentReaders) {
|
for (IndexReader parent : parentReaders) {
|
||||||
parent.closedByChild = true;
|
parent.closedByChild = true;
|
||||||
|
@ -232,6 +228,7 @@ public abstract class IndexReader implements Closeable {
|
||||||
*
|
*
|
||||||
* @see #incRef
|
* @see #incRef
|
||||||
*/
|
*/
|
||||||
|
@SuppressWarnings("try")
|
||||||
public final void decRef() throws IOException {
|
public final void decRef() throws IOException {
|
||||||
// only check refcount here (don't call ensureOpen()), so we can
|
// only check refcount here (don't call ensureOpen()), so we can
|
||||||
// still close the reader if it was made invalid by a child:
|
// still close the reader if it was made invalid by a child:
|
||||||
|
@ -242,17 +239,9 @@ public abstract class IndexReader implements Closeable {
|
||||||
final int rc = refCount.decrementAndGet();
|
final int rc = refCount.decrementAndGet();
|
||||||
if (rc == 0) {
|
if (rc == 0) {
|
||||||
closed = true;
|
closed = true;
|
||||||
Throwable throwable = null;
|
try (Closeable finalizer = this::reportCloseToParentReaders;
|
||||||
try {
|
Closeable finalizer1 = this::notifyReaderClosedListeners) {
|
||||||
doClose();
|
doClose();
|
||||||
} catch (Throwable th) {
|
|
||||||
throwable = th;
|
|
||||||
} finally {
|
|
||||||
try {
|
|
||||||
reportCloseToParentReaders();
|
|
||||||
} finally {
|
|
||||||
notifyReaderClosedListeners(throwable);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
} else if (rc < 0) {
|
} else if (rc < 0) {
|
||||||
throw new IllegalStateException("too many decRef calls: refCount is " + rc + " after decrement");
|
throw new IllegalStateException("too many decRef calls: refCount is " + rc + " after decrement");
|
||||||
|
|
|
@ -2586,6 +2586,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
* @return The <a href="#sequence_number">sequence number</a>
|
* @return The <a href="#sequence_number">sequence number</a>
|
||||||
* for this operation
|
* for this operation
|
||||||
*/
|
*/
|
||||||
|
@SuppressWarnings("try")
|
||||||
public long deleteAll() throws IOException {
|
public long deleteAll() throws IOException {
|
||||||
ensureOpen();
|
ensureOpen();
|
||||||
// Remove any buffered docs
|
// Remove any buffered docs
|
||||||
|
@ -2604,7 +2605,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
*/
|
*/
|
||||||
try {
|
try {
|
||||||
synchronized (fullFlushLock) {
|
synchronized (fullFlushLock) {
|
||||||
try (Closeable release = docWriter.lockAndAbortAll(this)) {
|
try (Closeable finalizer = docWriter.lockAndAbortAll(this)) {
|
||||||
processEvents(false);
|
processEvents(false);
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
try {
|
try {
|
||||||
|
@ -3957,6 +3958,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
return mergedDeletesAndUpdates;
|
return mergedDeletesAndUpdates;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("try")
|
||||||
synchronized private boolean commitMerge(MergePolicy.OneMerge merge, MergeState mergeState) throws IOException {
|
synchronized private boolean commitMerge(MergePolicy.OneMerge merge, MergeState mergeState) throws IOException {
|
||||||
|
|
||||||
testPoint("startCommitMerge");
|
testPoint("startCommitMerge");
|
||||||
|
@ -4069,22 +4071,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
deleteNewFiles(merge.info.files());
|
deleteNewFiles(merge.info.files());
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try (Closeable finalizer = this::checkpoint) {
|
||||||
// Must close before checkpoint, otherwise IFD won't be
|
// Must close before checkpoint, otherwise IFD won't be
|
||||||
// able to delete the held-open files from the merge
|
// able to delete the held-open files from the merge
|
||||||
// readers:
|
// readers:
|
||||||
closeMergeReaders(merge, false);
|
closeMergeReaders(merge, false);
|
||||||
checkpoint();
|
|
||||||
} catch (Throwable t) {
|
|
||||||
// Must note the change to segmentInfos so any commits
|
|
||||||
// in-flight don't lose it (IFD will incRef/protect the
|
|
||||||
// new files we created):
|
|
||||||
try {
|
|
||||||
checkpoint();
|
|
||||||
} catch (Throwable t1) {
|
|
||||||
t.addSuppressed(t1);
|
|
||||||
}
|
|
||||||
throw t;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
|
@ -4403,16 +4394,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
runningMerges.remove(merge);
|
runningMerges.remove(merge);
|
||||||
}
|
}
|
||||||
|
|
||||||
private final synchronized void closeMergeReaders(MergePolicy.OneMerge merge, boolean suppressExceptions) throws IOException {
|
@SuppressWarnings("try")
|
||||||
final int numSegments = merge.readers.size();
|
private synchronized void closeMergeReaders(MergePolicy.OneMerge merge, boolean suppressExceptions) throws IOException {
|
||||||
Throwable th = null;
|
final boolean drop = suppressExceptions == false;
|
||||||
|
try (Closeable finalizer = merge::mergeFinished) {
|
||||||
boolean drop = suppressExceptions == false;
|
IOUtils.applyToAll(merge.readers, sr -> {
|
||||||
|
|
||||||
for (int i = 0; i < numSegments; i++) {
|
|
||||||
final SegmentReader sr = merge.readers.get(i);
|
|
||||||
if (sr != null) {
|
|
||||||
try {
|
|
||||||
final ReadersAndUpdates rld = readerPool.get(sr.getSegmentInfo(), false);
|
final ReadersAndUpdates rld = readerPool.get(sr.getSegmentInfo(), false);
|
||||||
// We still hold a ref so it should not have been removed:
|
// We still hold a ref so it should not have been removed:
|
||||||
assert rld != null;
|
assert rld != null;
|
||||||
|
@ -4426,22 +4412,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
if (drop) {
|
if (drop) {
|
||||||
readerPool.drop(rld.info);
|
readerPool.drop(rld.info);
|
||||||
}
|
}
|
||||||
} catch (Throwable t) {
|
});
|
||||||
th = IOUtils.useOrSuppress(th, t);
|
} finally {
|
||||||
}
|
Collections.fill(merge.readers, null);
|
||||||
merge.readers.set(i, null);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
try {
|
|
||||||
merge.mergeFinished();
|
|
||||||
} catch (Throwable t) {
|
|
||||||
th = IOUtils.useOrSuppress(th, t);
|
|
||||||
}
|
|
||||||
|
|
||||||
// If any error occurred, throw it.
|
|
||||||
if (!suppressExceptions && th != null) {
|
|
||||||
throw IOUtils.rethrowAlways(th);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -845,7 +845,7 @@ class ReadersAndUpdates {
|
||||||
success = true;
|
success = true;
|
||||||
} finally {
|
} finally {
|
||||||
if (success == false) {
|
if (success == false) {
|
||||||
newReader.decRef();
|
newReader.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
reader = newReader;
|
reader = newReader;
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
package org.apache.lucene.index;
|
package org.apache.lucene.index;
|
||||||
|
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
import java.io.EOFException;
|
import java.io.EOFException;
|
||||||
import java.io.FileNotFoundException;
|
import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -161,16 +162,13 @@ final class SegmentCoreReaders {
|
||||||
throw new AlreadyClosedException("SegmentCoreReaders is already closed");
|
throw new AlreadyClosedException("SegmentCoreReaders is already closed");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("try")
|
||||||
void decRef() throws IOException {
|
void decRef() throws IOException {
|
||||||
if (ref.decrementAndGet() == 0) {
|
if (ref.decrementAndGet() == 0) {
|
||||||
Throwable th = null;
|
Throwable th = null;
|
||||||
try {
|
try (Closeable finalizer = this::notifyCoreClosedListeners){
|
||||||
IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
|
IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
|
||||||
cfsReader, normsProducer, pointsReader);
|
cfsReader, normsProducer, pointsReader);
|
||||||
} catch (Throwable throwable) {
|
|
||||||
th = throwable;
|
|
||||||
} finally {
|
|
||||||
notifyCoreClosedListeners(th);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -193,25 +191,9 @@ final class SegmentCoreReaders {
|
||||||
return cacheHelper;
|
return cacheHelper;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void notifyCoreClosedListeners(Throwable th) throws IOException {
|
private void notifyCoreClosedListeners() throws IOException {
|
||||||
synchronized(coreClosedListeners) {
|
synchronized(coreClosedListeners) {
|
||||||
for (IndexReader.ClosedListener listener : coreClosedListeners) {
|
IOUtils.applyToAll(coreClosedListeners, l -> l.onClose(cacheHelper.getKey()));
|
||||||
// SegmentReader uses our instance as its
|
|
||||||
// coreCacheKey:
|
|
||||||
try {
|
|
||||||
listener.onClose(cacheHelper.getKey());
|
|
||||||
} catch (Throwable t) {
|
|
||||||
if (th == null) {
|
|
||||||
th = t;
|
|
||||||
} else {
|
|
||||||
th.addSuppressed(t);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (th != null) {
|
|
||||||
throw IOUtils.rethrowAlways(th);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -78,21 +78,10 @@ final class SegmentDocValues {
|
||||||
* generations.
|
* generations.
|
||||||
*/
|
*/
|
||||||
synchronized void decRef(List<Long> dvProducersGens) throws IOException {
|
synchronized void decRef(List<Long> dvProducersGens) throws IOException {
|
||||||
Throwable t = null;
|
IOUtils.applyToAll(dvProducersGens, gen -> {
|
||||||
for (Long gen : dvProducersGens) {
|
|
||||||
RefCount<DocValuesProducer> dvp = genDVProducers.get(gen);
|
RefCount<DocValuesProducer> dvp = genDVProducers.get(gen);
|
||||||
assert dvp != null : "gen=" + gen;
|
assert dvp != null : "gen=" + gen;
|
||||||
try {
|
|
||||||
dvp.decRef();
|
dvp.decRef();
|
||||||
} catch (Throwable th) {
|
});
|
||||||
if (t == null) {
|
|
||||||
t = th;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (t != null) {
|
|
||||||
throw IOUtils.rethrowAlways(t);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -55,7 +55,6 @@ class SegmentDocValuesProducer extends DocValuesProducer {
|
||||||
* @param segDocValues producer map
|
* @param segDocValues producer map
|
||||||
*/
|
*/
|
||||||
SegmentDocValuesProducer(SegmentCommitInfo si, Directory dir, FieldInfos coreInfos, FieldInfos allInfos, SegmentDocValues segDocValues) throws IOException {
|
SegmentDocValuesProducer(SegmentCommitInfo si, Directory dir, FieldInfos coreInfos, FieldInfos allInfos, SegmentDocValues segDocValues) throws IOException {
|
||||||
boolean success = false;
|
|
||||||
try {
|
try {
|
||||||
DocValuesProducer baseProducer = null;
|
DocValuesProducer baseProducer = null;
|
||||||
for (FieldInfo fi : allInfos) {
|
for (FieldInfo fi : allInfos) {
|
||||||
|
@ -80,15 +79,13 @@ class SegmentDocValuesProducer extends DocValuesProducer {
|
||||||
dvProducersByField.put(fi.name, dvp);
|
dvProducersByField.put(fi.name, dvp);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
success = true;
|
} catch (Throwable t) {
|
||||||
} finally {
|
|
||||||
if (success == false) {
|
|
||||||
try {
|
try {
|
||||||
segDocValues.decRef(dvGens);
|
segDocValues.decRef(dvGens);
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t1) {
|
||||||
// Ignore so we keep throwing first exception
|
t.addSuppressed(t1);
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
throw t;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -297,23 +297,9 @@ public final class SegmentReader extends CodecReader {
|
||||||
private final Set<ClosedListener> readerClosedListeners = new CopyOnWriteArraySet<>();
|
private final Set<ClosedListener> readerClosedListeners = new CopyOnWriteArraySet<>();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void notifyReaderClosedListeners(Throwable th) throws IOException {
|
void notifyReaderClosedListeners() throws IOException {
|
||||||
synchronized(readerClosedListeners) {
|
synchronized(readerClosedListeners) {
|
||||||
for(ClosedListener listener : readerClosedListeners) {
|
IOUtils.applyToAll(readerClosedListeners, l -> l.onClose(readerCacheHelper.getKey()));
|
||||||
try {
|
|
||||||
listener.onClose(readerCacheHelper.getKey());
|
|
||||||
} catch (Throwable t) {
|
|
||||||
if (th == null) {
|
|
||||||
th = t;
|
|
||||||
} else {
|
|
||||||
th.addSuppressed(t);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (th != null) {
|
|
||||||
IOUtils.rethrowAlways(th);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
package org.apache.lucene.index;
|
package org.apache.lucene.index;
|
||||||
|
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
@ -91,7 +92,6 @@ public final class StandardDirectoryReader extends DirectoryReader {
|
||||||
|
|
||||||
final SegmentInfos segmentInfos = infos.clone();
|
final SegmentInfos segmentInfos = infos.clone();
|
||||||
int infosUpto = 0;
|
int infosUpto = 0;
|
||||||
boolean success = false;
|
|
||||||
try {
|
try {
|
||||||
for (int i = 0; i < numSegments; i++) {
|
for (int i = 0; i < numSegments; i++) {
|
||||||
// NOTE: important that we use infos not
|
// NOTE: important that we use infos not
|
||||||
|
@ -121,19 +121,14 @@ public final class StandardDirectoryReader extends DirectoryReader {
|
||||||
StandardDirectoryReader result = new StandardDirectoryReader(dir,
|
StandardDirectoryReader result = new StandardDirectoryReader(dir,
|
||||||
readers.toArray(new SegmentReader[readers.size()]), writer,
|
readers.toArray(new SegmentReader[readers.size()]), writer,
|
||||||
segmentInfos, applyAllDeletes, writeAllDeletes);
|
segmentInfos, applyAllDeletes, writeAllDeletes);
|
||||||
success = true;
|
|
||||||
return result;
|
return result;
|
||||||
} finally {
|
} catch (Throwable t) {
|
||||||
if (!success) {
|
|
||||||
for (SegmentReader r : readers) {
|
|
||||||
try {
|
try {
|
||||||
r.decRef();
|
IOUtils.applyToAll(readers, SegmentReader::decRef);
|
||||||
} catch (Throwable th) {
|
} catch (Throwable t1) {
|
||||||
// ignore any exception that is thrown here to not mask any original
|
t.addSuppressed(t1);
|
||||||
// exception.
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
throw t;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -365,19 +360,9 @@ public final class StandardDirectoryReader extends DirectoryReader {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@SuppressWarnings("try")
|
||||||
protected void doClose() throws IOException {
|
protected void doClose() throws IOException {
|
||||||
Throwable firstExc = null;
|
Closeable decRefDeleter = () -> {
|
||||||
for (final LeafReader r : getSequentialSubReaders()) {
|
|
||||||
// try to close each reader, even if an exception is thrown
|
|
||||||
try {
|
|
||||||
r.decRef();
|
|
||||||
} catch (Throwable t) {
|
|
||||||
if (firstExc == null) {
|
|
||||||
firstExc = t;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (writer != null) {
|
if (writer != null) {
|
||||||
try {
|
try {
|
||||||
writer.decRefDeleter(segmentInfos);
|
writer.decRefDeleter(segmentInfos);
|
||||||
|
@ -389,10 +374,11 @@ public final class StandardDirectoryReader extends DirectoryReader {
|
||||||
// index, it will delete them.
|
// index, it will delete them.
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
};
|
||||||
// throw the first exception
|
try (Closeable finalizer = decRefDeleter) {
|
||||||
if (firstExc != null) {
|
// try to close each reader, even if an exception is thrown
|
||||||
throw IOUtils.rethrowAlways(firstExc);
|
final List<? extends LeafReader> sequentialSubReaders = getSequentialSubReaders();
|
||||||
|
IOUtils.applyToAll(sequentialSubReaders, LeafReader::decRef);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -493,23 +479,9 @@ public final class StandardDirectoryReader extends DirectoryReader {
|
||||||
};
|
};
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void notifyReaderClosedListeners(Throwable th) throws IOException {
|
void notifyReaderClosedListeners() throws IOException {
|
||||||
synchronized(readerClosedListeners) {
|
synchronized(readerClosedListeners) {
|
||||||
for(ClosedListener listener : readerClosedListeners) {
|
IOUtils.applyToAll(readerClosedListeners, l -> l.onClose(cacheHelper.getKey()));
|
||||||
try {
|
|
||||||
listener.onClose(cacheHelper.getKey());
|
|
||||||
} catch (Throwable t) {
|
|
||||||
if (th == null) {
|
|
||||||
th = t;
|
|
||||||
} else {
|
|
||||||
th.addSuppressed(t);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (th != null) {
|
|
||||||
throw IOUtils.rethrowAlways(th);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.index;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A utility for executing 2-phase commit on several objects.
|
* A utility for executing 2-phase commit on several objects.
|
||||||
*
|
*
|
||||||
|
|
|
@ -16,7 +16,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.lucene.util;
|
package org.apache.lucene.util;
|
||||||
|
|
||||||
|
|
||||||
import java.io.BufferedReader;
|
import java.io.BufferedReader;
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -40,6 +39,7 @@ import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.LinkedHashMap;
|
import java.util.LinkedHashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
import org.apache.lucene.store.FSDirectory;
|
import org.apache.lucene.store.FSDirectory;
|
||||||
|
@ -638,4 +638,24 @@ public final class IOUtils {
|
||||||
}
|
}
|
||||||
return first;
|
return first;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Applies the consumer to all non-null elements in the collection even if an exception is thrown. The first exception
|
||||||
|
* thrown by the consumer is re-thrown and subsequent exceptions are suppressed.
|
||||||
|
*/
|
||||||
|
public static <T> void applyToAll(Collection<T> collection, IOConsumer<T> consumer) throws IOException {
|
||||||
|
IOUtils.close(collection.stream().filter(Objects::nonNull).map(t -> (Closeable) () -> consumer.accept(t))::iterator);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An IO operation with a single input.
|
||||||
|
* @see java.util.function.Consumer
|
||||||
|
*/
|
||||||
|
@FunctionalInterface
|
||||||
|
public interface IOConsumer<T> {
|
||||||
|
/**
|
||||||
|
* Performs this operation on the given argument.
|
||||||
|
*/
|
||||||
|
void accept(T input) throws IOException;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,6 +27,8 @@ import java.nio.file.LinkOption;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.nio.file.attribute.FileAttributeView;
|
import java.nio.file.attribute.FileAttributeView;
|
||||||
import java.nio.file.attribute.FileStoreAttributeView;
|
import java.nio.file.attribute.FileStoreAttributeView;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
@ -487,4 +489,19 @@ public class TestIOUtils extends LuceneTestCase {
|
||||||
// no exception
|
// no exception
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testApplyToAll() {
|
||||||
|
ArrayList<Integer> closed = new ArrayList<>();
|
||||||
|
RuntimeException runtimeException = expectThrows(RuntimeException.class, () ->
|
||||||
|
IOUtils.applyToAll(Arrays.asList(1, 2), i -> {
|
||||||
|
closed.add(i);
|
||||||
|
throw new RuntimeException("" + i);
|
||||||
|
}));
|
||||||
|
assertEquals("1", runtimeException.getMessage());
|
||||||
|
assertEquals(1, runtimeException.getSuppressed().length);
|
||||||
|
assertEquals("2", runtimeException.getSuppressed()[0].getMessage());
|
||||||
|
assertEquals(2, closed.size());
|
||||||
|
assertEquals(1, closed.get(0).intValue());
|
||||||
|
assertEquals(2, closed.get(1).intValue());
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -57,23 +57,9 @@ public final class OwnCacheKeyMultiReader extends MultiReader {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void notifyReaderClosedListeners(Throwable th) throws IOException {
|
void notifyReaderClosedListeners() throws IOException {
|
||||||
synchronized(readerClosedListeners) {
|
synchronized(readerClosedListeners) {
|
||||||
for(ClosedListener listener : readerClosedListeners) {
|
IOUtils.applyToAll(readerClosedListeners, l -> l.onClose(cacheHelper.getKey()));
|
||||||
try {
|
|
||||||
listener.onClose(cacheHelper.getKey());
|
|
||||||
} catch (Throwable t) {
|
|
||||||
if (th == null) {
|
|
||||||
th = t;
|
|
||||||
} else {
|
|
||||||
th.addSuppressed(t);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (th != null) {
|
|
||||||
throw IOUtils.rethrowAlways(th);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue