kevinrr888 commented on code in PR #5576:
URL: https://github.com/apache/accumulo/pull/5576#discussion_r2103253534


##########
server/manager/src/main/java/org/apache/accumulo/manager/Manager.java:
##########
@@ -1265,14 +1265,21 @@ public void run() {
 
     context.getTableManager().addObserver(this);
 
-    Thread statusThread = Threads.createThread("Status Thread", new 
StatusThread());
+    // TODO KEVIN RATHBUN updating the Manager state seems like a critical 
function. However, the
+    // thread already handles, waits, and continues in the case of any 
Exception, so critical or
+    // non critical doesn't make a difference here.
+    Thread statusThread = Threads.createCriticalThread("Status Thread", new 
StatusThread());
     statusThread.start();
 
-    Threads.createThread("Migration Cleanup Thread", new 
MigrationCleanupThread()).start();
+    // TODO KEVIN RATHBUN migration cleanup may be a critical function of the 
manager, but the
+    // thread will already handle, wait, and continue in the case of any 
Exception, so critical
+    // or non critical doesn't make a difference here.
+    Threads.createCriticalThread("Migration Cleanup Thread", new 
MigrationCleanupThread()).start();

Review Comment:
   migration cleanup may be a critical function of the manager, but the thread 
will already handle, wait, and continue in the case of any Exception, so 
critical or non critical doesn't make a difference here.



##########
server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java:
##########
@@ -514,7 +514,9 @@ public void run() {
     }
 
     // need to regularly fetch data so plot data is updated
-    Threads.createThread("Data fetcher", () -> {
+    // TODO KEVIN RATHBUN don't think this is a critical function of the 
Monitor (and the
+    // RuntimeException is already handled here)
+    Threads.createNonCriticalThread("Data fetcher", () -> {

Review Comment:
   don't think this is a critical function of the Monitor (and the 
RuntimeException is already handled here)



##########
server/manager/src/main/java/org/apache/accumulo/manager/Manager.java:
##########
@@ -1378,8 +1385,10 @@ boolean canSuspendTablets() {
       } catch (KeeperException | InterruptedException e) {
         throw new IllegalStateException("Exception setting up delegation-token 
key manager", e);
       }
-      authenticationTokenKeyManagerThread =
-          Threads.createThread("Delegation Token Key Manager", 
authenticationTokenKeyManager);
+      // TODO KEVIN RATHBUN managing delegation tokens seems like a critical 
function of the
+      // manager and this is not recreated on failures.
+      authenticationTokenKeyManagerThread = Threads
+          .createCriticalThread("Delegation Token Key Manager", 
authenticationTokenKeyManager);

Review Comment:
   managing delegation tokens seems like a critical function of the manager and 
this is not recreated on failures.



##########
server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java:
##########
@@ -660,13 +659,8 @@ private static ServerAddress startTServer(ThriftServerType 
serverType, TimedProc
 
     final TServer finalServer = serverAddress.server;
 
-    Threads.createThread(threadName, () -> {
-      try {
-        finalServer.serve();
-      } catch (Error e) {
-        Halt.halt("Unexpected error in TThreadPoolServer " + e + ", halting.", 
1);
-      }
-    }).start();
+    // TODO KEVIN RATHBUN I can't imagine that the process would be healthy if 
this is not running
+    Threads.createCriticalThread(threadName, finalServer::serve).start();

Review Comment:
   I can't imagine that the process would be healthy if this is not running



##########
server/manager/src/main/java/org/apache/accumulo/manager/Manager.java:
##########
@@ -1265,14 +1265,21 @@ public void run() {
 
     context.getTableManager().addObserver(this);
 
-    Thread statusThread = Threads.createThread("Status Thread", new 
StatusThread());
+    // TODO KEVIN RATHBUN updating the Manager state seems like a critical 
function. However, the
+    // thread already handles, waits, and continues in the case of any 
Exception, so critical or
+    // non critical doesn't make a difference here.
+    Thread statusThread = Threads.createCriticalThread("Status Thread", new 
StatusThread());
     statusThread.start();
 
-    Threads.createThread("Migration Cleanup Thread", new 
MigrationCleanupThread()).start();
+    // TODO KEVIN RATHBUN migration cleanup may be a critical function of the 
manager, but the
+    // thread will already handle, wait, and continue in the case of any 
Exception, so critical
+    // or non critical doesn't make a difference here.
+    Threads.createCriticalThread("Migration Cleanup Thread", new 
MigrationCleanupThread()).start();
 
     tserverSet.startListeningForTabletServerChanges();
 
-    Threads.createThread("ScanServer Cleanup Thread", new 
ScanServerZKCleaner()).start();
+    // TODO KEVIN RATHBUN Some ZK cleanup doesn't seem like a critical 
function of manager
+    Threads.createNonCriticalThread("ScanServer Cleanup Thread", new 
ScanServerZKCleaner()).start();

Review Comment:
   Some ZK cleanup doesn't seem like a critical function of manager



##########
server/manager/src/main/java/org/apache/accumulo/manager/Manager.java:
##########
@@ -1265,14 +1265,21 @@ public void run() {
 
     context.getTableManager().addObserver(this);
 
-    Thread statusThread = Threads.createThread("Status Thread", new 
StatusThread());
+    // TODO KEVIN RATHBUN updating the Manager state seems like a critical 
function. However, the
+    // thread already handles, waits, and continues in the case of any 
Exception, so critical or
+    // non critical doesn't make a difference here.
+    Thread statusThread = Threads.createCriticalThread("Status Thread", new 
StatusThread());

Review Comment:
   updating the Manager state seems like a critical function. However, the 
thread already handles, waits, and continues in the case of any Exception, so 
critical or non critical doesn't make a difference here.



##########
server/base/src/main/java/org/apache/accumulo/server/security/SecurityUtil.java:
##########
@@ -115,13 +115,14 @@ public static String getServerPrincipal(String 
configuredPrincipal) {
   /**
    * Start a thread that periodically attempts to renew the current Kerberos 
user's ticket.
    *
-   * @param conf Accumulo configuration
    * @param ugi The current Kerberos user.
    * @param renewalPeriod The amount of time between attempting renewals.
    */
-  static void startTicketRenewalThread(AccumuloConfiguration conf, final 
UserGroupInformation ugi,
-      final long renewalPeriod) {
-    Threads.createThread("Kerberos Ticket Renewal", () -> {
+  static void startTicketRenewalThread(final UserGroupInformation ugi, final 
long renewalPeriod) {
+    // TODO KEVIN RATHBUN this renewal seems like a critical task of any 
process running it, as not
+    // renewing the ticket would probably lead to authentication problems. 
This thread is also only
+    // created once.
+    Threads.createCriticalThread("Kerberos Ticket Renewal", () -> {

Review Comment:
   this renewal seems like a critical task of any process running it, as not 
renewing the ticket would probably lead to authentication problems. This thread 
is also only created once.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java:
##########
@@ -73,7 +73,8 @@ public class ThriftTransportPool {
 
   private ThriftTransportPool(LongSupplier maxAgeMillis) {
     this.maxAgeMillis = maxAgeMillis;
-    this.checkThread = Threads.createThread("Thrift Connection Pool Checker", 
() -> {
+    // TODO KEVIN RATHBUN all this does is perform some resource cleanup, so 
may not be critical.
+    this.checkThread = Threads.createNonCriticalThread("Thrift Connection Pool 
Checker", () -> {

Review Comment:
   all this does is perform some resource cleanup, so may not be critical.



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java:
##########
@@ -232,7 +232,9 @@ public void run() {
               AssignmentHandler handler = new AssignmentHandler(server, 
extent, retryAttempt + 1);
               if (extent.isMeta()) {
                 if (extent.isRootTablet()) {
-                  Threads.createThread("Root tablet assignment retry", 
handler).start();
+                  // TODO KEVIN RATHBUN should remain non critical for same 
reason explained in
+                  // TabletClientHandler
+                  Threads.createNonCriticalThread("Root tablet assignment 
retry", handler).start();

Review Comment:
   should remain non critical for same reason explained in TabletClientHandler



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java:
##########
@@ -480,10 +480,13 @@ private class MemoryManagementFramework {
       tabletReports = Collections.synchronizedMap(new HashMap<>());
       memUsageReports = new LinkedBlockingQueue<>();
       maxMem = context.getConfiguration().getAsBytes(Property.TSERV_MAXMEM);
-      memoryGuardThread = Threads.createThread("Accumulo Memory Guard",
+      // TODO KEVIN RATHBUN guarding against excessive memory usage and 
initiating minor
+      // compactions are critical tasks of the tablet server. Also, these 
threads are only created
+      // once.
+      memoryGuardThread = Threads.createCriticalThread("Accumulo Memory Guard",
           OptionalInt.of(Thread.NORM_PRIORITY + 1), 
this::processTabletMemStats);
       minorCompactionInitiatorThread =
-          Threads.createThread("Accumulo Minor Compaction Initiator", 
this::manageMemory);
+          Threads.createCriticalThread("Accumulo Minor Compaction Initiator", 
this::manageMemory);

Review Comment:
   guarding against excessive memory usage and initiating minor compactions are 
critical tasks of the tablet server. Also, these threads are only created once.



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java:
##########
@@ -1246,24 +1246,24 @@ public void loadTablet(TInfo tinfo, TCredentials 
credentials, String lock,
     TabletLogger.loading(extent, server.getTabletSession());
 
     final AssignmentHandler ah = new AssignmentHandler(server, extent);
-    // final Runnable ah = new LoggingRunnable(log, );
     // Root tablet assignment must take place immediately
 
     if (extent.isRootTablet()) {
-      Threads.createThread("Root Tablet Assignment", () -> {
+      // TODO KEVIN RATHBUN I think this should remain non critical. This 
method is ultimately
+      // called by TabletGroupWatcher.flushChanges which is always called 
within a loop, so will
+      // continue to retry/recreate the thread
+      Threads.createNonCriticalThread("Root Tablet Assignment", () -> {

Review Comment:
   I think this should remain non critical. This method is ultimately called by 
TabletGroupWatcher.flushChanges which is always called within a loop, so will 
continue to retry/recreate the thread



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java:
##########
@@ -1110,7 +1110,9 @@ private static void checkWalCanSync(ServerContext 
context) {
 
   private void config() {
     log.info("Tablet server starting on {}", getHostname());
-    Threads.createThread("Split/MajC initiator", new 
MajorCompactor(context)).start();
+    // TODO KEVIN RATHBUN running major compactions is a critical function of 
the TabletServer.
+    // also this thread is only created once.
+    Threads.createCriticalThread("Split/MajC initiator", new 
MajorCompactor(context)).start();

Review Comment:
   running major compactions is a critical function of the TabletServer. also 
this thread is only created once.



##########
core/src/main/java/org/apache/accumulo/core/util/Halt.java:
##########
@@ -51,7 +51,9 @@ public static void halt(final int status, Runnable runnable) {
 
     try {
       // give ourselves a little time to try and do something
-      Threads.createThread("Halt Thread", () -> {
+      // TODO KEVIN RATHBUN doesn't matter if this is critical or not, halt() 
will be called in
+      // this method no matter what
+      Threads.createNonCriticalThread("Halt Thread", () -> {

Review Comment:
   doesn't matter if this is critical or not, halt() will be called in this 
method no matter what



##########
server/manager/src/main/java/org/apache/accumulo/manager/Manager.java:
##########
@@ -1616,14 +1625,16 @@ private TServer setupReplication()
         Property.MANAGER_REPLICATION_COORDINATOR_THREADCHECK, 
maxMessageSizeProperty);
 
     log.info("Started replication coordinator service at " + 
replAddress.address);
+    // TODO KEVIN RATHBUN this thread creation exists within a task which is 
labeled non-critical
+    // so assuming these are as well.
     // Start the daemon to scan the replication table and make units of work
-    replicationWorkThread = Threads.createThread("Replication Driver",
+    replicationWorkThread = Threads.createNonCriticalThread("Replication 
Driver",
         new org.apache.accumulo.manager.replication.ReplicationDriver(this));
     replicationWorkThread.start();
 
     // Start the daemon to assign work to tservers to replicate to our peers
     var wd = new org.apache.accumulo.manager.replication.WorkDriver(this);
-    replicationAssignerThread = Threads.createThread(wd.getName(), wd);
+    replicationAssignerThread = Threads.createNonCriticalThread(wd.getName(), 
wd);

Review Comment:
   this thread creation exists within a task which is labeled non-critical so 
assuming these are as well.



##########
server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java:
##########
@@ -243,7 +243,9 @@ public void startServiceLockVerificationThread() {
     final long interval =
         
getConfiguration().getTimeInMillis(Property.GENERAL_SERVER_LOCK_VERIFICATION_INTERVAL);
     if (interval > 0) {
-      verificationThread = 
Threads.createThread("service-lock-verification-thread",
+      // TODO KEVIN RATHBUN verifying the service lock is a critical function 
of any process
+      // calling this and the thread would not be recreated on failures
+      verificationThread = 
Threads.createCriticalThread("service-lock-verification-thread",

Review Comment:
   verifying the service lock is a critical function of any process calling 
this and the thread would not be recreated on failures



##########
server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java:
##########
@@ -87,8 +87,9 @@ public ServerConfigurationFactory(ServerContext context, 
SiteConfiguration siteC
         Caffeine.newBuilder().expireAfterAccess(CACHE_EXPIRATION_HRS, 
TimeUnit.HOURS).build();
 
     refresher = new ConfigRefreshRunner();
-    Runtime.getRuntime()
-        .addShutdownHook(Threads.createThread("config-refresh-shutdownHook", 
refresher::shutdown));
+    // TODO KEVIN RATHBUN JVM already shutting down, no need to be critical
+    Runtime.getRuntime().addShutdownHook(
+        Threads.createNonCriticalThread("config-refresh-shutdownHook", 
refresher::shutdown));

Review Comment:
   JVM already shutting down, no need to be critical



##########
core/src/main/java/org/apache/accumulo/core/util/threads/NamedThreadFactory.java:
##########
@@ -55,6 +55,11 @@ public Thread newThread(Runnable r) {
       threadName =
           String.format(FORMAT, name, r.getClass().getSimpleName(), 
threadNum.getAndIncrement());
     }
-    return Threads.createThread(threadName, priority, r, handler);
+    // TODO KEVIN RATHBUN I don't believe this needs to be critical (or needs 
to have a way of
+    // configuring whether or not the thread is critical). This class is used 
in the ThreadPools
+    // class to create a ThreadPoolExecutor. Tasks submitted to this pool are 
configured to be
+    // critical or not with watchCriticalScheduledTask, 
watchCriticalFixedDelay, and
+    // watchNonCriticalScheduledTask
+    return Threads.createNonCriticalThread(threadName, priority, r, handler);

Review Comment:
   I don't believe this needs to be critical (or needs to have a way of 
configuring whether or not the thread is critical). This class is used in the 
ThreadPools class to create a ThreadPoolExecutor. Tasks submitted to this pool 
are configured to be critical or not with watchCriticalScheduledTask, 
watchCriticalFixedDelay, and watchNonCriticalScheduledTask



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java:
##########
@@ -475,7 +475,10 @@ public synchronized void open(String address) throws 
IOException {
       throw new IOException(ex);
     }
 
-    syncThread = Threads.createThread("Accumulo WALog thread " + this, new 
LogSyncingTask());
+    // TODO KEVIN RATHBUN this seems like a vital thread for TabletServer, but 
appears that the
+    // thread will continuously be recreated, so probably fine to stay non 
critical
+    syncThread =
+        Threads.createNonCriticalThread("Accumulo WALog thread " + this, new 
LogSyncingTask());

Review Comment:
   this seems like a vital thread for TabletServer, but appears that the thread 
will continuously be recreated, so probably fine to stay non critical



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionManager.java:
##########
@@ -298,7 +298,9 @@ private synchronized void checkForConfigChanges(boolean 
force) {
 
   public void start() {
     log.debug("Started compaction manager");
-    Threads.createThread("Compaction Manager", () -> mainLoop()).start();
+    // TODO KEVIN RATHBUN This is a critical thread for the TabletServer to 
run properly and is
+    // only called once.
+    Threads.createCriticalThread("Compaction Manager", () -> 
mainLoop()).start();

Review Comment:
   This is a critical thread for the TabletServer to run properly and is only 
called once.



##########
server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java:
##########
@@ -736,8 +736,10 @@ public void run() {
           final FileCompactorRunnable fcr =
               createCompactionJob(job, totalInputEntries, totalInputBytes, 
started, stopped, err);
 
-          final Thread compactionThread =
-              Threads.createThread("Compaction job for tablet " + 
job.getExtent().toString(), fcr);
+          // TODO KEVIN RATHBUN exists within a while(!shutdown) loop so 
thread is repeatedly
+          // recreated. No need to be critical. If a single job fails, that's 
okay.
+          final Thread compactionThread = Threads.createNonCriticalThread(
+              "Compaction job for tablet " + job.getExtent().toString(), fcr);

Review Comment:
   exists within a while(!shutdown) loop so thread is repeatedly recreated. No 
need to be critical. If a single job fails, that's okay.



-- 
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-unsubscr...@accumulo.apache.org

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

Reply via email to