uschindler commented on code in PR #12569:
URL: https://github.com/apache/lucene/pull/12569#discussion_r1330347382


##########
lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java:
##########
@@ -22,18 +22,28 @@
 import java.util.Collection;
 import java.util.List;
 import java.util.Objects;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Future;
-import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.FutureTask;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
  * Executor wrapper responsible for the execution of concurrent tasks. Used to 
parallelize search
- * across segments as well as query rewrite in some cases.
+ * across segments as well as query rewrite in some cases. Exposes a {@link 
#createTask(Callable)}
+ * method to create tasks given a {@link Callable}, as well as the {@link 
#invokeAll(Collection)}
+ * method to execute a set of tasks 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 executors, as well as to 
limit the level of
+ * parallelism.
  */
 class TaskExecutor {
+  private static final ThreadLocal<Boolean> isConcurrentTask = 
ThreadLocal.withInitial(() -> false);

Review Comment:
   Hi, the latest commit looks fine, as we have at least not the binary thread 
local anymore. 
   
   Small suggestion: Do we have a MutableInt class availble in Lucene? It would 
make it easier to decrement/increment. An alternative is to use 
`ThreadLocal<int[]>` with a one-length array. This would also prevent 
autoboxing. Just initialize with:
   
   ```java
   private static final ThreadLocal<int[]> runSameThread = 
ThreadLocal.withInitial(() -> new int[1]);
   ``` 
   
   and use like this:
   
   ```java
   final int[] counter = runSameThread.get(); 
   counter[0]++;
   try {
   ....
   } finally {
    counter[0]--;
   }
   ```
   
   My problem was mainly if external code like Elasticsearch passes a shared 
thread pool to multiple IndexSearchers (like different indexes on same node 
using same "searcher" thread pool).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org

Reply via email to