Improve RIW exception handling and opt out of concurrent flushing if exception is expected

This commit is contained in:
Simon Willnauer 2019-03-15 10:59:52 +01:00
parent d89387553d
commit 9ccac0f1bb
3 changed files with 40 additions and 18 deletions

View File

@ -481,7 +481,7 @@ public class TestIDVersionPostingsFormat extends LuceneTestCase {
doc.add(newStringField("id", "id", Field.Store.NO)); doc.add(newStringField("id", "id", Field.Store.NO));
expectThrows(IllegalArgumentException.class, () -> { expectThrows(IllegalArgumentException.class, () -> {
w.addDocument(doc); w.addDocument(doc);
w.commit(); w.commit(false);
}); });

View File

@ -87,7 +87,7 @@ public class TestContextSuggestField extends LuceneTestCase {
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> { IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
document.add(new ContextSuggestField("name", charsRefBuilder.toString(), 1, "sugg")); document.add(new ContextSuggestField("name", charsRefBuilder.toString(), 1, "sugg"));
iw.addDocument(document); iw.addDocument(document);
iw.commit(); iw.commit(false);
}); });
assertTrue(expected.getMessage().contains("[0x1d]")); assertTrue(expected.getMessage().contains("[0x1d]"));
} }
@ -139,7 +139,7 @@ public class TestContextSuggestField extends LuceneTestCase {
// mixing suggest field types for same field name should error out // mixing suggest field types for same field name should error out
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> { IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
iw.addDocument(document); iw.addDocument(document);
iw.commit(); iw.commit(false);
}); });
assertTrue(expected.getMessage().contains("mixed types")); assertTrue(expected.getMessage().contains("mixed types"));
} }

View File

@ -20,8 +20,9 @@ import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Iterator; import java.util.Iterator;
import java.util.List;
import java.util.Random; import java.util.Random;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockAnalyzer;
@ -314,29 +315,50 @@ public class RandomIndexWriter implements Closeable {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig()); LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
return w.deleteDocuments(q); return w.deleteDocuments(q);
} }
public long commit() throws IOException { public long commit() throws IOException {
return commit(r.nextInt(10) == 0);
}
public long commit(boolean flushConcurrently) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig()); LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
if (r.nextInt(10) == 0) { if (flushConcurrently) {
AtomicReference<Throwable> exception = new AtomicReference<>(); List<Throwable> throwableList = new CopyOnWriteArrayList<>();
Thread t = new Thread(() -> { Thread thread = new Thread(() -> {
try { try {
flushAllBuffersSequentially(); flushAllBuffersSequentially();
} catch (IOException e) { } catch (Throwable e) {
exception.set(e); throwableList.add(e);
} }
}); });
t.start(); thread.start();
long seqId = w.commit();
try { try {
t.join(); return w.commit();
} catch (InterruptedException e) { } catch (Throwable t) {
throw new AssertionError(e); throwableList.add(t);
} finally {
try {
// make sure we wait for the thread to join otherwise it might still be processing events
// and the IW won't be fully closed in the case of a fatal exception
thread.join();
} catch (InterruptedException e) {
throwableList.add(e);
}
} }
if (exception.get() != null) { if (throwableList.size() != 0) {
throw new AssertionError(exception.get()); Throwable primary = throwableList.get(0);
for (int i = 1; i < throwableList.size(); i++) {
primary.addSuppressed(throwableList.get(i));
}
if (primary instanceof IOException) {
throw (IOException)primary;
} else if (primary instanceof RuntimeException) {
throw (RuntimeException)primary;
} else {
throw new AssertionError(primary);
}
} }
return seqId;
} }
return w.commit(); return w.commit();
} }