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


The following commit(s) were added to refs/heads/main by this push:
     new 372fe8fd4f Use NanoTime object for age tracking in 
FileCompactorRunnable and FileCompactor
372fe8fd4f is described below

commit 372fe8fd4fddb44bd7ff280fd7cf8dbe7fde3e75
Author: Dom Garguilo <domgargu...@apache.org>
AuthorDate: Fri Jun 14 15:26:41 2024 -0400

    Use NanoTime object for age tracking in FileCompactorRunnable and 
FileCompactor
---
 .../org/apache/accumulo/server/compaction/FileCompactor.java     | 8 ++++----
 .../src/main/java/org/apache/accumulo/compactor/Compactor.java   | 9 +++++----
 2 files changed, 9 insertions(+), 8 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 906dcc0331..b4e53ce2ce 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
@@ -125,7 +125,7 @@ public class FileCompactor implements 
Callable<CompactionStats> {
 
   // things to report
   private String currentLocalityGroup = "";
-  private volatile long startTime = -1;
+  private volatile NanoTime startTime;
 
   private final AtomicInteger timesPaused = new AtomicInteger(0);
 
@@ -284,7 +284,7 @@ public class FileCompactor implements 
Callable<CompactionStats> {
 
     CompactionStats majCStats = new CompactionStats();
 
-    startTime = System.nanoTime();
+    startTime = NanoTime.now();
 
     boolean remove = runningCompactions.add(this);
 
@@ -605,11 +605,11 @@ public class FileCompactor implements 
Callable<CompactionStats> {
    * @return the duration since {@link #call()} was called
    */
   Duration getAge() {
-    if (startTime == -1) {
+    if (startTime == null) {
       // call() has not been called yet
       return Duration.ZERO;
     }
-    return Duration.ofNanos(System.nanoTime() - startTime);
+    return startTime.elapsed();
   }
 
   Iterable<IteratorSetting> getIterators() {
diff --git 
a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java 
b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
index 0649b9e8fe..26a610a65c 100644
--- 
a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
+++ 
b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
@@ -97,6 +97,7 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
+import org.apache.accumulo.core.util.time.NanoTime;
 import org.apache.accumulo.server.AbstractServer;
 import org.apache.accumulo.server.client.ClientServiceHandler;
 import org.apache.accumulo.server.compaction.CompactionInfo;
@@ -510,12 +511,12 @@ public class Compactor extends AbstractServer implements 
MetricsProducer, Compac
     return new FileCompactorRunnable() {
 
       private final AtomicReference<FileCompactor> compactor = new 
AtomicReference<>();
-      private volatile long startTimeNanos = -1;
+      private volatile NanoTime compactionStartTime;
 
       @Override
       public void initialize() throws RetriesExceededException {
         LOG.info("Starting up compaction runnable for job: {}", job);
-        this.startTimeNanos = System.nanoTime();
+        this.compactionStartTime = NanoTime.now();
         TCompactionStatusUpdate update = new 
TCompactionStatusUpdate(TCompactionState.STARTED,
             "Compaction started", -1, -1, -1, getCompactionAge().toNanos());
         updateCompactionState(job, update);
@@ -605,11 +606,11 @@ public class Compactor extends AbstractServer implements 
MetricsProducer, Compac
 
       @Override
       public Duration getCompactionAge() {
-        if (startTimeNanos == -1) {
+        if (compactionStartTime == null) {
           // compaction hasn't started yet
           return Duration.ZERO;
         }
-        return Duration.ofNanos(System.nanoTime() - startTimeNanos);
+        return compactionStartTime.elapsed();
       }
 
     };

Reply via email to