tibrewalpratik17 commented on code in PR #11849:
URL: https://github.com/apache/pinot/pull/11849#discussion_r1373482116
##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -124,23 +126,33 @@ public IngestionDelayTracker(ServerMetrics serverMetrics,
String tableNameWithTy
_clock = Clock.systemUTC();
_isServerReadyToServeQueries = isServerReadyToServeQueries;
// Handle negative timer values
- if (timerThreadTickIntervalMs <= 0) {
+ if (scheduledExecutorThreadTickIntervalMs <= 0) {
throw new RuntimeException(String.format("Illegal timer timeout
argument, expected > 0, got=%d for table=%s",
- timerThreadTickIntervalMs, _tableNameWithType));
+ scheduledExecutorThreadTickIntervalMs, _tableNameWithType));
}
- _timerThreadTickIntervalMs = timerThreadTickIntervalMs;
- _timer = new Timer("IngestionDelayTimerThread-" +
TableNameBuilder.extractRawTableName(tableNameWithType));
- _timer.schedule(new TimerTask() {
- @Override
- public void run() {
- timeoutInactivePartitions();
- }
- }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+ _scheduledExecutorThreadTickIntervalMs =
scheduledExecutorThreadTickIntervalMs;
+
+ // ThreadFactory to set the thread's name
+ ThreadFactory threadFactory = new ThreadFactory() {
+ private final ThreadFactory _defaultFactory =
Executors.defaultThreadFactory();
+
+ @Override
+ public Thread newThread(Runnable r) {
+ Thread thread = _defaultFactory.newThread(r);
+ thread.setName("IngestionDelayTimerThread-" +
TableNameBuilder.extractRawTableName(tableNameWithType));
+ return thread;
+ }
+ };
+
+ _scheduledExecutor =
Executors.newSingleThreadScheduledExecutor(threadFactory);
Review Comment:
Instead of initialising a new thread for every single object of this class,
better to we define a static thread-pool of size: 20 (later we can move this to
a cluster level config as well).
We can submit this task -`timeoutInactivePartitions` to the thread-pool.
--
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]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]