From 59cc1f1239b151fbcf7d7cfe1e46a2ffb04dbd1e Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 7 Jun 2024 23:20:08 +0200 Subject: [PATCH 1/8] TaskExecutor should not fork for one of its tasks When executing N tasks and waiting on the result of all of them, we should not fork one of them and just execute at least one of them outright. This saves at least one context switch, removes the need for any reentrancy protection, and makes full use of the supplied task executor if tasks fan out again. --- .../apache/lucene/search/TaskExecutor.java | 71 +++++++------------ .../lucene/search/TestIndexSearcher.java | 2 +- .../lucene/search/TestTaskExecutor.java | 12 ++-- 3 files changed, 33 insertions(+), 52 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java b/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java index 2763a9800e85..e94d73b5a0fc 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java +++ b/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java @@ -30,6 +30,7 @@ import java.util.concurrent.FutureTask; import java.util.concurrent.RunnableFuture; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.ThreadInterruptedException; @@ -38,19 +39,11 @@ * across segments as well as query rewrite in some cases. Exposes a single {@link * #invokeAll(Collection)} method that takes a collection of {@link Callable}s and executes them * concurrently/ Once all tasks are submitted to the executor, it blocks and wait for all tasks to - * be completed, and then returns a list with the obtained results. Ensures that the underlying - * executor is only used for top-level {@link #invokeAll(Collection)} calls, and not for potential - * {@link #invokeAll(Collection)} calls made from one of the tasks. This is to prevent deadlock with - * certain types of pool based executors (e.g. {@link java.util.concurrent.ThreadPoolExecutor}). + * be completed, and then returns a list with the obtained results. * * @lucene.experimental */ public final class TaskExecutor { - // a static thread local is ok as long as we use a counter, which accounts for multiple - // searchers holding a different TaskExecutor all backed by the same executor - private static final ThreadLocal numberOfRunningTasksInCurrentThread = - ThreadLocal.withInitial(() -> 0); - private final Executor executor; /** @@ -84,26 +77,23 @@ public String toString() { /** * Holds all the sub-tasks that a certain operation gets split into as it gets parallelized and * exposes the ability to invoke such tasks and wait for them all to complete their execution and - * provide their results. Ensures that each task does not get parallelized further: this is - * important to avoid a deadlock in situations where one executor thread waits on other executor - * threads to complete before it can progress. This happens in situations where for instance - * {@link Query#createWeight(IndexSearcher, ScoreMode, float)} is called as part of searching each - * slice, like {@link TopFieldCollector#populateScores(ScoreDoc[], IndexSearcher, Query)} does. - * Additionally, if one task throws an exception, all other tasks from the same group are - * cancelled, to avoid needless computation as their results would not be exposed anyways. Creates - * one {@link FutureTask} for each {@link Callable} provided + * provide their results. Additionally, if one task throws an exception, all other tasks from the + * same group are cancelled, to avoid needless computation as their results would not be exposed + * anyways. Creates one {@link FutureTask} for each {@link Callable} provided * * @param the return type of all the callables */ private static final class TaskGroup { - private final Collection> futures; + private final List> futures; + private final AtomicInteger taskId; TaskGroup(Collection> callables) { List> tasks = new ArrayList<>(callables.size()); for (Callable callable : callables) { tasks.add(createTask(callable)); } - this.futures = Collections.unmodifiableCollection(tasks); + this.futures = Collections.unmodifiableList(tasks); + this.taskId = new AtomicInteger(0); } RunnableFuture createTask(Callable callable) { @@ -112,15 +102,10 @@ RunnableFuture createTask(Callable callable) { () -> { if (startedOrCancelled.compareAndSet(false, true)) { try { - Integer counter = numberOfRunningTasksInCurrentThread.get(); - numberOfRunningTasksInCurrentThread.set(counter + 1); return callable.call(); } catch (Throwable t) { cancelAll(); throw t; - } finally { - Integer counter = numberOfRunningTasksInCurrentThread.get(); - numberOfRunningTasksInCurrentThread.set(counter - 1); } } // task is cancelled hence it has no results to return. That's fine: they would be @@ -144,32 +129,30 @@ public boolean cancel(boolean mayInterruptIfRunning) { } List invokeAll(Executor executor) throws IOException { - boolean runOnCallerThread = numberOfRunningTasksInCurrentThread.get() > 0; - for (Runnable runnable : futures) { - if (runOnCallerThread) { - runnable.run(); - } else { - executor.execute(runnable); - } + final int count = futures.size(); + for (int j = 0; j < count - 1; j++) { + executor.execute( + () -> { + int id = taskId.getAndIncrement(); + if (id < count) { + futures.get(id).run(); + } + }); + } + int id; + while ((id = taskId.getAndIncrement()) < count) { + futures.get(id).run(); } Throwable exc = null; - List results = new ArrayList<>(futures.size()); - for (Future future : futures) { + List results = new ArrayList<>(count); + for (int i = 0; i < count; i++) { + Future future = futures.get(i); try { results.add(future.get()); } catch (InterruptedException e) { - var newException = new ThreadInterruptedException(e); - if (exc == null) { - exc = newException; - } else { - exc.addSuppressed(newException); - } + exc = IOUtils.useOrSuppress(exc, new ThreadInterruptedException(e)); } catch (ExecutionException e) { - if (exc == null) { - exc = e.getCause(); - } else { - exc.addSuppressed(e.getCause()); - } + exc = IOUtils.useOrSuppress(exc, e.getCause()); } } assert assertAllFuturesCompleted() : "Some tasks are still running?"; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java b/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java index 3cd2ecbdd195..d8053b4f797b 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java @@ -286,7 +286,7 @@ protected LeafSlice[] slices(List leaves) { } }; searcher.search(new MatchAllDocsQuery(), 10); - assertEquals(leaves.size(), numExecutions.get()); + assertEquals(leaves.size(), numExecutions.get() + 1); } public void testNullExecutorNonNullTaskExecutor() { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java b/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java index ba5fa90f2276..90a91d3f6723 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java @@ -234,11 +234,8 @@ public void testInvokeAllDoesNotLeaveTasksBehind() { TaskExecutor taskExecutor = new TaskExecutor( command -> { - executorService.execute( - () -> { - tasksStarted.incrementAndGet(); - command.run(); - }); + tasksStarted.incrementAndGet(); + command.run(); }); AtomicInteger tasksExecuted = new AtomicInteger(0); List> callables = new ArrayList<>(); @@ -258,7 +255,8 @@ public void testInvokeAllDoesNotLeaveTasksBehind() { expectThrows(RuntimeException.class, () -> taskExecutor.invokeAll(callables)); assertEquals(1, tasksExecuted.get()); // the callables are technically all run, but the cancelled ones will be no-op - assertEquals(100, tasksStarted.get()); + // add one for the task the gets executed on the current thread + assertEquals(100, tasksStarted.get() + 1); } /** @@ -308,7 +306,7 @@ public void testInvokeAllCatchesMultipleExceptions() { } public void testCancelTasksOnException() { - TaskExecutor taskExecutor = new TaskExecutor(executorService); + TaskExecutor taskExecutor = new TaskExecutor(Runnable::run); final int numTasks = random().nextInt(10, 50); final int throwingTask = random().nextInt(numTasks); boolean error = random().nextBoolean(); From 29bdbf94594b04b6ff2329f7c6b0d9d4d6ab1f16 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 15 Jun 2024 19:14:36 +0200 Subject: [PATCH 2/8] optimizei3lock --- .../org/apache/lucene/search/TaskExecutor.java | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java b/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java index e94d73b5a0fc..2fc343198ac4 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java +++ b/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java @@ -85,7 +85,6 @@ public String toString() { */ private static final class TaskGroup { private final List> futures; - private final AtomicInteger taskId; TaskGroup(Collection> callables) { List> tasks = new ArrayList<>(callables.size()); @@ -93,7 +92,6 @@ private static final class TaskGroup { tasks.add(createTask(callable)); } this.futures = Collections.unmodifiableList(tasks); - this.taskId = new AtomicInteger(0); } RunnableFuture createTask(Callable callable) { @@ -130,18 +128,28 @@ public boolean cancel(boolean mayInterruptIfRunning) { List invokeAll(Executor executor) throws IOException { final int count = futures.size(); - for (int j = 0; j < count - 1; j++) { - executor.execute( + // taskId provides the first index of an un-executed task in #futures + final AtomicInteger taskId = new AtomicInteger(0); + // we fork execution count - 1 times and execute the last task on the current thread + if (count > 1) { + final Runnable work = () -> { int id = taskId.getAndIncrement(); if (id < count) { futures.get(id).run(); } - }); + }; + for (int j = 0; j < count - 1; j++) { + executor.execute(work); + } } int id; while ((id = taskId.getAndIncrement()) < count) { futures.get(id).run(); + if (id == count - 1) { + // save redundant CAS in case this was the last task + break; + } } Throwable exc = null; List results = new ArrayList<>(count); From 55fa51441d0c99c12865784db4be27eafdaae306 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 15 Jun 2024 20:07:59 +0200 Subject: [PATCH 3/8] add some docs --- .../src/java/org/apache/lucene/search/TaskExecutor.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java b/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java index 2fc343198ac4..14c4123654bd 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java +++ b/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java @@ -130,7 +130,8 @@ List invokeAll(Executor executor) throws IOException { final int count = futures.size(); // taskId provides the first index of an un-executed task in #futures final AtomicInteger taskId = new AtomicInteger(0); - // we fork execution count - 1 times and execute the last task on the current thread + // we fork execution count - 1 tasks to execute at least one task on the current thread to + // minimize needless forking and blocking of the current thread if (count > 1) { final Runnable work = () -> { @@ -143,10 +144,14 @@ List invokeAll(Executor executor) throws IOException { executor.execute(work); } } + // try to execute as many tasks as possible on the current thread to minimize context + // switching in case of long running concurrent + // tasks as well as dead-locking if the current thread is part of #executor for executors that + // have limited or no parallelism int id; while ((id = taskId.getAndIncrement()) < count) { futures.get(id).run(); - if (id == count - 1) { + if (id >= count - 1) { // save redundant CAS in case this was the last task break; } From 9781daea79375f618320fcee7e227300bcc7c48e Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 15 Jun 2024 20:17:03 +0200 Subject: [PATCH 4/8] improve comments --- .../core/src/java/org/apache/lucene/search/TaskExecutor.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java b/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java index 14c4123654bd..8832cf478e99 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java +++ b/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java @@ -38,8 +38,9 @@ * Executor wrapper responsible for the execution of concurrent tasks. Used to parallelize search * across segments as well as query rewrite in some cases. Exposes a single {@link * #invokeAll(Collection)} method that takes a collection of {@link Callable}s and executes them - * concurrently/ Once all tasks are submitted to the executor, it blocks and wait for all tasks to - * be completed, and then returns a list with the obtained results. + * concurrently. Once all but one task have been submitted to the executor, it tries to run as many + * tasks as possible on the calling thread, then waits for all tasks that have been executed in + * parallel on the executor to be completed and then returns a list with the obtained results. * * @lucene.experimental */ From 1d310aea43b3cee6c8f011e8ea031aa740976266 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 2 Jul 2024 18:08:35 +0200 Subject: [PATCH 5/8] enhance tests + changes entry --- lucene/CHANGES.txt | 9 ++++++ .../lucene/search/TestIndexSearcher.java | 4 +-- .../lucene/search/TestTaskExecutor.java | 29 ++++++++++++------- 3 files changed, 30 insertions(+), 12 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 320de7aaefe8..530eb81806fa 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -277,6 +277,15 @@ Optimizations * GITHUB#12941: Don't preserve auxiliary buffer contents in LSBRadixSorter if it grows. (Stefan Vodita) +Changes in runtime behavior +--------------------- + +* GITHUB#13472: IndexSearcher now executes tasks on the thread that invoked a search as well as its configured + executor. Users that invoke searches from threads not belonging to the executor configured in IndexSearcher should + reduce the executor's thread-count by 1 to retain the previous level of parallelism. Searches invoked from a thread + not part of the IndexSearcher's executor will now complete on the calling thread in case the executor is busy or + blocked and unable to complete any work. + Bug Fixes --------------------- diff --git a/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java b/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java index d8053b4f797b..724013abac71 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java @@ -266,7 +266,7 @@ protected LeafSlice[] slices(List leaves) { IOUtils.close(r, dir); } - public void testSlicesAllOffloadedToTheExecutor() throws IOException { + public void testSlicesOffloadedToTheExecutor() throws IOException { List leaves = reader.leaves(); AtomicInteger numExecutions = new AtomicInteger(0); IndexSearcher searcher = @@ -286,7 +286,7 @@ protected LeafSlice[] slices(List leaves) { } }; searcher.search(new MatchAllDocsQuery(), 10); - assertEquals(leaves.size(), numExecutions.get() + 1); + assertEquals(leaves.size() - 1, numExecutions.get()); } public void testNullExecutorNonNullTaskExecutor() { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java b/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java index 90a91d3f6723..0721d234ef78 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java @@ -21,10 +21,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; import org.apache.lucene.document.Document; import org.apache.lucene.index.DirectoryReader; @@ -112,6 +109,12 @@ public void testUnwrappedExceptions() { } public void testInvokeAllFromTaskDoesNotDeadlockSameSearcher() throws IOException { + doTestInvokeAllFromTaskDoesNotDeadlockSameSearcher(executorService); + doTestInvokeAllFromTaskDoesNotDeadlockSameSearcher(Runnable::run); + } + + private static void doTestInvokeAllFromTaskDoesNotDeadlockSameSearcher(Executor executor) + throws IOException { try (Directory dir = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), dir)) { for (int i = 0; i < 500; i++) { @@ -119,7 +122,7 @@ public void testInvokeAllFromTaskDoesNotDeadlockSameSearcher() throws IOExceptio } try (DirectoryReader reader = iw.getReader()) { IndexSearcher searcher = - new IndexSearcher(reader, executorService) { + new IndexSearcher(reader, executor) { @Override protected LeafSlice[] slices(List leaves) { return slices(leaves, 1, 1); @@ -173,6 +176,12 @@ public Void reduce(Collection collectors) { } public void testInvokeAllFromTaskDoesNotDeadlockMultipleSearchers() throws IOException { + doTestInvokeAllFromTaskDoesNotDeadlockMultipleSearchers(executorService); + doTestInvokeAllFromTaskDoesNotDeadlockMultipleSearchers(Runnable::run); + } + + private static void doTestInvokeAllFromTaskDoesNotDeadlockMultipleSearchers(Executor executor) + throws IOException { try (Directory dir = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), dir)) { for (int i = 0; i < 500; i++) { @@ -180,7 +189,7 @@ public void testInvokeAllFromTaskDoesNotDeadlockMultipleSearchers() throws IOExc } try (DirectoryReader reader = iw.getReader()) { IndexSearcher searcher = - new IndexSearcher(reader, executorService) { + new IndexSearcher(reader, executor) { @Override protected LeafSlice[] slices(List leaves) { return slices(leaves, 1, 1); @@ -202,7 +211,7 @@ public void setScorer(Scorable scorer) throws IOException { // searcher has its own // TaskExecutor, the safeguard is shared among all the searchers that get // the same executor - IndexSearcher indexSearcher = new IndexSearcher(reader, executorService); + IndexSearcher indexSearcher = new IndexSearcher(reader, executor); indexSearcher .getTaskExecutor() .invokeAll(Collections.singletonList(() -> null)); @@ -248,15 +257,15 @@ public void testInvokeAllDoesNotLeaveTasksBehind() { for (int i = 0; i < tasksWithNormalExit; i++) { callables.add( () -> { - tasksExecuted.incrementAndGet(); - return null; + throw new AssertionError( + "must not be called since the first task failing cancels all subsequent tasks"); }); } expectThrows(RuntimeException.class, () -> taskExecutor.invokeAll(callables)); assertEquals(1, tasksExecuted.get()); // the callables are technically all run, but the cancelled ones will be no-op // add one for the task the gets executed on the current thread - assertEquals(100, tasksStarted.get() + 1); + assertEquals(tasksWithNormalExit, tasksStarted.get()); } /** From 18e66a8c6e0827911998d9132f3f498543f2b824 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 2 Jul 2024 18:16:01 +0200 Subject: [PATCH 6/8] enhance tests --- .../apache/lucene/search/TestTaskExecutor.java | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java b/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java index 0721d234ef78..11f9057cbdb2 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java @@ -108,9 +108,16 @@ public void testUnwrappedExceptions() { assertEquals("exc", runtimeException.getCause().getMessage()); } - public void testInvokeAllFromTaskDoesNotDeadlockSameSearcher() throws IOException { + public void testInvokeAllFromTaskDoesNotDeadlockSameSearcher() throws Exception { doTestInvokeAllFromTaskDoesNotDeadlockSameSearcher(executorService); doTestInvokeAllFromTaskDoesNotDeadlockSameSearcher(Runnable::run); + executorService + .submit( + () -> { + doTestInvokeAllFromTaskDoesNotDeadlockSameSearcher(executorService); + return null; + }) + .get(); } private static void doTestInvokeAllFromTaskDoesNotDeadlockSameSearcher(Executor executor) @@ -175,9 +182,16 @@ public Void reduce(Collection collectors) { } } - public void testInvokeAllFromTaskDoesNotDeadlockMultipleSearchers() throws IOException { + public void testInvokeAllFromTaskDoesNotDeadlockMultipleSearchers() throws Exception { doTestInvokeAllFromTaskDoesNotDeadlockMultipleSearchers(executorService); doTestInvokeAllFromTaskDoesNotDeadlockMultipleSearchers(Runnable::run); + executorService + .submit( + () -> { + doTestInvokeAllFromTaskDoesNotDeadlockMultipleSearchers(executorService); + return null; + }) + .get(); } private static void doTestInvokeAllFromTaskDoesNotDeadlockMultipleSearchers(Executor executor) From 6baf3dcd8db26a2be7251d42bf26de85d4c27e85 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 3 Jul 2024 17:14:23 +0200 Subject: [PATCH 7/8] CR --- lucene/CHANGES.txt | 10 +++++----- .../org/apache/lucene/search/TestTaskExecutor.java | 1 - 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 530eb81806fa..4db963e4caee 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -280,11 +280,11 @@ Optimizations Changes in runtime behavior --------------------- -* GITHUB#13472: IndexSearcher now executes tasks on the thread that invoked a search as well as its configured - executor. Users that invoke searches from threads not belonging to the executor configured in IndexSearcher should - reduce the executor's thread-count by 1 to retain the previous level of parallelism. Searches invoked from a thread - not part of the IndexSearcher's executor will now complete on the calling thread in case the executor is busy or - blocked and unable to complete any work. +* GITHUB#13472: When an executor is provided to the IndexSearcher constructor, the searcher now executes tasks on the + thread that invoked a search as well as its configured executor. Users should reduce the executor's thread-count by 1 + to retain the previous level of parallelism. Moreover, it is now possible to start searches from the same executor + that is configured in the IndexSearcher without risk of deadlocking. A separate executor for starting searches is no + longer required. Bug Fixes --------------------- diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java b/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java index 11f9057cbdb2..1949afbd51cd 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java @@ -278,7 +278,6 @@ public void testInvokeAllDoesNotLeaveTasksBehind() { expectThrows(RuntimeException.class, () -> taskExecutor.invokeAll(callables)); assertEquals(1, tasksExecuted.get()); // the callables are technically all run, but the cancelled ones will be no-op - // add one for the task the gets executed on the current thread assertEquals(tasksWithNormalExit, tasksStarted.get()); } From e2bf7aa2fe53bf46797e32545441683e3f318b7a Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 4 Jul 2024 10:29:07 +0200 Subject: [PATCH 8/8] add name :) --- lucene/CHANGES.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 4db963e4caee..74ff491b7817 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -284,7 +284,7 @@ Changes in runtime behavior thread that invoked a search as well as its configured executor. Users should reduce the executor's thread-count by 1 to retain the previous level of parallelism. Moreover, it is now possible to start searches from the same executor that is configured in the IndexSearcher without risk of deadlocking. A separate executor for starting searches is no - longer required. + longer required. (Armin Braun) Bug Fixes ---------------------