dlmarion commented on a change in pull request #2259:
URL: https://github.com/apache/accumulo/pull/2259#discussion_r717848832
##########
File path:
core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
##########
@@ -101,88 +106,105 @@ public static ExecutorService
createExecutorService(final AccumuloConfiguration
switch (p) {
case GENERAL_SIMPLETIMER_THREADPOOL_SIZE:
- return createScheduledExecutorService(conf.getCount(p), "SimpleTimer",
false);
+ return createScheduledExecutorService(conf.getCount(p), "SimpleTimer");
case MANAGER_BULK_THREADPOOL_SIZE:
return createFixedThreadPool(conf.getCount(p),
conf.getTimeInMillis(Property.MANAGER_BULK_THREADPOOL_TIMEOUT),
TimeUnit.MILLISECONDS,
- "bulk import", true);
+ "bulk import");
case MANAGER_RENAME_THREADS:
- return createFixedThreadPool(conf.getCount(p), "bulk move", false);
+ return createFixedThreadPool(conf.getCount(p), "bulk move");
case MANAGER_FATE_THREADPOOL_SIZE:
- return createFixedThreadPool(conf.getCount(p), "Repo Runner", false);
+ return createFixedThreadPool(conf.getCount(p), "Repo Runner");
case MANAGER_STATUS_THREAD_POOL_SIZE:
int threads = conf.getCount(p);
if (threads == 0) {
return createThreadPool(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS,
- "GatherTableInformation", new SynchronousQueue<Runnable>(),
OptionalInt.empty(),
- false);
+ "GatherTableInformation", new SynchronousQueue<Runnable>(),
OptionalInt.empty());
} else {
- return createFixedThreadPool(threads, "GatherTableInformation",
false);
+ return createFixedThreadPool(threads, "GatherTableInformation");
}
case TSERV_WORKQ_THREADS:
- return createFixedThreadPool(conf.getCount(p), "distributed work
queue", false);
+ return createFixedThreadPool(conf.getCount(p), "distributed work
queue");
case TSERV_MINC_MAXCONCURRENT:
- return createFixedThreadPool(conf.getCount(p), 0L,
TimeUnit.MILLISECONDS, "minor compactor",
- true);
+ return createFixedThreadPool(conf.getCount(p), 0L,
TimeUnit.MILLISECONDS,
+ "minor compactor");
case TSERV_MIGRATE_MAXCONCURRENT:
return createFixedThreadPool(conf.getCount(p), 0L,
TimeUnit.MILLISECONDS,
- "tablet migration", true);
+ "tablet migration");
case TSERV_ASSIGNMENT_MAXCONCURRENT:
return createFixedThreadPool(conf.getCount(p), 0L,
TimeUnit.MILLISECONDS,
- "tablet assignment", true);
+ "tablet assignment");
case TSERV_SUMMARY_RETRIEVAL_THREADS:
return createThreadPool(conf.getCount(p), conf.getCount(p), 60,
TimeUnit.SECONDS,
- "summary file retriever", true);
+ "summary file retriever");
case TSERV_SUMMARY_REMOTE_THREADS:
return createThreadPool(conf.getCount(p), conf.getCount(p), 60,
TimeUnit.SECONDS,
- "summary remote", true);
+ "summary remote");
case TSERV_SUMMARY_PARTITION_THREADS:
return createThreadPool(conf.getCount(p), conf.getCount(p), 60,
TimeUnit.SECONDS,
- "summary partition", true);
+ "summary partition");
case GC_DELETE_THREADS:
- return createFixedThreadPool(conf.getCount(p), "deleting", false);
+ return createFixedThreadPool(conf.getCount(p), "deleting");
case REPLICATION_WORKER_THREADS:
- return createFixedThreadPool(conf.getCount(p), "replication task",
false);
+ return createFixedThreadPool(conf.getCount(p), "replication task");
default:
throw new RuntimeException("Unhandled thread pool property: " + p);
}
}
- public static ThreadPoolExecutor createFixedThreadPool(int numThreads, final
String name,
- boolean enableTracing) {
- return createFixedThreadPool(numThreads, DEFAULT_TIMEOUT_MILLISECS,
TimeUnit.MILLISECONDS, name,
- enableTracing);
+ public static ThreadPoolExecutor createFixedThreadPool(int numThreads, final
String name) {
+ return createFixedThreadPool(numThreads, DEFAULT_TIMEOUT_MILLISECS,
TimeUnit.MILLISECONDS,
+ name);
}
public static ThreadPoolExecutor createFixedThreadPool(int numThreads, final
String name,
- BlockingQueue<Runnable> queue, boolean enableTracing) {
+ BlockingQueue<Runnable> queue) {
return createThreadPool(numThreads, numThreads, DEFAULT_TIMEOUT_MILLISECS,
- TimeUnit.MILLISECONDS, name, queue, OptionalInt.empty(),
enableTracing);
+ TimeUnit.MILLISECONDS, name, queue, OptionalInt.empty());
}
public static ThreadPoolExecutor createFixedThreadPool(int numThreads, long
timeOut,
- TimeUnit units, final String name, boolean enableTracing) {
+ TimeUnit units, final String name) {
return createThreadPool(numThreads, numThreads, timeOut, units, name,
- new LinkedBlockingQueue<Runnable>(), OptionalInt.empty(),
enableTracing);
+ new LinkedBlockingQueue<Runnable>(), OptionalInt.empty());
}
public static ThreadPoolExecutor createThreadPool(int coreThreads, int
maxThreads, long timeOut,
- TimeUnit units, final String name, boolean enableTracing) {
+ TimeUnit units, final String name) {
return createThreadPool(coreThreads, maxThreads, timeOut, units, name,
- new LinkedBlockingQueue<Runnable>(), OptionalInt.empty(),
enableTracing);
+ new LinkedBlockingQueue<Runnable>(), OptionalInt.empty());
}
public static ThreadPoolExecutor createThreadPool(int coreThreads, int
maxThreads, long timeOut,
- TimeUnit units, final String name, BlockingQueue<Runnable> queue,
OptionalInt priority,
- boolean enableTracing) {
- ThreadPoolExecutor result = null;
- if (enableTracing) {
- result = new TracingThreadPoolExecutor(coreThreads, maxThreads, timeOut,
units, queue,
- new NamedThreadFactory(name, priority));
- } else {
- result = new ThreadPoolExecutor(coreThreads, maxThreads, timeOut, units,
queue,
- new NamedThreadFactory(name, priority));
- }
+ TimeUnit units, final String name, BlockingQueue<Runnable> queue,
OptionalInt priority) {
+ ThreadPoolExecutor result = new ThreadPoolExecutor(coreThreads,
maxThreads, timeOut, units,
+ queue, new NamedThreadFactory(name, priority)) {
+
+ @Override
+ public void execute(Runnable arg0) {
+ super.execute(Context.current().wrap(arg0));
+ }
Review comment:
Looks like they have *something*:
https://github.com/open-telemetry/opentelemetry-java/blob/main/context/src/main/java/io/opentelemetry/context/ContextExecutorService.java.
Not sure yet if it will 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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]