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

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


The following commit(s) were added to refs/heads/elasticity by this push:
     new 076b091a33 fixes split exceptions (#4720)
076b091a33 is described below

commit 076b091a3393feab88346e27c9158587a2dddd21
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Tue Jul 2 09:57:22 2024 -0700

    fixes split exceptions (#4720)
    
    This commit restores behavior for wrapping exceptions in background threads 
when adding splits.  It restores code that used to run when using Futures and 
makes the prior code work with CompletableFuture.
---
 .../core/clientImpl/TableOperationsImpl.java       | 37 ++++++++++++++++++++--
 1 file changed, 34 insertions(+), 3 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
index 780177d68e..43b108e1d1 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
@@ -68,6 +68,7 @@ import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
@@ -525,7 +526,9 @@ public class TableOperationsImpl extends 
TableOperationsHelper {
             }, tableName);
           } catch (TableExistsException | NamespaceExistsException | 
NamespaceNotFoundException
               | AccumuloSecurityException | TableNotFoundException | 
AccumuloException e) {
-            throw new RuntimeException(e);
+            // This exception type is used because it makes it easier in the 
foreground thread to do
+            // exception analysis when using CompletableFuture.
+            throw new CompletionException(e);
           }
           // wait for the fate operation to complete in a separate thread pool
         }, startExecutor).thenApplyAsync(pair -> {
@@ -540,7 +543,9 @@ public class TableOperationsImpl extends 
TableOperationsHelper {
             }
           } catch (TableExistsException | NamespaceExistsException | 
NamespaceNotFoundException
               | AccumuloSecurityException | TableNotFoundException | 
AccumuloException e) {
-            throw new RuntimeException(e);
+            // This exception type is used because it makes it easier in the 
foreground thread to do
+            // exception analysis when using CompletableFuture.
+            throw new CompletionException(e);
           } finally {
             // always finish table op, even when exception
             if (opid != null) {
@@ -556,7 +561,33 @@ public class TableOperationsImpl extends 
TableOperationsHelper {
         futures.add(future);
       }
 
-      futures.forEach(CompletableFuture::join);
+      try {
+        futures.forEach(CompletableFuture::join);
+      } catch (CompletionException ee) {
+        Throwable excep = ee.getCause();
+        // Below all exceptions are wrapped and rethrown. This is done so that 
the user knows
+        // what code path got them here. If the wrapping was not done, the 
user would only
+        // have the stack trace for the background thread.
+        if (excep instanceof TableNotFoundException) {
+          TableNotFoundException tnfe = (TableNotFoundException) excep;
+          throw new TableNotFoundException(tableId.canonical(), tableName,
+              "Table not found by background thread", tnfe);
+        } else if (excep instanceof TableOfflineException) {
+          log.debug("TableOfflineException occurred in background thread. 
Throwing new exception",
+              excep);
+          throw new TableOfflineException(tableId, tableName);
+        } else if (excep instanceof AccumuloSecurityException) {
+          // base == background accumulo security exception
+          AccumuloSecurityException base = (AccumuloSecurityException) excep;
+          throw new AccumuloSecurityException(base.getUser(), 
base.asThriftException().getCode(),
+              base.getTableInfo(), excep);
+        } else if (excep instanceof AccumuloServerException) {
+          throw new AccumuloServerException((AccumuloServerException) excep);
+        } else {
+          throw new AccumuloException(excep);
+        }
+      }
+
     }
     startExecutor.shutdown();
     waitExecutor.shutdown();

Reply via email to