This is an automated email from the ASF dual-hosted git repository.

kturner pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/2.1 by this push:
     new aeddc583b1 Workaround flaw in ScheduledThreadPoolExecutor.execute() 
(#6068)
aeddc583b1 is described below

commit aeddc583b125eaa07361ec990d6b8ac7eca0cdb5
Author: Keith Turner <[email protected]>
AuthorDate: Wed Jan 21 07:31:23 2026 -0800

    Workaround flaw in ScheduledThreadPoolExecutor.execute() (#6068)
---
 .../accumulo/core/util/threads/ThreadPools.java    | 12 ++++-
 .../core/util/threads/ThreadPoolsTest.java         | 60 ++++++++++++++++++++++
 2 files changed, 71 insertions(+), 1 deletion(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java 
b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
index 22e5c7491c..376a192ce3 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
@@ -667,7 +667,17 @@ public class ThreadPools {
 
           @Override
           public void execute(@NonNull Runnable command) {
-            super.execute(TraceUtil.wrap(command));
+            // ScheduledThreadPoolExecutor.execute() will internally create a 
future that is not
+            // returned. This inaccessible future will silently eat uncaught 
exceptions. This code
+            // is a workaround for this behavior that avoids completely losing 
exceptions.
+            var wrapped = TraceUtil.wrap(command);
+            super.execute(() -> {
+              try {
+                wrapped.run();
+              } catch (Throwable t) {
+                handler.uncaughtException(Thread.currentThread(), t);
+              }
+            });
           }
 
           @Override
diff --git 
a/core/src/test/java/org/apache/accumulo/core/util/threads/ThreadPoolsTest.java 
b/core/src/test/java/org/apache/accumulo/core/util/threads/ThreadPoolsTest.java
new file mode 100644
index 0000000000..64059fa461
--- /dev/null
+++ 
b/core/src/test/java/org/apache/accumulo/core/util/threads/ThreadPoolsTest.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util.threads;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.junit.jupiter.api.Test;
+
+public class ThreadPoolsTest {
+  @Test
+  public void testExecuteUncaught() throws InterruptedException {
+
+    var tq = new LinkedBlockingQueue<Throwable>();
+    Thread.UncaughtExceptionHandler ueh = (thread, throwable) -> {
+      tq.add(throwable);
+    };
+    var threadPools = 
ThreadPools.getClientThreadPools(DefaultConfiguration.getInstance(), ueh);
+
+    var pool1 = threadPools.getPoolBuilder("test").build();
+    var pool2 = threadPools.createScheduledExecutorService(1, "test2");
+    int i = 0;
+    // Test a normal thread pool and scheduled thread pool
+    for (var pool : List.of(pool1, pool2)) {
+      assertTrue(tq.isEmpty());
+
+      String msg = "msg" + i;
+
+      // ensure this error makes its way to the uncaught exception handler 
configured for the pool
+      pool.execute(() -> {
+        throw new Error(msg);
+      });
+
+      var throwable1 = tq.take();
+      assertEquals(msg, throwable1.getMessage());
+      assertEquals(Error.class, throwable1.getClass());
+      i++;
+    }
+  }
+}

Reply via email to