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 627ed8555582a4f34bc6678dc4a8f7829321ea41 Author: Dom Garguilo <domgargu...@apache.org> AuthorDate: Wed May 29 15:26:18 2024 -0400 Use NanoTime object for rate limiting in FileCompactor stats update --- .../java/org/apache/accumulo/server/compaction/FileCompactor.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java b/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java index e2ea4c46b2..cebdce684d 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java +++ b/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java @@ -66,6 +66,7 @@ import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.LocalityGroupUtil; import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError; import org.apache.accumulo.core.util.ratelimit.RateLimiter; +import org.apache.accumulo.core.util.time.NanoTime; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.fs.VolumeManager; import org.apache.accumulo.server.iterators.SystemIteratorEnvironment; @@ -137,7 +138,7 @@ public class FileCompactor implements Callable<CompactionStats> { private static final LongAdder totalEntriesRead = new LongAdder(); private static final LongAdder totalEntriesWritten = new LongAdder(); - private static volatile long lastUpdateTime = 0; + private static volatile NanoTime lastUpdateTime = NanoTime.now(); private final DateFormat dateFormatter = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS"); @@ -208,12 +209,11 @@ public class FileCompactor implements Callable<CompactionStats> { * is rate limited, so it will not cause issues if called too frequently. */ private static void updateTotalEntries() { - long currentTime = System.nanoTime(); - if (currentTime - lastUpdateTime < Duration.ofMillis(100).toNanos()) { + if (lastUpdateTime.elapsed().compareTo(Duration.ofMillis(100)) < 0) { return; } runningCompactions.forEach(FileCompactor::updateGlobalEntryCounts); - lastUpdateTime = currentTime; + lastUpdateTime = NanoTime.now(); } protected static final Set<FileCompactor> runningCompactions =