This is an automated email from the ASF dual-hosted git repository. domgarguilo pushed a commit to branch main in repository https://gitbox.apache.org/repos/asf/accumulo.git
commit 000855d4c1f68d7bf3bb22cc7fedf5674259104e Merge: 9ec0087069 528711eb17 Author: Dom Garguilo <domgargu...@apache.org> AuthorDate: Thu Jun 6 11:34:45 2024 -0400 Merge remote-tracking branch 'upstream/2.1' # Conflicts: # test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java .../accumulo/core/metrics/MetricsProducer.java | 27 ++++++- .../org/apache/accumulo/compactor/Compactor.java | 8 ++ .../compaction/ExternalCompactionProgressIT.java | 89 +++++++++++++--------- 3 files changed, 89 insertions(+), 35 deletions(-) diff --cc server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java index 4ba279162e,b82358581a..f8ab5b9386 --- a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java +++ b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java @@@ -118,10 -116,11 +118,11 @@@ import org.apache.zookeeper.KeeperExcep import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.beust.jcommander.Parameter; import com.google.common.base.Preconditions; +import com.google.common.net.HostAndPort; import io.micrometer.core.instrument.FunctionCounter; + import io.micrometer.core.instrument.Gauge; import io.micrometer.core.instrument.LongTaskTimer; import io.micrometer.core.instrument.MeterRegistry; @@@ -184,8 -194,22 +185,15 @@@ public class Compactor extends Abstract LongTaskTimer timer = LongTaskTimer.builder(METRICS_COMPACTOR_MAJC_STUCK) .description("Number and duration of stuck major compactions").register(registry); CompactionWatcher.setTimer(timer); + + Gauge + .builder(METRICS_COMPACTOR_BUSY, this.compactionRunning, + isRunning -> isRunning.get() ? 1 : 0) + .description( + "Indicates if the compactor is busy or not. The value will be 0 when idle and 1 when busy.") + .register(registry); } - protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) { - ScheduledFuture<?> future = - schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0, - TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS); - ThreadPools.watchNonCriticalScheduledTask(future); - } - protected void startCancelChecker(ScheduledThreadPoolExecutor schedExecutor, long timeBetweenChecks) { ThreadPools.watchCriticalScheduledTask(schedExecutor.scheduleWithFixedDelay(