mirror of https://github.com/apache/lucene.git
don't allow publishFlushSegment after IW is closed
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1631152 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
67bff82e65
commit
f7cc185443
|
@ -25,9 +25,11 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Matcher;
|
||||
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
|
@ -78,7 +80,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
/* Files that we tried to delete but failed (likely
|
||||
* because they are open and we are running on Windows),
|
||||
* so we will retry them again later: */
|
||||
private List<String> deletable;
|
||||
private Set<String> deletable;
|
||||
|
||||
/* Reference count for all files in the index.
|
||||
* Counts how many existing commits reference a file.
|
||||
|
@ -359,7 +361,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
* Remove the CommitPoints in the commitsToDelete List by
|
||||
* DecRef'ing all files from each SegmentInfos.
|
||||
*/
|
||||
private void deleteCommits() throws IOException {
|
||||
private void deleteCommits() {
|
||||
|
||||
int size = commitsToDelete.size();
|
||||
|
||||
|
@ -384,7 +386,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
commitsToDelete.clear();
|
||||
|
||||
// NOTE: does nothing if firstThrowable is null
|
||||
IOUtils.reThrow(firstThrowable);
|
||||
IOUtils.reThrowUnchecked(firstThrowable);
|
||||
|
||||
// Now compact commits to remove deleted ones (preserving the sort):
|
||||
size = commits.size();
|
||||
|
@ -462,7 +464,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
public void close() {
|
||||
// DecRef old files from the last checkpoint, if any:
|
||||
assert locked();
|
||||
|
||||
|
@ -498,14 +500,12 @@ final class IndexFileDeleter implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
public void deletePendingFiles() throws IOException {
|
||||
public void deletePendingFiles() {
|
||||
assert locked();
|
||||
if (deletable != null) {
|
||||
List<String> oldDeletable = deletable;
|
||||
Set<String> oldDeletable = deletable;
|
||||
deletable = null;
|
||||
int size = oldDeletable.size();
|
||||
for(int i=0;i<size;i++) {
|
||||
String fileName = oldDeletable.get(i);
|
||||
for(String fileName : oldDeletable) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "delete pending file " + fileName);
|
||||
}
|
||||
|
@ -611,7 +611,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
}
|
||||
|
||||
/** Decrefs all provided files, even on exception; throws first exception hit, if any. */
|
||||
void decRef(Collection<String> files) throws IOException {
|
||||
void decRef(Collection<String> files) {
|
||||
assert locked();
|
||||
Throwable firstThrowable = null;
|
||||
for(final String file : files) {
|
||||
|
@ -626,12 +626,12 @@ final class IndexFileDeleter implements Closeable {
|
|||
}
|
||||
|
||||
// NOTE: does nothing if firstThrowable is null
|
||||
IOUtils.reThrow(firstThrowable);
|
||||
IOUtils.reThrowUnchecked(firstThrowable);
|
||||
}
|
||||
|
||||
/** Decrefs all provided files, ignoring any exceptions hit; call this if
|
||||
* you are already handling an exception. */
|
||||
void decRefWhileHandlingException(Collection<String> files) throws IOException {
|
||||
void decRefWhileHandlingException(Collection<String> files) {
|
||||
assert locked();
|
||||
for(final String file : files) {
|
||||
try {
|
||||
|
@ -641,7 +641,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
void decRef(String fileName) throws IOException {
|
||||
void decRef(String fileName) {
|
||||
assert locked();
|
||||
RefCount rc = getRefCount(fileName);
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
|
@ -679,6 +679,8 @@ final class IndexFileDeleter implements Closeable {
|
|||
RefCount rc;
|
||||
if (!refCounts.containsKey(fileName)) {
|
||||
rc = new RefCount(fileName);
|
||||
// We should never incRef a file we are already wanting to delete:
|
||||
assert deletable == null || deletable.contains(fileName) == false: "file \"" + fileName + "\" cannot be incRef'd: it's already pending delete";
|
||||
refCounts.put(fileName, rc);
|
||||
} else {
|
||||
rc = refCounts.get(fileName);
|
||||
|
@ -686,7 +688,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
return rc;
|
||||
}
|
||||
|
||||
void deleteFiles(List<String> files) throws IOException {
|
||||
void deleteFiles(List<String> files) {
|
||||
assert locked();
|
||||
for(final String file: files) {
|
||||
deleteFile(file);
|
||||
|
@ -695,7 +697,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
|
||||
/** Deletes the specified files, but only if they are new
|
||||
* (have not yet been incref'd). */
|
||||
void deleteNewFiles(Collection<String> files) throws IOException {
|
||||
void deleteNewFiles(Collection<String> files) {
|
||||
assert locked();
|
||||
for (final String fileName: files) {
|
||||
// NOTE: it's very unusual yet possible for the
|
||||
|
@ -713,8 +715,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
void deleteFile(String fileName)
|
||||
throws IOException {
|
||||
void deleteFile(String fileName) {
|
||||
assert locked();
|
||||
ensureOpen();
|
||||
try {
|
||||
|
@ -734,7 +735,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
infoStream.message("IFD", "unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
|
||||
}
|
||||
if (deletable == null) {
|
||||
deletable = new ArrayList<>();
|
||||
deletable = new HashSet<>();
|
||||
}
|
||||
deletable.add(fileName); // add to deletable
|
||||
}
|
||||
|
|
|
@ -2024,6 +2024,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
deleter.close();
|
||||
|
||||
// Must set closed while inside same sync block where we call deleter.refresh, else concurrent threads may try to sneak a flush in,
|
||||
// after we leave this sync block and before we enter the sync block in the finally clause below that sets closed:
|
||||
closed = true;
|
||||
|
||||
IOUtils.close(writeLock); // release write lock
|
||||
writeLock = null;
|
||||
|
||||
|
@ -2267,6 +2271,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
try {
|
||||
synchronized (this) {
|
||||
// Lock order IW -> BDS
|
||||
ensureOpen(false);
|
||||
synchronized (bufferedUpdatesStream) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "publishFlushedSegment");
|
||||
|
@ -2542,10 +2547,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
return;
|
||||
}
|
||||
|
||||
MergeState mergeState;
|
||||
boolean success = false;
|
||||
try {
|
||||
mergeState = merger.merge(); // merge 'em
|
||||
merger.merge(); // merge 'em
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -3845,8 +3849,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
merge.checkAborted(directory);
|
||||
|
||||
final String mergedName = merge.info.info.name;
|
||||
|
||||
List<SegmentCommitInfo> sourceSegments = merge.segments;
|
||||
|
||||
IOContext context = new IOContext(merge.getMergeInfo());
|
||||
|
@ -4511,7 +4513,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
Set<String> siFiles = new HashSet<>();
|
||||
for (String cfsFile : cfsFiles) {
|
||||
siFiles.add(cfsFile);
|
||||
};
|
||||
}
|
||||
info.setFiles(siFiles);
|
||||
|
||||
return files;
|
||||
|
|
Loading…
Reference in New Issue