Skip to content

Commit c303692

Browse files
authored
OAK-11619 - Fix flaky ConcurrentMultiplexingIndexWriterTest
1 parent 55bc3e1 commit c303692

File tree

2 files changed

+32
-33
lines changed

2 files changed

+32
-33
lines changed

oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexWriterFactory.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@
2626
import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
2727
import org.apache.lucene.index.IndexableField;
2828

29-
public interface LuceneIndexWriterFactory extends FulltextIndexWriterFactory<Iterable<? extends IndexableField>> {
29+
public interface LuceneIndexWriterFactory extends FulltextIndexWriterFactory<Iterable<? extends IndexableField>>, AutoCloseable {
3030
@Override
3131
LuceneIndexWriter newInstance(IndexDefinition definition, NodeBuilder definitionBuilder, CommitInfo commitInfo, boolean reindex);
3232
}

oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/ConcurrentMultiplexingIndexWriterTest.java

+31-32
Original file line numberDiff line numberDiff line change
@@ -26,8 +26,8 @@
2626
import java.util.concurrent.CountDownLatch;
2727
import java.util.concurrent.ExecutorService;
2828
import java.util.concurrent.Executors;
29-
import java.util.concurrent.TimeUnit;
3029

30+
import org.apache.jackrabbit.oak.commons.concurrent.ExecutorCloser;
3131
import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexDefinition;
3232
import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexWriterFactory;
3333
import org.apache.jackrabbit.oak.plugins.index.lucene.directory.DefaultDirectoryFactory;
@@ -44,54 +44,53 @@ public class ConcurrentMultiplexingIndexWriterTest {
4444
@Rule
4545
public TemporaryFolder folder = new TemporaryFolder(new File("target"));
4646

47-
private NodeState root = INITIAL_CONTENT;
48-
private NodeBuilder builder = EMPTY_NODE.builder();
49-
private LuceneIndexDefinition defn = new LuceneIndexDefinition(root, builder.getNodeState(), "/foo");
50-
private MountInfoProvider mip = Mounts.newBuilder()
47+
private final NodeState root = INITIAL_CONTENT;
48+
private final NodeBuilder builder = EMPTY_NODE.builder();
49+
private final LuceneIndexDefinition defn = new LuceneIndexDefinition(root, builder.getNodeState(), "/foo");
50+
private final MountInfoProvider mip = Mounts.newBuilder()
5151
.mount("foo", "/libs", "/apps")
5252
.readOnlyMount("ro", "/ro-tree")
5353
.build();
54-
private LuceneIndexWriterConfig writerConfig = new LuceneIndexWriterConfig();
54+
private final LuceneIndexWriterConfig writerConfig = new LuceneIndexWriterConfig();
5555

5656
@Test
5757
public void concurrentWrite() throws Exception{
58-
LuceneIndexWriterFactory factory = newDirectoryFactory();
59-
final LuceneIndexWriter writer = factory.newInstance(defn, builder, null, true);
60-
6158
int THREAD_COUNT = 100;
6259
final int LOOP_COUNT = 10;
6360
ExecutorService executorService = Executors.newFixedThreadPool(THREAD_COUNT);
6461
CountDownLatch startLatch = new CountDownLatch(1);
6562
CountDownLatch doneLatch = new CountDownLatch(THREAD_COUNT);
6663
Exception[] firstException = new Exception[1];
67-
68-
for (int i = 0; i < THREAD_COUNT; i++) {
69-
executorService.submit(() -> {
70-
try {
71-
// wait for the signal
72-
startLatch.await();
73-
for (int j = 0; j < LOOP_COUNT; j++) {
74-
writer.updateDocument("/libs/config", newDoc("/libs/config"));
64+
65+
try (LuceneIndexWriterFactory factory = newDirectoryFactory()) {
66+
LuceneIndexWriter writer = factory.newInstance(defn, builder, null, true);
67+
for (int i = 0; i < THREAD_COUNT; i++) {
68+
executorService.submit(() -> {
69+
try {
70+
// wait for the signal
71+
startLatch.await();
72+
for (int j = 0; j < LOOP_COUNT; j++) {
73+
writer.updateDocument("/libs/config", newDoc("/libs/config"));
74+
}
75+
} catch (InterruptedException e) {
76+
Thread.currentThread().interrupt();
77+
} catch (Exception e) {
78+
firstException[0] = e;
79+
} finally {
80+
doneLatch.countDown();
7581
}
76-
} catch (InterruptedException e) {
77-
Thread.currentThread().interrupt();
78-
} catch (Exception e) {
79-
firstException[0] = e;
80-
} finally {
81-
doneLatch.countDown();
82-
}
83-
});
82+
});
83+
}
84+
// signal
85+
startLatch.countDown();
86+
doneLatch.await();
87+
writer.close(0);
8488
}
85-
// signal
86-
startLatch.countDown();
87-
doneLatch.await();
88-
executorService.shutdown();
89-
executorService.awaitTermination(10, TimeUnit.SECONDS);
89+
new ExecutorCloser(executorService).close();
9090
if (firstException[0] != null) {
9191
throw firstException[0];
9292
}
93-
writer.close(0);
94-
}
93+
}
9594

9695
private LuceneIndexWriterFactory newDirectoryFactory(){
9796
return newDirectoryFactory(mip);

0 commit comments

Comments
 (0)