mirror of https://github.com/apache/lucene.git
LUCENE-8201: improve the test a bit to try to diagnose timeouts on jenkins.
This commit is contained in:
parent
0cd9f7f35a
commit
cc3c4b6123
|
@ -19,13 +19,16 @@ package org.apache.lucene.codecs;
|
||||||
|
|
||||||
import java.nio.file.Paths;
|
import java.nio.file.Paths;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
import java.util.concurrent.CyclicBarrier;
|
import java.util.concurrent.CyclicBarrier;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.IntStream;
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
import org.apache.lucene.util.LuceneTestCase;
|
import org.apache.lucene.util.LuceneTestCase;
|
||||||
|
@ -42,12 +45,13 @@ import com.carrotsearch.randomizedtesting.RandomizedRunner;
|
||||||
|
|
||||||
@RunWith(RandomizedRunner.class)
|
@RunWith(RandomizedRunner.class)
|
||||||
public class TestCodecLoadingDeadlock extends Assert {
|
public class TestCodecLoadingDeadlock extends Assert {
|
||||||
|
private static int MAX_TIME_SECONDS = 30;
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDeadlock() throws Exception {
|
public void testDeadlock() throws Exception {
|
||||||
LuceneTestCase.assumeFalse("This test fails on UNIX with Turkish default locale (https://issues.apache.org/jira/browse/LUCENE-6036)",
|
LuceneTestCase.assumeFalse("This test fails on UNIX with Turkish default locale (https://issues.apache.org/jira/browse/LUCENE-6036)",
|
||||||
new Locale("tr").getLanguage().equals(Locale.getDefault().getLanguage()));
|
new Locale("tr").getLanguage().equals(Locale.getDefault().getLanguage()));
|
||||||
|
|
||||||
// pick random codec names for stress test in separate process:
|
// pick random codec names for stress test in separate process:
|
||||||
final Random rnd = RandomizedContext.current().getRandom();
|
final Random rnd = RandomizedContext.current().getRandom();
|
||||||
Set<String> avail;
|
Set<String> avail;
|
||||||
|
@ -68,24 +72,39 @@ public class TestCodecLoadingDeadlock extends Assert {
|
||||||
pfName,
|
pfName,
|
||||||
dvfName
|
dvfName
|
||||||
).inheritIO().start();
|
).inheritIO().start();
|
||||||
if (p.waitFor(30, TimeUnit.SECONDS)) {
|
if (p.waitFor(MAX_TIME_SECONDS * 2, TimeUnit.SECONDS)) {
|
||||||
assertEquals("Process died abnormally", 0, p.waitFor());
|
assertEquals("Process died abnormally?", 0, p.waitFor());
|
||||||
} else {
|
} else {
|
||||||
p.destroyForcibly().waitFor();
|
p.destroyForcibly().waitFor();
|
||||||
fail("Process did not exit after 30 secs -> classloader deadlock?");
|
fail("Process did not exit after 60 secs?");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// this method is called in a spawned process:
|
// This method is called in a spawned process.
|
||||||
public static void main(final String... args) throws Exception {
|
public static void main(final String... args) throws Exception {
|
||||||
final String codecName = args[0];
|
final String codecName = args[0];
|
||||||
final String pfName = args[1];
|
final String pfName = args[1];
|
||||||
final String dvfName = args[2];
|
final String dvfName = args[2];
|
||||||
|
|
||||||
|
System.out.println(String.format(Locale.ROOT,
|
||||||
|
"codec: %s, pf: %s, dvf: %s", codecName, pfName, dvfName));
|
||||||
|
|
||||||
final int numThreads = 14; // two times the modulo in switch statement below
|
final int numThreads = 14; // two times the modulo in switch statement below
|
||||||
final ExecutorService pool = Executors.newFixedThreadPool(numThreads, new NamedThreadFactory("deadlockchecker"));
|
final CopyOnWriteArrayList<Thread> allThreads = new CopyOnWriteArrayList<>();
|
||||||
|
final ExecutorService pool = Executors.newFixedThreadPool(numThreads, new NamedThreadFactory("deadlockchecker") {
|
||||||
|
@Override
|
||||||
|
public Thread newThread(Runnable r) {
|
||||||
|
Thread t = super.newThread(r);
|
||||||
|
allThreads.add(t);
|
||||||
|
return t;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
final CyclicBarrier barrier = new CyclicBarrier(numThreads);
|
final CyclicBarrier barrier = new CyclicBarrier(numThreads);
|
||||||
IntStream.range(0, numThreads).forEach(taskNo -> pool.execute(() -> {
|
IntStream.range(0, numThreads).forEach(taskNo -> pool.execute(() -> {
|
||||||
try {
|
try {
|
||||||
|
// Await a common barrier point for all threads and then
|
||||||
|
// run racy code. This is intentional.
|
||||||
barrier.await();
|
barrier.await();
|
||||||
switch (taskNo % 7) {
|
switch (taskNo % 7) {
|
||||||
case 0:
|
case 0:
|
||||||
|
@ -120,8 +139,22 @@ public class TestCodecLoadingDeadlock extends Assert {
|
||||||
Runtime.getRuntime().halt(1); // signal failure to caller
|
Runtime.getRuntime().halt(1); // signal failure to caller
|
||||||
}
|
}
|
||||||
}));
|
}));
|
||||||
pool.shutdown();
|
|
||||||
while (!pool.awaitTermination(1, TimeUnit.MINUTES));
|
|
||||||
}
|
|
||||||
|
|
||||||
|
pool.shutdown();
|
||||||
|
|
||||||
|
if (!pool.awaitTermination(MAX_TIME_SECONDS, TimeUnit.SECONDS)) {
|
||||||
|
// Try to collect stacks so that we can better diagnose the failure.
|
||||||
|
System.err.println("Pool didn't return after " + MAX_TIME_SECONDS +
|
||||||
|
" seconds, classloader deadlock? Dumping stack traces.");
|
||||||
|
|
||||||
|
for (Thread t : allThreads) {
|
||||||
|
System.err.println(
|
||||||
|
"# Thread: " + t + ", " +
|
||||||
|
"state: " + t.getState() + ", " +
|
||||||
|
"stack:\n\t" + Arrays.stream(t.getStackTrace()).map(Object::toString)
|
||||||
|
.collect(Collectors.joining("\t")) + "\n");
|
||||||
|
}
|
||||||
|
Runtime.getRuntime().halt(1); // signal failure to caller
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue