LUCENE-8275: Fix BaseLockFactoryTestCase to step out on Windowns if pending files are found

The particular test here is #testStressLocks that has several protectesion against
WindowsFS and special logic in the catch clause that steps out on fatal exceptions with
pending deletes. Since we now check this consistently in the IW ctor we need to also
skip this entire test if we are on windows and have pending deletes.
This commit is contained in:
Simon Willnauer 2018-04-26 11:51:58 +02:00
parent d53de2a385
commit 933d8a6995
2 changed files with 16 additions and 7 deletions

View File

@ -25,6 +25,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
import org.apache.lucene.util.LuceneTestCase;
@ -65,6 +66,10 @@ public class TestIndexWriterOutOfFileDescriptors extends LuceneTestCase {
if (ms instanceof ConcurrentMergeScheduler) {
((ConcurrentMergeScheduler) ms).setSuppressExceptions();
}
if (Constants.WINDOWS && dir.checkPendingDeletions()) {
// if we are on windows and we have pending deletions we can't execute this test
break;
}
w = new IndexWriter(dir, iwc);
if (r != null && random().nextInt(5) == 3) {
if (random().nextBoolean()) {

View File

@ -173,9 +173,8 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
writer.start();
searcher.start();
while(writer.isAlive() || searcher.isAlive()) {
Thread.sleep(1000);
}
writer.join();
searcher.join();
assertTrue("IndexWriter hit unexpected exceptions", !writer.hitException);
assertTrue("IndexSearcher hit unexpected exceptions", !searcher.hitException);
@ -232,14 +231,20 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
printStream.println("\nTEST: WriterThread iter=" + i);
iwc.setOpenMode(OpenMode.APPEND);
try {
if (Constants.WINDOWS && dir.checkPendingDeletions()) {
// on windows we can potentially have pending deletes here if this happens we step out like in the catch clause
// tests using this also assumes no mock window FS
break;
}
writer = new IndexWriter(dir, iwc);
} catch (Throwable t) {
if (Constants.WINDOWS && t instanceof AccessDeniedException) {
// LUCENE-6684: suppress this: on Windows, a file in the curious "pending delete" state can
// cause this exc on IW init, where one thread/process deleted an old
// segments_N, but the delete hasn't finished yet because other threads/processes
// still have it open
printStream.println("TEST: AccessDeniedException on init witer");
printStream.println("TEST: AccessDeniedException on init writer");
t.printStackTrace(printStream);
} else {
hitException = true;
@ -268,7 +273,6 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
System.out.println(toString(baos));
break;
}
writer = null;
}
}
}