mirror of https://github.com/apache/lucene.git
LUCENE-6616: IW lists files only once on init, IFD no longer suppresses FNFE, IFD deletes segments_N files last
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1689893 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
1e71ee4c07
commit
dd75010e6b
|
@ -311,6 +311,14 @@ Optimizations
|
|||
|
||||
* LUCENE-6617: Reduce heap usage for small FSTs (Mike McCandless)
|
||||
|
||||
* LUCENE-6616: IndexWriter now lists the files in the index directory
|
||||
only once on init, and IndexFileDeleter no longer suppresses
|
||||
FileNotFoundException and NoSuchFileException. This also improves
|
||||
IndexFileDeleter to delete segments_N files last, so that in the
|
||||
presence of a virus checker, the index is never left in a state
|
||||
where an expired segments_N references non-existing files (Robert
|
||||
Muir, Mike McCandless)
|
||||
|
||||
Build
|
||||
|
||||
* LUCENE-6518: Don't report false thread leaks from IBM J9
|
||||
|
|
|
@ -395,7 +395,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
if (state.isActive() && state.dwpt == null) {
|
||||
final FieldInfos.Builder infos = new FieldInfos.Builder(
|
||||
writer.globalFieldNumberMap);
|
||||
state.dwpt = new DocumentsWriterPerThread(writer.newSegmentName(), directoryOrig,
|
||||
state.dwpt = new DocumentsWriterPerThread(writer, writer.newSegmentName(), directoryOrig,
|
||||
directory, config, infoStream, deleteQueue, infos,
|
||||
writer.pendingNumDocs, writer.enableTestPoints);
|
||||
}
|
||||
|
|
|
@ -157,9 +157,11 @@ class DocumentsWriterPerThread {
|
|||
private final AtomicLong pendingNumDocs;
|
||||
private final LiveIndexWriterConfig indexWriterConfig;
|
||||
private final boolean enableTestPoints;
|
||||
private final IndexWriter indexWriter;
|
||||
|
||||
public DocumentsWriterPerThread(String segmentName, Directory directoryOrig, Directory directory, LiveIndexWriterConfig indexWriterConfig, InfoStream infoStream, DocumentsWriterDeleteQueue deleteQueue,
|
||||
public DocumentsWriterPerThread(IndexWriter writer, String segmentName, Directory directoryOrig, Directory directory, LiveIndexWriterConfig indexWriterConfig, InfoStream infoStream, DocumentsWriterDeleteQueue deleteQueue,
|
||||
FieldInfos.Builder fieldInfos, AtomicLong pendingNumDocs, boolean enableTestPoints) throws IOException {
|
||||
this.indexWriter = writer;
|
||||
this.directoryOrig = directoryOrig;
|
||||
this.directory = new TrackingDirectoryWrapper(directory);
|
||||
this.fieldInfos = fieldInfos;
|
||||
|
@ -488,7 +490,7 @@ class DocumentsWriterPerThread {
|
|||
if (indexWriterConfig.getUseCompoundFile()) {
|
||||
Set<String> originalFiles = newSegment.info.files();
|
||||
// TODO: like addIndexes, we are relying on createCompoundFile to successfully cleanup...
|
||||
IndexWriter.createCompoundFile(infoStream, new TrackingDirectoryWrapper(directory), newSegment.info, context);
|
||||
indexWriter.createCompoundFile(infoStream, new TrackingDirectoryWrapper(directory), newSegment.info, context);
|
||||
filesToDelete.addAll(originalFiles);
|
||||
newSegment.info.setUseCompoundFile(true);
|
||||
}
|
||||
|
|
|
@ -17,6 +17,12 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
|
@ -32,12 +38,6 @@ import java.util.Objects;
|
|||
import java.util.Set;
|
||||
import java.util.regex.Matcher;
|
||||
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/*
|
||||
* This class keeps track of each SegmentInfos instance that
|
||||
* is still "live", either because it corresponds to a
|
||||
|
@ -80,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 Set<String> deletable;
|
||||
private final Set<String> deletable = new HashSet<>();
|
||||
|
||||
/* Reference count for all files in the index.
|
||||
* Counts how many existing commits reference a file.
|
||||
|
@ -127,7 +127,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
* any files not referenced by any of the commits.
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public IndexFileDeleter(Directory directoryOrig, Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos,
|
||||
public IndexFileDeleter(String[] files, Directory directoryOrig, Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos,
|
||||
InfoStream infoStream, IndexWriter writer, boolean initialIndexExists,
|
||||
boolean isReaderInit) throws IOException {
|
||||
Objects.requireNonNull(writer);
|
||||
|
@ -147,7 +147,6 @@ final class IndexFileDeleter implements Closeable {
|
|||
// First pass: walk the files and initialize our ref
|
||||
// counts:
|
||||
CommitPoint currentCommitPoint = null;
|
||||
String[] files = directory.listAll();
|
||||
|
||||
if (currentSegmentsFile != null) {
|
||||
Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
|
||||
|
@ -166,32 +165,17 @@ final class IndexFileDeleter implements Closeable {
|
|||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "init: load commit \"" + fileName + "\"");
|
||||
}
|
||||
SegmentInfos sis = null;
|
||||
try {
|
||||
sis = SegmentInfos.readCommit(directoryOrig, fileName);
|
||||
} catch (FileNotFoundException | NoSuchFileException e) {
|
||||
// LUCENE-948: on NFS (and maybe others), if
|
||||
// you have writers switching back and forth
|
||||
// between machines, it's very likely that the
|
||||
// dir listing will be stale and will claim a
|
||||
// file segments_X exists when in fact it
|
||||
// doesn't. So, we catch this and handle it
|
||||
// as if the file does not exist
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
|
||||
}
|
||||
SegmentInfos sis = SegmentInfos.readCommit(directoryOrig, fileName);
|
||||
|
||||
final CommitPoint commitPoint = new CommitPoint(commitsToDelete, directoryOrig, sis);
|
||||
if (sis.getGeneration() == segmentInfos.getGeneration()) {
|
||||
currentCommitPoint = commitPoint;
|
||||
}
|
||||
if (sis != null) {
|
||||
final CommitPoint commitPoint = new CommitPoint(commitsToDelete, directoryOrig, sis);
|
||||
if (sis.getGeneration() == segmentInfos.getGeneration()) {
|
||||
currentCommitPoint = commitPoint;
|
||||
}
|
||||
commits.add(commitPoint);
|
||||
incRef(sis, true);
|
||||
commits.add(commitPoint);
|
||||
incRef(sis, true);
|
||||
|
||||
if (lastSegmentInfos == null || sis.getGeneration() > lastSegmentInfos.getGeneration()) {
|
||||
lastSegmentInfos = sis;
|
||||
}
|
||||
if (lastSegmentInfos == null || sis.getGeneration() > lastSegmentInfos.getGeneration()) {
|
||||
lastSegmentInfos = sis;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -238,6 +222,10 @@ final class IndexFileDeleter implements Closeable {
|
|||
RefCount rc = entry.getValue();
|
||||
final String fileName = entry.getKey();
|
||||
if (0 == rc.count) {
|
||||
// A segments_N file should never have ref count 0 on init:
|
||||
if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
|
||||
throw new IllegalStateException("file \"" + fileName + "\" has refCount=0, which should never happen on init");
|
||||
}
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "init: removing unreferenced file \"" + fileName + "\"");
|
||||
}
|
||||
|
@ -459,17 +447,16 @@ final class IndexFileDeleter implements Closeable {
|
|||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "refresh [prefix=" + segmentName + "]: removing newly created unreferenced file \"" + fileName + "\"");
|
||||
}
|
||||
deleteFile(fileName);
|
||||
deletable.add(fileName);
|
||||
}
|
||||
}
|
||||
|
||||
deletePendingFiles();
|
||||
}
|
||||
|
||||
void refresh() throws IOException {
|
||||
// Set to null so that we regenerate the list of pending
|
||||
// files; else we can accumulate same file more than
|
||||
// once
|
||||
assert locked();
|
||||
deletable = null;
|
||||
deletable.clear();
|
||||
refresh(null);
|
||||
}
|
||||
|
||||
|
@ -512,22 +499,35 @@ final class IndexFileDeleter implements Closeable {
|
|||
|
||||
public void deletePendingFiles() {
|
||||
assert locked();
|
||||
if (deletable != null) {
|
||||
Set<String> oldDeletable = deletable;
|
||||
deletable = null;
|
||||
for(String fileName : oldDeletable) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "delete pending file " + fileName);
|
||||
}
|
||||
RefCount rc = refCounts.get(fileName);
|
||||
if (rc != null && rc.count > 0) {
|
||||
// LUCENE-5904: should never happen! This means we are about to pending-delete a referenced index file
|
||||
assert false: "fileName=" + fileName + " is in pending delete list but also has refCount=" + rc.count;
|
||||
} else {
|
||||
deleteFile(fileName);
|
||||
|
||||
// Clone the set because it will change as we iterate:
|
||||
List<String> toDelete = new ArrayList<>(deletable);
|
||||
|
||||
// First pass: delete any segments_N files. We do these first to be certain stale commit points are removed
|
||||
// before we remove any files they reference. If any delete of segments_N fails, we leave all other files
|
||||
// undeleted so index is never in a corrupt state:
|
||||
for (String fileName : toDelete) {
|
||||
RefCount rc = refCounts.get(fileName);
|
||||
if (rc != null && rc.count > 0) {
|
||||
// LUCENE-5904: should never happen! This means we are about to pending-delete a referenced index file
|
||||
throw new IllegalStateException("file \"" + fileName + "\" is in pending delete set but has non-zero refCount=" + rc.count);
|
||||
} else if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
|
||||
if (deleteFile(fileName) == false) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "failed to remove commit point \"" + fileName + "\"; skipping deletion of all other pending files");
|
||||
}
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Only delete other files if we were able to remove the segments_N files; this way we never
|
||||
// leave a corrupt commit in the index even in the presense of virus checkers:
|
||||
for(String fileName : toDelete) {
|
||||
if (fileName.startsWith(IndexFileNames.SEGMENTS) == false) {
|
||||
deleteFile(fileName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -560,10 +560,6 @@ final class IndexFileDeleter implements Closeable {
|
|||
infoStream.message("IFD", "now checkpoint \"" + writer.segString(writer.toLiveInfos(segmentInfos)) + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
|
||||
}
|
||||
|
||||
// Try again now to delete any previously un-deletable
|
||||
// files (because they were in use, on Windows):
|
||||
deletePendingFiles();
|
||||
|
||||
// Incref the files:
|
||||
incRef(segmentInfos, isCommit);
|
||||
|
||||
|
@ -587,6 +583,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
// Save files so we can decr on next checkpoint/commit:
|
||||
lastFiles.addAll(segmentInfos.files(false));
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
long t1 = System.nanoTime();
|
||||
infoStream.message("IFD", ((t1-t0)/1000000) + " msec to checkpoint");
|
||||
|
@ -635,6 +632,15 @@ final class IndexFileDeleter implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
try {
|
||||
deletePendingFiles();
|
||||
} catch (Throwable t) {
|
||||
if (firstThrowable == null) {
|
||||
// Save first exception and throw it in the end, but be sure to finish decRef all files
|
||||
firstThrowable = t;
|
||||
}
|
||||
}
|
||||
|
||||
// NOTE: does nothing if firstThrowable is null
|
||||
IOUtils.reThrowUnchecked(firstThrowable);
|
||||
}
|
||||
|
@ -649,9 +655,14 @@ final class IndexFileDeleter implements Closeable {
|
|||
} catch (Throwable t) {
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
deletePendingFiles();
|
||||
} catch (Throwable t) {
|
||||
}
|
||||
}
|
||||
|
||||
void decRef(String fileName) {
|
||||
private void decRef(String fileName) {
|
||||
assert locked();
|
||||
RefCount rc = getRefCount(fileName);
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
|
@ -663,7 +674,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
// This file is no longer referenced by any past
|
||||
// commit points nor by the in-memory SegmentInfos:
|
||||
try {
|
||||
deleteFile(fileName);
|
||||
deletable.add(fileName);
|
||||
} finally {
|
||||
refCounts.remove(fileName);
|
||||
}
|
||||
|
@ -698,16 +709,9 @@ final class IndexFileDeleter implements Closeable {
|
|||
return rc;
|
||||
}
|
||||
|
||||
void deleteFiles(List<String> files) {
|
||||
assert locked();
|
||||
for(final String file: files) {
|
||||
deleteFile(file);
|
||||
}
|
||||
}
|
||||
|
||||
/** Deletes the specified files, but only if they are new
|
||||
* (have not yet been incref'd). */
|
||||
void deleteNewFiles(Collection<String> files) {
|
||||
void deleteNewFiles(Collection<String> files) throws IOException {
|
||||
assert locked();
|
||||
for (final String fileName: files) {
|
||||
// NOTE: it's very unusual yet possible for the
|
||||
|
@ -718,14 +722,19 @@ final class IndexFileDeleter implements Closeable {
|
|||
// TestCrash.testCrashAfterReopen can hit this:
|
||||
if (!refCounts.containsKey(fileName) || refCounts.get(fileName).count == 0) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "delete new file \"" + fileName + "\"");
|
||||
infoStream.message("IFD", "will delete new file \"" + fileName + "\"");
|
||||
}
|
||||
deleteFile(fileName);
|
||||
deletable.add(fileName);
|
||||
}
|
||||
}
|
||||
|
||||
deletePendingFiles();
|
||||
}
|
||||
|
||||
void deleteFile(String fileName) {
|
||||
/** Returns true if the delete succeeded. Otherwise, the fileName is
|
||||
* added to the deletable set so we will retry the delete later, and
|
||||
* we return false. */
|
||||
private boolean deleteFile(String fileName) {
|
||||
assert locked();
|
||||
ensureOpen();
|
||||
try {
|
||||
|
@ -733,7 +742,14 @@ final class IndexFileDeleter implements Closeable {
|
|||
infoStream.message("IFD", "delete \"" + fileName + "\"");
|
||||
}
|
||||
directory.deleteFile(fileName);
|
||||
deletable.remove(fileName);
|
||||
return true;
|
||||
} catch (IOException e) { // if delete fails
|
||||
|
||||
// IndexWriter should only ask us to delete files it knows it wrote, so if we hit this, something is wrong!
|
||||
assert e instanceof NoSuchFileException == false: "file=" + fileName;
|
||||
assert e instanceof FileNotFoundException == false: "file=" + fileName;
|
||||
|
||||
// Some operating systems (e.g. Windows) don't
|
||||
// permit a file to be deleted while it is opened
|
||||
// for read (e.g. by another process or thread). So
|
||||
|
@ -744,10 +760,8 @@ final class IndexFileDeleter implements Closeable {
|
|||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
|
||||
}
|
||||
if (deletable == null) {
|
||||
deletable = new HashSet<>();
|
||||
}
|
||||
deletable.add(fileName); // add to deletable
|
||||
deletable.add(fileName);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -794,6 +794,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
boolean initialIndexExists = true;
|
||||
boolean fromReader = false;
|
||||
|
||||
String[] files = directory.listAll();
|
||||
|
||||
// Set up our initial SegmentInfos:
|
||||
IndexCommit commit = config.getIndexCommit();
|
||||
|
||||
|
@ -884,7 +886,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
} else {
|
||||
// Init from either the latest commit point, or an explicit prior commit point:
|
||||
|
||||
String[] files = directory.listAll();
|
||||
String lastSegmentsFile = SegmentInfos.getLastCommitSegmentsFileName(files);
|
||||
if (lastSegmentsFile == null) {
|
||||
throw new IndexNotFoundException("no segments* file found in " + directory + ": files: " + Arrays.toString(files));
|
||||
|
@ -928,8 +929,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
// Default deleter (for backwards compatibility) is
|
||||
// KeepOnlyLastCommitDeleter:
|
||||
|
||||
// Sync'd is silly here, but IFD asserts we sync'd on the IW instance:
|
||||
synchronized(this) {
|
||||
deleter = new IndexFileDeleter(directoryOrig, directory,
|
||||
deleter = new IndexFileDeleter(files, directoryOrig, directory,
|
||||
config.getIndexDeletionPolicy(),
|
||||
segmentInfos, infoStream, this,
|
||||
initialIndexExists, reader != null);
|
||||
|
@ -2493,7 +2496,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
} finally {
|
||||
if (!success) {
|
||||
for(SegmentCommitInfo sipc : infos) {
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory, sipc.files().toArray(new String[0]));
|
||||
// Safe: these files must exist
|
||||
deleteNewFiles(sipc.files());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2510,12 +2514,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
} finally {
|
||||
if (!success) {
|
||||
for(SegmentCommitInfo sipc : infos) {
|
||||
for(String file : sipc.files()) {
|
||||
try {
|
||||
directory.deleteFile(file);
|
||||
} catch (Throwable t) {
|
||||
}
|
||||
}
|
||||
// Safe: these files must exist
|
||||
deleteNewFiles(sipc.files());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2628,7 +2628,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
boolean useCompoundFile;
|
||||
synchronized(this) { // Guard segmentInfos
|
||||
if (stopMerges) {
|
||||
deleter.deleteNewFiles(infoPerCommit.files());
|
||||
// Safe: these files must exist
|
||||
deleteNewFiles(infoPerCommit.files());
|
||||
return;
|
||||
}
|
||||
ensureOpen();
|
||||
|
@ -2646,9 +2647,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
} finally {
|
||||
// delete new non cfs files directly: they were never
|
||||
// registered with IFD
|
||||
synchronized(this) {
|
||||
deleter.deleteNewFiles(filesToDelete);
|
||||
}
|
||||
deleteNewFiles(filesToDelete);
|
||||
}
|
||||
info.setUseCompoundFile(true);
|
||||
}
|
||||
|
@ -2674,7 +2673,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
// Register the new segment
|
||||
synchronized(this) {
|
||||
if (stopMerges) {
|
||||
deleter.deleteNewFiles(info.files());
|
||||
// Safe: these files must exist
|
||||
deleteNewFiles(infoPerCommit.files());
|
||||
return;
|
||||
}
|
||||
ensureOpen();
|
||||
|
@ -2706,6 +2706,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
boolean success = false;
|
||||
|
||||
Set<String> copiedFiles = new HashSet<>();
|
||||
try {
|
||||
// Copy the segment's files
|
||||
for (String file: info.files()) {
|
||||
|
@ -2714,13 +2715,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
assert !slowFileExists(directory, newFileName): "file \"" + newFileName + "\" already exists; newInfo.files=" + newInfo.files();
|
||||
|
||||
directory.copyFrom(info.info.dir, file, newFileName, context);
|
||||
copiedFiles.add(newFileName);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory, newInfo.files().toArray(new String[0]));
|
||||
// Safe: these files must exist
|
||||
deleteNewFiles(copiedFiles);
|
||||
}
|
||||
}
|
||||
|
||||
assert copiedFiles.equals(newInfoPerCommit.files());
|
||||
|
||||
return newInfoPerCommit;
|
||||
}
|
||||
|
@ -3469,7 +3474,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
// doing this makes MockDirWrapper angry in
|
||||
// TestNRTThreads (LUCENE-5434):
|
||||
readerPool.drop(merge.info);
|
||||
deleter.deleteNewFiles(merge.info.files());
|
||||
|
||||
// Safe: these files must exist:
|
||||
deleteNewFiles(merge.info.files());
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -3536,7 +3543,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
if (dropSegment) {
|
||||
assert !segmentInfos.contains(merge.info);
|
||||
readerPool.drop(merge.info);
|
||||
deleter.deleteNewFiles(merge.info.files());
|
||||
// Safe: these files must exist
|
||||
deleteNewFiles(merge.info.files());
|
||||
}
|
||||
|
||||
boolean success = false;
|
||||
|
@ -3639,9 +3647,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
} finally {
|
||||
synchronized(this) {
|
||||
|
||||
mergeFinish(merge);
|
||||
|
||||
if (!success) {
|
||||
if (success == false) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception during merge");
|
||||
}
|
||||
|
@ -4119,31 +4128,27 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
try {
|
||||
createCompoundFile(infoStream, trackingCFSDir, merge.info.info, context);
|
||||
success = true;
|
||||
} catch (IOException ioe) {
|
||||
} catch (Throwable t) {
|
||||
synchronized(this) {
|
||||
if (merge.rateLimiter.getAbort()) {
|
||||
// This can happen if rollback or close(false)
|
||||
// is called -- fall through to logic below to
|
||||
// remove the partially created CFS:
|
||||
// This can happen if rollback is called while we were building
|
||||
// our CFS -- fall through to logic below to remove the non-CFS
|
||||
// merged files:
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit merge abort exception creating compound file during merge");
|
||||
}
|
||||
return 0;
|
||||
} else {
|
||||
handleMergeException(ioe, merge);
|
||||
handleMergeException(t, merge);
|
||||
}
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
handleMergeException(t, merge);
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (success == false) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception creating compound file during merge");
|
||||
}
|
||||
|
||||
synchronized(this) {
|
||||
Set<String> cfsFiles = new HashSet<>(trackingCFSDir.getCreatedFiles());
|
||||
for (String cfsFile : cfsFiles) {
|
||||
deleter.deleteFile(cfsFile);
|
||||
}
|
||||
deleter.deleteNewFiles(merge.info.files());
|
||||
}
|
||||
// Safe: these files must exist
|
||||
deleteNewFiles(merge.info.files());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -4156,16 +4161,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
// delete new non cfs files directly: they were never
|
||||
// registered with IFD
|
||||
deleter.deleteNewFiles(filesToRemove);
|
||||
deleteNewFiles(filesToRemove);
|
||||
|
||||
if (merge.rateLimiter.getAbort()) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "abort merge after building CFS");
|
||||
}
|
||||
Set<String> cfsFiles = new HashSet<>(trackingCFSDir.getCreatedFiles());
|
||||
for (String cfsFile : cfsFiles) {
|
||||
deleter.deleteFile(cfsFile);
|
||||
}
|
||||
// Safe: these files must exist
|
||||
deleteNewFiles(merge.info.files());
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
@ -4188,9 +4191,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
success2 = true;
|
||||
} finally {
|
||||
if (!success2) {
|
||||
synchronized(this) {
|
||||
deleter.deleteNewFiles(merge.info.files());
|
||||
}
|
||||
// Safe: these files must exist
|
||||
deleteNewFiles(merge.info.files());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -4227,7 +4229,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
} finally {
|
||||
// Readers are already closed in commitMerge if we didn't hit
|
||||
// an exc:
|
||||
if (!success) {
|
||||
if (success == false) {
|
||||
closeMergeReaders(merge, true);
|
||||
}
|
||||
}
|
||||
|
@ -4637,8 +4639,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* deletion files, this SegmentInfo must not reference such files when this
|
||||
* method is called, because they are not allowed within a compound file.
|
||||
*/
|
||||
static final void createCompoundFile(InfoStream infoStream, TrackingDirectoryWrapper directory, final SegmentInfo info, IOContext context)
|
||||
throws IOException {
|
||||
final void createCompoundFile(InfoStream infoStream, TrackingDirectoryWrapper directory, final SegmentInfo info, IOContext context) throws IOException {
|
||||
|
||||
// maybe this check is not needed, but why take the risk?
|
||||
if (!directory.getCreatedFiles().isEmpty()) {
|
||||
|
@ -4655,16 +4656,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
Set<String> cfsFiles = new HashSet<>(directory.getCreatedFiles());
|
||||
for (String file : cfsFiles) {
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory, file);
|
||||
}
|
||||
// Safe: these files must exist
|
||||
deleteNewFiles(directory.getCreatedFiles());
|
||||
}
|
||||
}
|
||||
|
||||
// Replace all previous files with the CFS/CFE files:
|
||||
Set<String> siFiles = new HashSet<>(directory.getCreatedFiles());
|
||||
info.setFiles(siFiles);
|
||||
info.setFiles(new HashSet<>(directory.getCreatedFiles()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -4765,7 +4763,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* (can be opened), false if it cannot be opened, and
|
||||
* (unlike Java's File.exists) throws IOException if
|
||||
* there's some unexpected error. */
|
||||
private static boolean slowFileExists(Directory dir, String fileName) throws IOException {
|
||||
static boolean slowFileExists(Directory dir, String fileName) throws IOException {
|
||||
try {
|
||||
dir.openInput(fileName, IOContext.DEFAULT).close();
|
||||
return true;
|
||||
|
|
|
@ -235,7 +235,7 @@ public class TestDoc extends LuceneTestCase {
|
|||
|
||||
if (useCompoundFile) {
|
||||
Collection<String> filesToDelete = si.files();
|
||||
IndexWriter.createCompoundFile(InfoStream.getDefault(), new TrackingDirectoryWrapper(dir), si, newIOContext(random()));
|
||||
codec.compoundFormat().write(dir, si, context);
|
||||
si.setUseCompoundFile(true);
|
||||
for (final String fileToDelete : filesToDelete) {
|
||||
si1.info.dir.deleteFile(fileToDelete);
|
||||
|
|
|
@ -1467,33 +1467,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testNoSegmentFile() throws IOException {
|
||||
BaseDirectoryWrapper dir = newDirectory(random(), NoLockFactory.INSTANCE);
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
|
||||
.setMaxBufferedDocs(2));
|
||||
|
||||
Document doc = new Document();
|
||||
FieldType customType = new FieldType(TextField.TYPE_STORED);
|
||||
customType.setStoreTermVectors(true);
|
||||
customType.setStoreTermVectorPositions(true);
|
||||
customType.setStoreTermVectorOffsets(true);
|
||||
doc.add(newField("c", "val", customType));
|
||||
w.addDocument(doc);
|
||||
w.addDocument(doc);
|
||||
IndexWriter w2 = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
|
||||
.setMaxBufferedDocs(2)
|
||||
.setOpenMode(OpenMode.CREATE));
|
||||
|
||||
w2.close();
|
||||
// If we don't do that, the test fails on Windows
|
||||
w.rollback();
|
||||
|
||||
// This test leaves only segments.gen, which causes
|
||||
// DirectoryReader.indexExists to return true:
|
||||
dir.setCheckIndexOnClose(false);
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testNoUnwantedTVFiles() throws Exception {
|
||||
|
||||
Directory dir = newDirectory();
|
||||
|
|
|
@ -151,7 +151,7 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
|
|||
// no unexpected exceptions are raised:
|
||||
public void testStressLocks() throws Exception {
|
||||
Directory dir = getDirectory(createTempDir());
|
||||
|
||||
|
||||
// First create a 1 doc index:
|
||||
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())).setOpenMode(OpenMode.CREATE));
|
||||
addDoc(w);
|
||||
|
@ -190,6 +190,9 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
|
|||
public void run() {
|
||||
IndexWriter writer = null;
|
||||
for(int i=0;i<this.numIteration;i++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: WriterThread iter=" + i);
|
||||
}
|
||||
try {
|
||||
writer = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())).setOpenMode(OpenMode.APPEND));
|
||||
} catch (LockObtainFailedException e) {
|
||||
|
|
Loading…
Reference in New Issue