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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]