You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2022/03/08 20:55:41 UTC

[GitHub] [accumulo] dlmarion opened a new pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

dlmarion opened a new pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554


   When an ExecutorService's task or Thread in Accumulo encounter an
   unhandled exception the default UncaughtExceptionHandler will be
   invoked. An Exception is logged, while an Error is logged and then
   the VM is terminated. This change allows clients to override the
   default uncaught exception handling logic so that the client VM is
   not terminated when an Error happens in a background thread of a
   client object.
   
   Closes #2331 and supersedes #2346


-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r824820683



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,13 +175,48 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    String uehClassName = serverConf.get(ClientProperty.UNCAUGHT_EXCEPTION_HANDLER.getKey());

Review comment:
       @ctubbsii - Thanks for the explanation, that helped. I'm about to push a commit that:
   
   1. addresses your comments above about lazy initialization of the thread pools
   2. removes the UEH handler property entirely
   3. Sets the UEH to `Threads.UEH` when `ClientContext` is initialized from `ServerContext`
   4. Allows the client to set a UEH class on the AccumuloClient when building it
   5. Uses a default UEH in the client if one is not supplied that logs the error. This should provide some backwards compatibility for clients so the client VM does not get terminated.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion merged pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion merged pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554


   


-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r823947265



##########
File path: core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
##########
@@ -65,8 +66,18 @@ public ExecutionError(String message, Throwable cause) {
   // the number of seconds before we allow a thread to terminate with non-use.
   public static final long DEFAULT_TIMEOUT_MILLISECS = 180000L;
 
+  private static final ThreadPools SERVER_INSTANCE = new ThreadPools(Threads.UEH);
+
+  public static final ThreadPools getServerThreadPools() {
+    return SERVER_INSTANCE;
+  }

Review comment:
       It is. Thanks.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r823703931



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -729,14 +787,18 @@ public AuthenticationToken token() {
 
   @Override
   public void close() {
-    closed = true;
+    closed.compareAndSet(false, true);
     if (thriftTransportPool != null) {
       thriftTransportPool.shutdown();
     }
     if (tableZooHelper != null) {
       tableZooHelper.close();
     }
     singletonReservation.close();
+    this.scannerReadaheadPool.shutdownNow(); // abort all tasks, client is shutting down
+    this.scannerPoolCleaner.clean();
+    this.cleanupThreadPool.shutdown(); // wait for shutdown tasks to execute
+    this.cleaner.clean();

Review comment:
       Addressed in 180654d




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r825846960



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -736,6 +791,12 @@ public void close() {
     if (tableZooHelper != null) {
       tableZooHelper.close();
     }
+    if (scannerReadaheadPool != null) {

Review comment:
       I synchronized the `close` method in 47e90dca559f02683c7c9f58ae366a422367a2d7




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r822953740



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,6 +168,19 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    String uehClassName = serverConf.get(ClientProperty.UNCAUGHT_EXCEPTION_HANDLER.getKey());
+    if (!StringUtils.isEmpty(uehClassName)) {
+      try {
+        @SuppressWarnings("unchecked")
+        Class<? extends UncaughtExceptionHandler> clazz =
+            (Class<? extends UncaughtExceptionHandler>) Class.forName(uehClassName);
+        Threads.setUncaughtExceptionHandler(clazz.getDeclaredConstructor().newInstance());
+      } catch (ClassNotFoundException | InstantiationException | IllegalAccessException
+          | IllegalArgumentException | InvocationTargetException | NoSuchMethodException
+          | SecurityException e) {

Review comment:
       Addressed in 16bc28c

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,6 +168,19 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    String uehClassName = serverConf.get(ClientProperty.UNCAUGHT_EXCEPTION_HANDLER.getKey());
+    if (!StringUtils.isEmpty(uehClassName)) {
+      try {
+        @SuppressWarnings("unchecked")
+        Class<? extends UncaughtExceptionHandler> clazz =
+            (Class<? extends UncaughtExceptionHandler>) Class.forName(uehClassName);

Review comment:
       Addressed in 16bc28c

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,6 +168,19 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    String uehClassName = serverConf.get(ClientProperty.UNCAUGHT_EXCEPTION_HANDLER.getKey());
+    if (!StringUtils.isEmpty(uehClassName)) {
+      try {
+        @SuppressWarnings("unchecked")
+        Class<? extends UncaughtExceptionHandler> clazz =
+            (Class<? extends UncaughtExceptionHandler>) Class.forName(uehClassName);
+        Threads.setUncaughtExceptionHandler(clazz.getDeclaredConstructor().newInstance());
+      } catch (ClassNotFoundException | InstantiationException | IllegalAccessException
+          | IllegalArgumentException | InvocationTargetException | NoSuchMethodException
+          | SecurityException e) {
+        throw new RuntimeException("Error setting uncaughtExceptionHandler", e);

Review comment:
       Addressed in 16bc28c




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r823073519



##########
File path: core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java
##########
@@ -63,7 +64,12 @@ public DefaultContextClassLoaderFactory(final AccumuloConfiguration accConf) {
 
   private static void startCleanupThread(final AccumuloConfiguration conf,
       final Supplier<Map<String,String>> contextConfigSupplier) {
-    ScheduledFuture<?> future = ThreadPools.createGeneralScheduledExecutorService(conf)
+    ScheduledFuture<?> future = ThreadPools.getClientThreadPools(new UncaughtExceptionHandler() {
+      @Override
+      public void uncaughtException(Thread arg0, Throwable arg1) {
+        LOG.error("context classloader cleanup thread has failed.", arg1);
+      }
+    }).createGeneralScheduledExecutorService(conf)

Review comment:
       Addressed in ec0e7cf




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r823696044



##########
File path: core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
##########
@@ -65,8 +66,18 @@ public ExecutionError(String message, Throwable cause) {
   // the number of seconds before we allow a thread to terminate with non-use.
   public static final long DEFAULT_TIMEOUT_MILLISECS = 180000L;
 
+  private static final ThreadPools SERVER_INSTANCE = new ThreadPools(Threads.UEH);
+
+  public static final ThreadPools getServerThreadPools() {
+    return SERVER_INSTANCE;
+  }

Review comment:
       I see the something in SingletonReservation that closes AccumuloClients if that is what you are referencing. I will remove the Cleaners from ClientContext.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r825489678



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -736,6 +791,12 @@ public void close() {
     if (tableZooHelper != null) {
       tableZooHelper.close();
     }
+    if (scannerReadaheadPool != null) {

Review comment:
       @keith-turner  Do you believe this is an existing problem with the thriftTransportPool also?




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r825976255



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,13 +176,51 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    if (ueh == Threads.UEH) {
+      this.ueh = ueh;
+      clientThreadPools = ThreadPools.getServerThreadPools();
+    } else {
+      // Provide a default UEH that just logs the error
+      if (ueh == null) {
+        this.ueh = (t, e) -> {
+          log.error("Caught an Exception in client background thread: {}. Thread is dead.", t, e);
+        };
+      } else {
+        this.ueh = ueh;
+      }
+      clientThreadPools = ThreadPools.getClientThreadPools(ueh);
+    }
   }
 
   public Ample getAmple() {
     ensureOpen();
     return new AmpleImpl(this);
   }
 
+  public synchronized Future<List<KeyValue>>
+      submitScannerReadAheadTask(Callable<List<KeyValue>> c) {
+    if (scannerReadaheadPool == null) {

Review comment:
       Addressed in a6236ae

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,13 +176,51 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    if (ueh == Threads.UEH) {
+      this.ueh = ueh;
+      clientThreadPools = ThreadPools.getServerThreadPools();
+    } else {
+      // Provide a default UEH that just logs the error
+      if (ueh == null) {
+        this.ueh = (t, e) -> {
+          log.error("Caught an Exception in client background thread: {}. Thread is dead.", t, e);
+        };
+      } else {
+        this.ueh = ueh;
+      }
+      clientThreadPools = ThreadPools.getClientThreadPools(ueh);
+    }
   }
 
   public Ample getAmple() {
     ensureOpen();
     return new AmpleImpl(this);
   }
 
+  public synchronized Future<List<KeyValue>>
+      submitScannerReadAheadTask(Callable<List<KeyValue>> c) {
+    if (scannerReadaheadPool == null) {
+      scannerReadaheadPool = clientThreadPools.createThreadPool(0, Integer.MAX_VALUE, 3L, SECONDS,
+          "Accumulo scanner read ahead thread", new SynchronousQueue<>(), true);
+    }
+    return scannerReadaheadPool.submit(c);
+  }
+
+  public synchronized void executeCleanupTask(Runnable r) {
+    if (cleanupThreadPool == null) {

Review comment:
       Addressed in a6236ae




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r822903804



##########
File path: core/src/main/java/org/apache/accumulo/core/util/threads/Threads.java
##########
@@ -25,6 +25,8 @@
 
 public class Threads {
 
+  private static UncaughtExceptionHandler UEH = new AccumuloUncaughtExceptionHandler();

Review comment:
       I think that other PR was trying to do a lot more than what I'm suggesting here, which is just the minimal amount of work to make Threads non-static and dangling off of ClientContext, so it can be have a per-client UEH. Yes, it would be larger than this PR. Maybe it could be done as a prerequisite, or even as a follow-on, to this one. But, I think the limitation of having one client control the UEH for all clients is substantial. Other than that limitation, I really like the direction this change took.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r824176972



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -737,6 +783,8 @@ public void close() {
       tableZooHelper.close();
     }
     singletonReservation.close();
+    this.scannerReadaheadPool.shutdownNow(); // abort all tasks, client is shutting down
+    this.cleanupThreadPool.shutdown(); // wait for shutdown tasks to execute

Review comment:
       I was referring to `thriftTransportPool`, a few lines above these. This class has evolved since I originally wrote that javadoc. While that was the original intent, I'm not sure it's entirely necessary or practical given how much state we're using in here. What is most important is that everything is available in the sense that no additional API calls are necessary to further initialize the state, that all of its functionality is ready after constructing. Lazy initialization would still be compatible with that looser definition of availability.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r822985372



##########
File path: core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java
##########
@@ -63,7 +64,12 @@ public DefaultContextClassLoaderFactory(final AccumuloConfiguration accConf) {
 
   private static void startCleanupThread(final AccumuloConfiguration conf,
       final Supplier<Map<String,String>> contextConfigSupplier) {
-    ScheduledFuture<?> future = ThreadPools.createGeneralScheduledExecutorService(conf)
+    ScheduledFuture<?> future = ThreadPools.getClientThreadPools(new UncaughtExceptionHandler() {
+      @Override
+      public void uncaughtException(Thread arg0, Throwable arg1) {
+        LOG.error("context classloader cleanup thread has failed.", arg1);
+      }
+    }).createGeneralScheduledExecutorService(conf)

Review comment:
       Formatting is probably bad, but should be possible to use a lambda here.
   
   ```suggestion
       ScheduledFuture<?> future = ThreadPools.getClientThreadPools((t, e) ->
           LOG.error("context classloader cleanup thread has failed.", e)).createGeneralScheduledExecutorService(conf)
   ```

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -65,11 +67,10 @@
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool =
-      ThreadPools.createThreadPool(0, Integer.MAX_VALUE, 3L, SECONDS,
-          "Accumulo scanner read ahead thread", new SynchronousQueue<>(), true);
+  private final ThreadPoolExecutor readaheadPool;

Review comment:
       Same as my previous comment, this could be moved to ClientContext and cleaned up on `client.close()`. No need for the Cleaner.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
##########
@@ -365,8 +371,12 @@ private void reschedule(SendTask task) {
     this.context = context;
     this.auths = config.getAuthorizations();
     this.ve = new VisibilityEvaluator(config.getAuthorizations());
-    this.threadPool = ThreadPools.createScheduledExecutorService(config.getMaxWriteThreads(),
-        this.getClass().getSimpleName(), false);
+    this.threadPool = context.getClientThreadPools().createScheduledExecutorService(
+        config.getMaxWriteThreads(), this.getClass().getSimpleName(), false);
+    this.cleanupThreadPool = context.getClientThreadPools().createFixedThreadPool(1, 3, SECONDS,
+        "Conditional Writer Cleanup Thread", true);
+    this.cleaner = CleanerUtil.shutdownThreadPoolExecutor(cleanupThreadPool, closed,
+        LoggerFactory.getLogger(ConditionalWriterImpl.class));

Review comment:
       This cleanupThreadPool would probably be better left in ClientContext, and shut down when that is closed, rather than store it as an instance member here, where it's only ever used in the close method of this instance. It's very weird to *start* using resources for an object in that object's close method, which should be cleaning up resources. Putting it in ClientContext makes this effectively a singleton again (per Client, not per JVM, though, but that's still better than per-ConditionalWriter) and its own cleanup will be a bit more sane there, I think.

##########
File path: core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
##########
@@ -65,8 +66,18 @@ public ExecutionError(String message, Throwable cause) {
   // the number of seconds before we allow a thread to terminate with non-use.
   public static final long DEFAULT_TIMEOUT_MILLISECS = 180000L;
 
+  private static final ThreadPools SERVER_INSTANCE = new ThreadPools(Threads.UEH);
+
+  public static final ThreadPools getServerThreadPools() {
+    return SERVER_INSTANCE;
+  }

Review comment:
       I don't know if it's easy to do in all cases, but I was thinking that instead of this, it can share the same sort of code as for the client-side cases.
   
   Essentially, ClientContext would have a `.threadPools()` method that returns an instance of `ThreadPools` constructed using the UEH provided when AccumuloClient was built (default if not provided). When ServerContext is constructed, it just passes the built-in UEH for the server-side code to ClientContext's constructor in `super()`.
   
   So, instead of calling `ThreadPools.getServerThreadPools().whatever`, you'd just use `context.threadPools().whatever` everywhere, regardless of client or server context. That way, there's no static instance of ThreadPools anywhere, but everything still works as expected. (`context.threadPools()` is also shorter than `ThreadPools.getServerThreadPools()`)




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r823047575



##########
File path: core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
##########
@@ -65,8 +66,18 @@ public ExecutionError(String message, Throwable cause) {
   // the number of seconds before we allow a thread to terminate with non-use.
   public static final long DEFAULT_TIMEOUT_MILLISECS = 180000L;
 
+  private static final ThreadPools SERVER_INSTANCE = new ThreadPools(Threads.UEH);
+
+  public static final ThreadPools getServerThreadPools() {
+    return SERVER_INSTANCE;
+  }

Review comment:
       Won't they still need a Cleaner in the case where the client is not closed?




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r822955189



##########
File path: core/src/main/java/org/apache/accumulo/core/util/threads/Threads.java
##########
@@ -25,6 +25,8 @@
 
 public class Threads {
 
+  private static UncaughtExceptionHandler UEH = new AccumuloUncaughtExceptionHandler();

Review comment:
       I was able to pull some of the ideas from #2346 into this PR. 16bc28c should address the issue that you raised here.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r825847106



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -121,7 +129,7 @@
   private TCredentials rpcCreds;
   private ThriftTransportPool thriftTransportPool;
 
-  private volatile boolean closed = false;

Review comment:
       Addressed in 47e90dca559f02683c7c9f58ae366a422367a2d7




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r823073918



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -65,11 +67,10 @@
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool =
-      ThreadPools.createThreadPool(0, Integer.MAX_VALUE, 3L, SECONDS,
-          "Accumulo scanner read ahead thread", new SynchronousQueue<>(), true);
+  private final ThreadPoolExecutor readaheadPool;

Review comment:
       Addressed in ec0e7cf




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r825489427



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,13 +176,51 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    if (ueh == Threads.UEH) {
+      this.ueh = ueh;
+      clientThreadPools = ThreadPools.getServerThreadPools();
+    } else {
+      // Provide a default UEH that just logs the error
+      if (ueh == null) {
+        this.ueh = (t, e) -> {
+          log.error("Caught an Exception in client background thread: {}. Thread is dead.", t, e);
+        };
+      } else {
+        this.ueh = ueh;
+      }
+      clientThreadPools = ThreadPools.getClientThreadPools(ueh);
+    }
   }
 
   public Ample getAmple() {
     ensureOpen();
     return new AmpleImpl(this);
   }
 
+  public synchronized Future<List<KeyValue>>
+      submitScannerReadAheadTask(Callable<List<KeyValue>> c) {
+    if (scannerReadaheadPool == null) {
+      scannerReadaheadPool = clientThreadPools.createThreadPool(0, Integer.MAX_VALUE, 3L, SECONDS,
+          "Accumulo scanner read ahead thread", new SynchronousQueue<>(), true);
+    }

Review comment:
       @keith-turner I believe that's what he did in a previous iteration, but I suggested he lazily instantiate, to follow the convention established with the thriftTransportPool which avoids allocating resources which might not be used by the context.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r824040031



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -737,6 +783,8 @@ public void close() {
       tableZooHelper.close();
     }
     singletonReservation.close();
+    this.scannerReadaheadPool.shutdownNow(); // abort all tasks, client is shutting down
+    this.cleanupThreadPool.shutdown(); // wait for shutdown tasks to execute

Review comment:
       I can move the `close()` calls up. Regarding lazy instantiation, what other thread pools were you referencing? The class javadoc says `Any state in this object should be available at the time of its construction.`, does that on mean configuration?




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r825098028



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,13 +176,51 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    if (ueh == Threads.UEH) {
+      this.ueh = ueh;
+      clientThreadPools = ThreadPools.getServerThreadPools();
+    } else {
+      // Provide a default UEH that just logs the error
+      if (ueh == null) {
+        this.ueh = (t, e) -> {
+          log.error("Caught an Exception in client background thread: {}. Thread is dead.", t, e);
+        };
+      } else {
+        this.ueh = ueh;
+      }
+      clientThreadPools = ThreadPools.getClientThreadPools(ueh);
+    }
   }
 
   public Ample getAmple() {
     ensureOpen();
     return new AmpleImpl(this);
   }
 
+  public synchronized Future<List<KeyValue>>
+      submitScannerReadAheadTask(Callable<List<KeyValue>> c) {
+    if (scannerReadaheadPool == null) {
+      scannerReadaheadPool = clientThreadPools.createThreadPool(0, Integer.MAX_VALUE, 3L, SECONDS,
+          "Accumulo scanner read ahead thread", new SynchronousQueue<>(), true);
+    }

Review comment:
       Could consider creating this in the constructor, then would not need to synchronize the method.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -121,7 +129,7 @@
   private TCredentials rpcCreds;
   private ThriftTransportPool thriftTransportPool;
 
-  private volatile boolean closed = false;

Review comment:
       Why was volatile removed?  W/o volatile or synchronization, changes may not be seen.  Like `ensureOpen()` may not see changes.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -736,6 +791,12 @@ public void close() {
     if (tableZooHelper != null) {
       tableZooHelper.close();
     }
+    if (scannerReadaheadPool != null) {

Review comment:
       Either need to make this variable volatile or synchronize this method to ensure changes to the variable made by different threads are seen.

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -127,7 +127,7 @@
   protected CompactionCoordinator(ServerOpts opts, String[] args) {
     super("compaction-coordinator", opts, args);
     aconf = getConfiguration();
-    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    schedExecutor = ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(aconf);

Review comment:
       Curious if the following would work? The super class constructor creates a ServerContext object that would be returned by getContext().  It seems like SeverContext.threadPools() will return ThreadPools.getServerThreadPools().   
   
   ```suggestion
       schedExecutor = getContext().threadPools().createGeneralScheduledExecutorService(aconf);
   ```




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r824050402



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,13 +175,48 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    String uehClassName = serverConf.get(ClientProperty.UNCAUGHT_EXCEPTION_HANDLER.getKey());

Review comment:
       I think I'm misunderstanding something as I'm not familiar with the new Accumulo client in 2.x. Does the Accumulo client in 2.x use the `accumulo-client.properties` file? I was thinking that it did, now I'm thinking that it does not. Is that file only used for the shell and utilities?




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r825975811



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -187,6 +237,11 @@ public String getPrincipal() {
     return getCredentials().getPrincipal();
   }
 
+  public UncaughtExceptionHandler getUncaughtExceptionHandler() {
+    ensureOpen();
+    return ueh;
+  }
+

Review comment:
       Method removed in a6236ae




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r825828933



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,13 +176,51 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    if (ueh == Threads.UEH) {
+      this.ueh = ueh;
+      clientThreadPools = ThreadPools.getServerThreadPools();
+    } else {
+      // Provide a default UEH that just logs the error
+      if (ueh == null) {
+        this.ueh = (t, e) -> {
+          log.error("Caught an Exception in client background thread: {}. Thread is dead.", t, e);
+        };
+      } else {
+        this.ueh = ueh;
+      }
+      clientThreadPools = ThreadPools.getClientThreadPools(ueh);
+    }
   }
 
   public Ample getAmple() {
     ensureOpen();
     return new AmpleImpl(this);
   }
 
+  public synchronized Future<List<KeyValue>>
+      submitScannerReadAheadTask(Callable<List<KeyValue>> c) {
+    if (scannerReadaheadPool == null) {
+      scannerReadaheadPool = clientThreadPools.createThreadPool(0, Integer.MAX_VALUE, 3L, SECONDS,
+          "Accumulo scanner read ahead thread", new SynchronousQueue<>(), true);
+    }

Review comment:
       That's correct.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r825828309



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -127,7 +127,7 @@
   protected CompactionCoordinator(ServerOpts opts, String[] args) {
     super("compaction-coordinator", opts, args);
     aconf = getConfiguration();
-    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    schedExecutor = ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(aconf);

Review comment:
       That would work in the cases where ServerContext is reachable. They are functionally equivalent.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r826473186



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,13 +176,51 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    if (ueh == Threads.UEH) {
+      this.ueh = ueh;
+      clientThreadPools = ThreadPools.getServerThreadPools();
+    } else {
+      // Provide a default UEH that just logs the error
+      if (ueh == null) {
+        this.ueh = (t, e) -> {
+          log.error("Caught an Exception in client background thread: {}. Thread is dead.", t, e);
+        };
+      } else {
+        this.ueh = ueh;
+      }
+      clientThreadPools = ThreadPools.getClientThreadPools(ueh);
+    }
   }
 
   public Ample getAmple() {
     ensureOpen();
     return new AmpleImpl(this);
   }
 
+  public synchronized Future<List<KeyValue>>
+      submitScannerReadAheadTask(Callable<List<KeyValue>> c) {
+    if (scannerReadaheadPool == null) {
+      scannerReadaheadPool = clientThreadPools.createThreadPool(0, Integer.MAX_VALUE, 3L, SECONDS,
+          "Accumulo scanner read ahead thread", new SynchronousQueue<>(), true);
+    }

Review comment:
       As long as closed() is synchronized, that addresses the main concern I had.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r826471905



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -127,7 +127,7 @@
   protected CompactionCoordinator(ServerOpts opts, String[] args) {
     super("compaction-coordinator", opts, args);
     aconf = getConfiguration();
-    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    schedExecutor = ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(aconf);

Review comment:
       If we can use the context method instead of the static method that would be nice, but that can be follow on work.  




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r823946741



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,13 +175,48 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    String uehClassName = serverConf.get(ClientProperty.UNCAUGHT_EXCEPTION_HANDLER.getKey());

Review comment:
       The way this is being read from the config is a bit weird. It's reading a ClientProperty from an AccumuloConfiguration, which means that the client property would have had to exist in the `accumulo.properties` file, but not documented as a server-side property. The property is explicitly named `client.uncaught.exception.handler`, but the way it is loaded, it will set it for the server-side as well. I think this would be cleaner if it were renamed to `instance.uncaught.exception.handler` and put in `Property.java` with other server properties. You could even use ClassLoaderUtil directly, or via ConfigurationTypeHelper to load the class from the configuration.
   
   If you wanted a separate client-side property name, you could do that (see how ClientConfConverter translates ZooKeeper host names, for an example), but I don't think that'd be necessary.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -737,6 +783,8 @@ public void close() {
       tableZooHelper.close();
     }
     singletonReservation.close();
+    this.scannerReadaheadPool.shutdownNow(); // abort all tasks, client is shutting down
+    this.cleanupThreadPool.shutdown(); // wait for shutdown tasks to execute

Review comment:
       These should be ordered before the `singletonReservation.close()` call above them. Also, I noticed the other thread pools held by this class are lazily instantiated, so could be null. Should these be lazily instantiated as well? It would save some resources when using a client for things that doesn't require them.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r823019144



##########
File path: core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
##########
@@ -65,8 +66,18 @@ public ExecutionError(String message, Throwable cause) {
   // the number of seconds before we allow a thread to terminate with non-use.
   public static final long DEFAULT_TIMEOUT_MILLISECS = 180000L;
 
+  private static final ThreadPools SERVER_INSTANCE = new ThreadPools(Threads.UEH);
+
+  public static final ThreadPools getServerThreadPools() {
+    return SERVER_INSTANCE;
+  }

Review comment:
       I don't know that in all cases server-side that the ServerContext is available to the code. I would have to wire it up in all cases where it's not reachable.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r823042217



##########
File path: core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
##########
@@ -65,8 +66,18 @@ public ExecutionError(String message, Throwable cause) {
   // the number of seconds before we allow a thread to terminate with non-use.
   public static final long DEFAULT_TIMEOUT_MILLISECS = 180000L;
 
+  private static final ThreadPools SERVER_INSTANCE = new ThreadPools(Threads.UEH);
+
+  public static final ThreadPools getServerThreadPools() {
+    return SERVER_INSTANCE;
+  }

Review comment:
       To be clear, only ClientContext would be necessary to wire up. ServerContext isn't available in the core jar. However, it is a subclass of ClientContext, so it would still work if only a ClientContext. But even then, ClientContext might not be available everywhere, and might need to be wired up... that's one of the things we've been fighting against to avoid static state in the JVM and the problems that come with it, so it's not a new issue. But, I was just hoping we could try to avoid a few additional occurrences of it that we'd have to fix later.
   
   In any case, that could be treated as a follow-on. I think the main things to address before this is merged in is to move the static thread pools that were converted to instance members for cleanup into ClientContext, so they exist in a Closeable object and don't need a separate Cleaner, and so they can be reused for the cleanup tasks they perform across multiple instances of the objects they clean up.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r824191645



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,13 +175,48 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    String uehClassName = serverConf.get(ClientProperty.UNCAUGHT_EXCEPTION_HANDLER.getKey());

Review comment:
       The client does use that file, but the server-side doesn't. This code is shared by both client and server code because `ClientContext` is the `Impl` class for `AccumuloClient`, and it is the class that is returned from the `AccumuloClient` builder. `ClientContext` is also the super-class of `ServerContext`, because servers can also perform client operations.
   
   The builder for `AccumuloClient` can initialize the configuration used in `ClientContext` from the client configuration file. However, the `ServerContext` initializes the configuration used in `ClientContext` from the server's `conf/accumulo.properties` file (which clients cannot be expected to have access to). Complicating the situation, we have two different versions of client properties, and have a translator tool to convert one into the other. And, to share code between the two use cases, we present the client configuration file to `ClientContext` in an `AccumuloConfiguration` view.
   
   Because this class is the super-class of `ServerContext`, we would need it to be able to read the value from the `conf/accumulo.properties` file for the server-side. Users can have the option of putting the same configuration property in their own `accumulo-client.properties` file, if they wish (and that's the main use case of this PR). For other properties where that happens, we try to keep the same name, so it's easier. However, for historical reasons, some of the properties, like the one for setting the ZooKeeper hosts, have a different property name that we translate if we see it.
   
   Yeah, it's complicated... but that's the trade-off for preserving deprecated functionality, properties, key names, etc. I would *love* to strip out the old stuff, and dramatically simplify a lot of this, but that has been resisted in the past (understandably). I'm hoping we can clean it up with a non-LTM 3.0 release before we start working on an LTM 3.x version.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r825885182



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -127,7 +127,7 @@
   protected CompactionCoordinator(ServerOpts opts, String[] args) {
     super("compaction-coordinator", opts, args);
     aconf = getConfiguration();
-    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    schedExecutor = ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(aconf);

Review comment:
       > That would work in the cases where ServerContext is reachable. They are functionally equivalent.
   
   While these are functionally equivalent now, it would save effort later, trying to wire in context where needed to get rid of the static references to these resources, in case we wanted to make the server-side UEH configurable at some point.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r825893967



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -736,6 +791,12 @@ public void close() {
     if (tableZooHelper != null) {
       tableZooHelper.close();
     }
+    if (scannerReadaheadPool != null) {

Review comment:
       I'm not sure it's necessary, since the `closed` variable is volatile again. As long as you add `ensureOpen();` to the two methods that retrieve the pools. That said, it's not like we're calling `close()` so frequently that synchronizing it is going to matter.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,13 +176,51 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    if (ueh == Threads.UEH) {
+      this.ueh = ueh;
+      clientThreadPools = ThreadPools.getServerThreadPools();
+    } else {
+      // Provide a default UEH that just logs the error
+      if (ueh == null) {
+        this.ueh = (t, e) -> {
+          log.error("Caught an Exception in client background thread: {}. Thread is dead.", t, e);
+        };
+      } else {
+        this.ueh = ueh;
+      }
+      clientThreadPools = ThreadPools.getClientThreadPools(ueh);
+    }
   }
 
   public Ample getAmple() {
     ensureOpen();
     return new AmpleImpl(this);
   }
 
+  public synchronized Future<List<KeyValue>>
+      submitScannerReadAheadTask(Callable<List<KeyValue>> c) {
+    if (scannerReadaheadPool == null) {

Review comment:
       ```suggestion
     public synchronized Future<List<KeyValue>>
         submitScannerReadAheadTask(Callable<List<KeyValue>> c) {
       ensureOpen();
       if (scannerReadaheadPool == null) {
   ```

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -187,6 +237,11 @@ public String getPrincipal() {
     return getCredentials().getPrincipal();
   }
 
+  public UncaughtExceptionHandler getUncaughtExceptionHandler() {
+    ensureOpen();
+    return ueh;
+  }
+

Review comment:
       Is this getter method ever used outside the constructor? If not, it can be removed, along with the field it gets.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,13 +176,51 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    if (ueh == Threads.UEH) {
+      this.ueh = ueh;
+      clientThreadPools = ThreadPools.getServerThreadPools();
+    } else {
+      // Provide a default UEH that just logs the error
+      if (ueh == null) {
+        this.ueh = (t, e) -> {
+          log.error("Caught an Exception in client background thread: {}. Thread is dead.", t, e);
+        };
+      } else {
+        this.ueh = ueh;
+      }
+      clientThreadPools = ThreadPools.getClientThreadPools(ueh);
+    }
   }
 
   public Ample getAmple() {
     ensureOpen();
     return new AmpleImpl(this);
   }
 
+  public synchronized Future<List<KeyValue>>
+      submitScannerReadAheadTask(Callable<List<KeyValue>> c) {
+    if (scannerReadaheadPool == null) {
+      scannerReadaheadPool = clientThreadPools.createThreadPool(0, Integer.MAX_VALUE, 3L, SECONDS,
+          "Accumulo scanner read ahead thread", new SynchronousQueue<>(), true);
+    }
+    return scannerReadaheadPool.submit(c);
+  }
+
+  public synchronized void executeCleanupTask(Runnable r) {
+    if (cleanupThreadPool == null) {

Review comment:
       ```suggestion
     public synchronized void executeCleanupTask(Runnable r) {
       ensureOpen();
       if (cleanupThreadPool == null) {
   ```




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r823085100



##########
File path: core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
##########
@@ -65,8 +66,18 @@ public ExecutionError(String message, Throwable cause) {
   // the number of seconds before we allow a thread to terminate with non-use.
   public static final long DEFAULT_TIMEOUT_MILLISECS = 180000L;
 
+  private static final ThreadPools SERVER_INSTANCE = new ThreadPools(Threads.UEH);
+
+  public static final ThreadPools getServerThreadPools() {
+    return SERVER_INSTANCE;
+  }

Review comment:
       I thought the client context already has a cleaner that ensures it is closed, because of the legacy Instance/Connector stuff that may not clean up the client context.
   
   However, even if it doesn't, Cleaner's (replacing finalizers) are a hack to close things during garbage collection, and are primarily useful for objects that don't explicitly get closed. For objects whose lifecycle is user-facing, like our Closeable clients, we don't need to specify a Cleaner... we (or the user) can just close them. Java helps us with this by giving us warnings about resource leaks and potential resource leaks for unclosed Closeable objects.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r823088380



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -729,14 +787,18 @@ public AuthenticationToken token() {
 
   @Override
   public void close() {
-    closed = true;
+    closed.compareAndSet(false, true);
     if (thriftTransportPool != null) {
       thriftTransportPool.shutdown();
     }
     if (tableZooHelper != null) {
       tableZooHelper.close();
     }
     singletonReservation.close();
+    this.scannerReadaheadPool.shutdownNow(); // abort all tasks, client is shutting down
+    this.scannerPoolCleaner.clean();
+    this.cleanupThreadPool.shutdown(); // wait for shutdown tasks to execute
+    this.cleaner.clean();

Review comment:
       I think these cleaners can go away, based on my previous comments.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r822666280



##########
File path: core/src/main/java/org/apache/accumulo/core/util/threads/Threads.java
##########
@@ -25,6 +25,8 @@
 
 public class Threads {
 
+  private static UncaughtExceptionHandler UEH = new AccumuloUncaughtExceptionHandler();

Review comment:
       Looking into this further, I think I want to just support a property in the client properties file and not on the AccumuloClient object. Everything in ThreadPools and Threads is static and I would need to change everything to be not-static and supply either a ClientContext or ServerContext object (which means I would have to change a ton more code). If we want to support a per-client UEH, then it might be easier to close this and update #2346 .




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r822265748



##########
File path: core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
##########
@@ -346,6 +347,16 @@ ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterCon
    */
   interface ClientFactory<T> {
 
+    /**
+     * Override default handling of uncaught exceptions in client threads
+     *
+     * @param ueh
+     *          UncaughtExceptionHandler implementation
+     * @return AccumuloClient or Properties
+     * @since 2.1.0
+     */
+    ClientFactory<T> withUncaughtExceptionHandler(Class<? extends UncaughtExceptionHandler> ueh);

Review comment:
       Instead of providing a class that we must instantiate (and may not have the expected constructors), why not require the instance of the class?
   
   ```suggestion
       ClientFactory<T> withUncaughtExceptionHandler(UncaughtExceptionHandler ueh);
   ```

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,6 +168,19 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    String uehClassName = serverConf.get(ClientProperty.UNCAUGHT_EXCEPTION_HANDLER.getKey());
+    if (!StringUtils.isEmpty(uehClassName)) {
+      try {
+        @SuppressWarnings("unchecked")
+        Class<? extends UncaughtExceptionHandler> clazz =
+            (Class<? extends UncaughtExceptionHandler>) Class.forName(uehClassName);
+        Threads.setUncaughtExceptionHandler(clazz.getDeclaredConstructor().newInstance());
+      } catch (ClassNotFoundException | InstantiationException | IllegalAccessException
+          | IllegalArgumentException | InvocationTargetException | NoSuchMethodException
+          | SecurityException e) {

Review comment:
       Several of these extend `ReflectiveOperationException`. Catching that should replace several others.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,6 +168,19 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    String uehClassName = serverConf.get(ClientProperty.UNCAUGHT_EXCEPTION_HANDLER.getKey());
+    if (!StringUtils.isEmpty(uehClassName)) {
+      try {
+        @SuppressWarnings("unchecked")
+        Class<? extends UncaughtExceptionHandler> clazz =
+            (Class<? extends UncaughtExceptionHandler>) Class.forName(uehClassName);

Review comment:
       ```suggestion
           var clazz = Class.forName(uehClassName).asSubclass(UncaughtExceptionHandler.class);
   ```

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -164,6 +168,19 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
+    String uehClassName = serverConf.get(ClientProperty.UNCAUGHT_EXCEPTION_HANDLER.getKey());
+    if (!StringUtils.isEmpty(uehClassName)) {
+      try {
+        @SuppressWarnings("unchecked")
+        Class<? extends UncaughtExceptionHandler> clazz =
+            (Class<? extends UncaughtExceptionHandler>) Class.forName(uehClassName);
+        Threads.setUncaughtExceptionHandler(clazz.getDeclaredConstructor().newInstance());
+      } catch (ClassNotFoundException | InstantiationException | IllegalAccessException
+          | IllegalArgumentException | InvocationTargetException | NoSuchMethodException
+          | SecurityException e) {
+        throw new RuntimeException("Error setting uncaughtExceptionHandler", e);

Review comment:
       Could choose a more specific RTE. IAE seems appropriate, since it is instantiated by a value provided by the user. It's a bit late to be throwing that, since it's not directly in the builder, but it's probably still better than ISE. Could throw this earlier, in the builder.

##########
File path: core/src/main/java/org/apache/accumulo/core/util/threads/Threads.java
##########
@@ -25,6 +25,8 @@
 
 public class Threads {
 
+  private static UncaughtExceptionHandler UEH = new AccumuloUncaughtExceptionHandler();

Review comment:
       We should be taking care to avoid using JVM-wide static state. What if the user creates two clients with different exception handlers in the same JVM?




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r826473855



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -736,6 +791,12 @@ public void close() {
     if (tableZooHelper != null) {
       tableZooHelper.close();
     }
+    if (scannerReadaheadPool != null) {

Review comment:
       Yeah closed does not need to be efficient, no need to worry about thread contention here.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r822609941



##########
File path: core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
##########
@@ -346,6 +347,16 @@ ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterCon
    */
   interface ClientFactory<T> {
 
+    /**
+     * Override default handling of uncaught exceptions in client threads
+     *
+     * @param ueh
+     *          UncaughtExceptionHandler implementation
+     * @return AccumuloClient or Properties
+     * @since 2.1.0
+     */
+    ClientFactory<T> withUncaughtExceptionHandler(Class<? extends UncaughtExceptionHandler> ueh);

Review comment:
       With the code the way that it is, would it allow a user to put the class name in the client properties file? If so, then doing it this way would preclude that.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r823073701



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
##########
@@ -365,8 +371,12 @@ private void reschedule(SendTask task) {
     this.context = context;
     this.auths = config.getAuthorizations();
     this.ve = new VisibilityEvaluator(config.getAuthorizations());
-    this.threadPool = ThreadPools.createScheduledExecutorService(config.getMaxWriteThreads(),
-        this.getClass().getSimpleName(), false);
+    this.threadPool = context.getClientThreadPools().createScheduledExecutorService(
+        config.getMaxWriteThreads(), this.getClass().getSimpleName(), false);
+    this.cleanupThreadPool = context.getClientThreadPools().createFixedThreadPool(1, 3, SECONDS,
+        "Conditional Writer Cleanup Thread", true);
+    this.cleaner = CleanerUtil.shutdownThreadPoolExecutor(cleanupThreadPool, closed,
+        LoggerFactory.getLogger(ConditionalWriterImpl.class));

Review comment:
       Addressed in ec0e7cf




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2554: Allow UncaughtExceptionHandler to be overridden in client

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2554:
URL: https://github.com/apache/accumulo/pull/2554#discussion_r825829296



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -121,7 +129,7 @@
   private TCredentials rpcCreds;
   private ThriftTransportPool thriftTransportPool;
 
-  private volatile boolean closed = false;

Review comment:
       I think it was lost in a prior change, which I then reverted. I will add it back.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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