You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2022/01/03 17:51:10 UTC

[GitHub] [accumulo] dlmarion opened a new pull request #2405: Created additional metrics for long assignments and stuck compactions

dlmarion opened a new pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405


   Related to #946


-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] EdColeman commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
EdColeman commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r778387739



##########
File path: server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionWatcher.java
##########
@@ -79,6 +84,10 @@ public void run() {
     copy.keySet().removeAll(newKeys);
 
     for (ObservedCompactionInfo oci : copy.values()) {
+      if (oci.stuckSample != null) {

Review comment:
       The null check / set sample to null may be unnecessary - from the micrometer docs, LongTaskTimers do not report on completed tasks - if may be enough to record the start / stop which would provide the active task count, and the active count duration info.




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777701067



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {
+    private final long warnTime;
+    private long time;
+    private long read;
+    private long written;
+    private boolean stuck = false;
+
+    public CompactionProgress(AccumuloConfiguration config) {
+      warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
+    }
+
+    @Override
+    public void registerMetrics(MeterRegistry registry) {
+      Gauge.builder(METRICS_TSERVER_MAJC_STUCK, this, CompactionProgress::getNumStuck)

Review comment:
       > Can it be kept as a count - but not as a metric? Or, if kept as a metric provide additional information?
   
   So, if it's not a value reported by the metrics system, then what do you mean by keeping it as a `count` ? What additional information do you think need needs to be conveyed?




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r778379645



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {
+    private final long warnTime;
+    private long time;
+    private long read;
+    private long written;
+    private boolean stuck = false;
+
+    public CompactionProgress(AccumuloConfiguration config) {
+      warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
+    }
+
+    @Override
+    public void registerMetrics(MeterRegistry registry) {
+      Gauge.builder(METRICS_TSERVER_MAJC_STUCK, this, CompactionProgress::getNumStuck)

Review comment:
       Check out the changes in bec830ff3d0dc606864e5172fc463f80224f5ca7, I modified the code to use a LongTaskTimer.




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r778390575



##########
File path: server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionWatcher.java
##########
@@ -79,6 +84,10 @@ public void run() {
     copy.keySet().removeAll(newKeys);
 
     for (ObservedCompactionInfo oci : copy.values()) {
+      if (oci.stuckSample != null) {

Review comment:
       A single compaction could get stuck several times over the life of the compaction. I'm trying to account for that.




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777663458



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {
+    private final long warnTime;
+    private long time;
+    private long read;
+    private long written;
+    private boolean stuck = false;
+
+    public CompactionProgress(AccumuloConfiguration config) {
+      warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
+    }
+
+    @Override
+    public void registerMetrics(MeterRegistry registry) {
+      Gauge.builder(METRICS_TSERVER_MAJC_STUCK, this, CompactionProgress::getNumStuck)

Review comment:
       Right, the Compactor only runs one majc at a time. So, the current one is either stuck, or not stuck. The summation across all processes (or by some other aggregation) would be done in a downstream time series database.




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777660918



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetrics.java
##########
@@ -46,6 +51,10 @@ public void registerMetrics(MeterRegistry registry) {
         .description("Number of queued major compactions").register(registry);
     Gauge.builder(METRICS_TSERVER_MINC_RUNNING, util, TabletServerMetricsUtil::getMinorCompactions)
         .description("Number of active minor compactions").register(registry);
+    Gauge
+        .builder(METRICS_TSERVER_MAJC_STUCK, util,

Review comment:
       They are the same metric, just being reported from different processes (the process name is in a tag). `METRICS_TSERVER_MAJC_STUCK` is just the variable name, the metric being reported is `metrics.tserver.majc.stuck`. 




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r778056835



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionWatcher.java
##########
@@ -75,35 +80,42 @@ public void run() {
     HashMap<List<Long>,ObservedCompactionInfo> copy = new HashMap<>(observedCompactions);
     copy.keySet().removeAll(newKeys);
 
+    long unstuck = 0;
     for (ObservedCompactionInfo oci : copy.values()) {
+      unstuck++;
       if (oci.loggedWarning) {
         LoggerFactory.getLogger(CompactionWatcher.class).info("Compaction of {} is no longer stuck",
             oci.compactionInfo.getExtent());
       }
     }
+    totalStuck = Math.min(0, (totalStuck - unstuck));
 
     // remove any compaction that completed or made progress
     observedCompactions.keySet().retainAll(newKeys);
 
     long warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
 
     // check for stuck compactions
+    long stuck = 0;
     for (ObservedCompactionInfo oci : observedCompactions.values()) {
-      if (time - oci.firstSeen > warnTime && !oci.loggedWarning) {
-        Thread compactionThread = oci.compactionInfo.getThread();
-        if (compactionThread != null) {
-          StackTraceElement[] trace = compactionThread.getStackTrace();
-          Exception e = new Exception(
-              "Possible stack trace of compaction stuck on " + oci.compactionInfo.getExtent());
-          e.setStackTrace(trace);
-          LoggerFactory.getLogger(CompactionWatcher.class)
-              .warn("Compaction of " + oci.compactionInfo.getExtent() + " to "
-                  + oci.compactionInfo.getOutputFile() + " has not made progress for at least "
-                  + (time - oci.firstSeen) + "ms", e);
-          oci.loggedWarning = true;
+      if (time - oci.firstSeen > warnTime) {
+        stuck++;
+        if (!oci.loggedWarning) {
+          Thread compactionThread = oci.compactionInfo.getThread();
+          if (compactionThread != null) {
+            StackTraceElement[] trace = compactionThread.getStackTrace();
+            Exception e = new Exception(
+                "Possible stack trace of compaction stuck on " + oci.compactionInfo.getExtent());
+            e.setStackTrace(trace);
+            LoggerFactory.getLogger(CompactionWatcher.class).warn("Compaction of "
+                + oci.compactionInfo.getExtent() + " to " + oci.compactionInfo.getOutputFile()
+                + " has not made progress for at least " + (time - oci.firstSeen) + "ms", e);
+            oci.loggedWarning = true;
+          }
         }
       }
     }
+    totalStuck += stuck;

Review comment:
       addressed in f5b3223f4cfb0d7ac2d99664f81898962a54c5c9

##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {

Review comment:
       addressed in f5b3223f4cfb0d7ac2d99664f81898962a54c5c9




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] EdColeman commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
EdColeman commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777653892



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {
+    private final long warnTime;
+    private long time;
+    private long read;
+    private long written;
+    private boolean stuck = false;
+
+    public CompactionProgress(AccumuloConfiguration config) {
+      warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
+    }
+
+    @Override
+    public void registerMetrics(MeterRegistry registry) {
+      Gauge.builder(METRICS_TSERVER_MAJC_STUCK, this, CompactionProgress::getNumStuck)

Review comment:
       With modification - this could show the "stuck" time rather than a boolean.  This would allow users to set external alarms based on time rather than on a boolean condition. It could also provide insight if things can somehow become "unstuck" without intervention, you could plot the stuck time over time to see how things are progressing.




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] EdColeman commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
EdColeman commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777654685



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {
+    private final long warnTime;
+    private long time;
+    private long read;
+    private long written;
+    private boolean stuck = false;
+
+    public CompactionProgress(AccumuloConfiguration config) {
+      warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
+    }
+
+    @Override
+    public void registerMetrics(MeterRegistry registry) {
+      Gauge.builder(METRICS_TSERVER_MAJC_STUCK, this, CompactionProgress::getNumStuck)
+          .description("Number of stuck major compactions").register(registry);
+    }
+
+    public long getNumStuck() {
+      return stuck ? 1 : 0;
+    }

Review comment:
       Something like if not stuck return 0, else return System.currentTimeMillis() - time




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] EdColeman commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
EdColeman commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777673786



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {
+    private final long warnTime;
+    private long time;
+    private long read;
+    private long written;
+    private boolean stuck = false;
+
+    public CompactionProgress(AccumuloConfiguration config) {
+      warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
+    }
+
+    @Override
+    public void registerMetrics(MeterRegistry registry) {
+      Gauge.builder(METRICS_TSERVER_MAJC_STUCK, this, CompactionProgress::getNumStuck)

Review comment:
       I'm using metric as - a value that is reported by the metrics system




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777667063



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {
+    private final long warnTime;
+    private long time;
+    private long read;
+    private long written;
+    private boolean stuck = false;
+
+    public CompactionProgress(AccumuloConfiguration config) {
+      warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
+    }
+
+    @Override
+    public void registerMetrics(MeterRegistry registry) {
+      Gauge.builder(METRICS_TSERVER_MAJC_STUCK, this, CompactionProgress::getNumStuck)

Review comment:
       I'm confused by your use of `metric` in the latest comment. I have it set as a `gauge` because the number of stuck compactions is going to go up and down, in both the TabletServer and Compactor, over time.




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777714581



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {

Review comment:
       I'll look into this.




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] EdColeman commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
EdColeman commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777659266



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetrics.java
##########
@@ -46,6 +51,10 @@ public void registerMetrics(MeterRegistry registry) {
         .description("Number of queued major compactions").register(registry);
     Gauge.builder(METRICS_TSERVER_MINC_RUNNING, util, TabletServerMetricsUtil::getMinorCompactions)
         .description("Number of active minor compactions").register(registry);
+    Gauge
+        .builder(METRICS_TSERVER_MAJC_STUCK, util,

Review comment:
       If this is separate from Compactor METRICS_TSERVER_MAJC_STUCK and is the sum of stuck counts, then maybe named METRICS_TSERVER_MAJC_STUCK_COUNT or METRICS_TSERVER_MAJC_STUCK_TOTAL?  (I don't recall suggested naming conventions)




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r778379237



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -693,6 +712,7 @@ public void run() {
                     "Compaction in progress, read %d of %d input entries ( %s %s ), written %d entries",
                     info.getEntriesRead(), inputEntries, percentComplete, "%",
                     info.getEntriesWritten());
+                watcher.run();

Review comment:
       In bec830ff3d0dc606864e5172fc463f80224f5ca7 I call `watcher.run()` after the compaction has finished to clear out the state.




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] keith-turner commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777708779



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {

Review comment:
       Would it be possible to move CompactionWatcher from the tserver to server-base and reuse it in the compactor instead of introducing similar code in the compactor?

##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionWatcher.java
##########
@@ -75,35 +80,42 @@ public void run() {
     HashMap<List<Long>,ObservedCompactionInfo> copy = new HashMap<>(observedCompactions);
     copy.keySet().removeAll(newKeys);
 
+    long unstuck = 0;
     for (ObservedCompactionInfo oci : copy.values()) {
+      unstuck++;
       if (oci.loggedWarning) {
         LoggerFactory.getLogger(CompactionWatcher.class).info("Compaction of {} is no longer stuck",
             oci.compactionInfo.getExtent());
       }
     }
+    totalStuck = Math.min(0, (totalStuck - unstuck));
 
     // remove any compaction that completed or made progress
     observedCompactions.keySet().retainAll(newKeys);
 
     long warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
 
     // check for stuck compactions
+    long stuck = 0;
     for (ObservedCompactionInfo oci : observedCompactions.values()) {
-      if (time - oci.firstSeen > warnTime && !oci.loggedWarning) {
-        Thread compactionThread = oci.compactionInfo.getThread();
-        if (compactionThread != null) {
-          StackTraceElement[] trace = compactionThread.getStackTrace();
-          Exception e = new Exception(
-              "Possible stack trace of compaction stuck on " + oci.compactionInfo.getExtent());
-          e.setStackTrace(trace);
-          LoggerFactory.getLogger(CompactionWatcher.class)
-              .warn("Compaction of " + oci.compactionInfo.getExtent() + " to "
-                  + oci.compactionInfo.getOutputFile() + " has not made progress for at least "
-                  + (time - oci.firstSeen) + "ms", e);
-          oci.loggedWarning = true;
+      if (time - oci.firstSeen > warnTime) {
+        stuck++;
+        if (!oci.loggedWarning) {
+          Thread compactionThread = oci.compactionInfo.getThread();
+          if (compactionThread != null) {
+            StackTraceElement[] trace = compactionThread.getStackTrace();
+            Exception e = new Exception(
+                "Possible stack trace of compaction stuck on " + oci.compactionInfo.getExtent());
+            e.setStackTrace(trace);
+            LoggerFactory.getLogger(CompactionWatcher.class).warn("Compaction of "
+                + oci.compactionInfo.getExtent() + " to " + oci.compactionInfo.getOutputFile()
+                + " has not made progress for at least " + (time - oci.firstSeen) + "ms", e);
+            oci.loggedWarning = true;
+          }
         }
       }
     }
+    totalStuck += stuck;

Review comment:
       Could the following be done?  If so could remove the unstuck calculation and deduction earlier in the code.
   
   ```suggestion
       totalStuck = stuck;
   ```




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion merged pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
dlmarion merged pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405


   


-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r778379277



##########
File path: server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionWatcher.java
##########
@@ -29,14 +29,13 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.compaction.CompactionInfo;
-import org.apache.accumulo.server.compaction.FileCompactor;
 import org.slf4j.LoggerFactory;
 
 public class CompactionWatcher implements Runnable {
   private final Map<List<Long>,ObservedCompactionInfo> observedCompactions = new HashMap<>();
   private final AccumuloConfiguration config;
   private static boolean watching = false;
+  private static long totalStuck = 0;

Review comment:
       Changed in bec830ff3d0dc606864e5172fc463f80224f5ca7




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] keith-turner commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r778283083



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -693,6 +712,7 @@ public void run() {
                     "Compaction in progress, read %d of %d input entries ( %s %s ), written %d entries",
                     info.getEntriesRead(), inputEntries, percentComplete, "%",
                     info.getEntriesWritten());
+                watcher.run();

Review comment:
       Not sure if this can happen, but I think there may be a case where stuck compaction stats are not cleaned up because of how this is called.  It seems like its only called periodocally when a compaction is running.  If a compaction that was stuck starts again, but watcher.run() is not called because of timing issues (like the previously stuck compaction quickly completes) then it would continue to look stuck.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionWatcher.java
##########
@@ -29,14 +29,13 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.compaction.CompactionInfo;
-import org.apache.accumulo.server.compaction.FileCompactor;
 import org.slf4j.LoggerFactory;
 
 public class CompactionWatcher implements Runnable {
   private final Map<List<Long>,ObservedCompactionInfo> observedCompactions = new HashMap<>();
   private final AccumuloConfiguration config;
   private static boolean watching = false;
+  private static long totalStuck = 0;

Review comment:
       This should probably be volatile or an atomic long




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] EdColeman commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
EdColeman commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777665983



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {
+    private final long warnTime;
+    private long time;
+    private long read;
+    private long written;
+    private boolean stuck = false;
+
+    public CompactionProgress(AccumuloConfiguration config) {
+      warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
+    }
+
+    @Override
+    public void registerMetrics(MeterRegistry registry) {
+      Gauge.builder(METRICS_TSERVER_MAJC_STUCK, this, CompactionProgress::getNumStuck)

Review comment:
       +1 to the summation across all.
   
   I think I am just triggered by the compactor count of 0 or 1 being a metric.  I guess I'm asking: Can it be kept as a count - but not as a metric? Or, if kept as a metric provide additional information?




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] EdColeman commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
EdColeman commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777661423



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {
+    private final long warnTime;
+    private long time;
+    private long read;
+    private long written;
+    private boolean stuck = false;
+
+    public CompactionProgress(AccumuloConfiguration config) {
+      warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
+    }
+
+    @Override
+    public void registerMetrics(MeterRegistry registry) {
+      Gauge.builder(METRICS_TSERVER_MAJC_STUCK, this, CompactionProgress::getNumStuck)

Review comment:
       From looking at line 134 - it seemed that the value would only be 0 or 1 - and that there is a summation occurring later?
   
   It seemed that more info could be conveyed other than a boolean. If it is being summed elsewhere, does it need to be a metric here?
   
   I could just be reading this wrong...




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2405: Created additional metrics for long assignments and stuck compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2405:
URL: https://github.com/apache/accumulo/pull/2405#discussion_r777657761



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -107,8 +108,58 @@
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class Compactor extends AbstractServer implements CompactorService.Iface {
 
+  private static class CompactionProgress implements MetricsProducer {
+    private final long warnTime;
+    private long time;
+    private long read;
+    private long written;
+    private boolean stuck = false;
+
+    public CompactionProgress(AccumuloConfiguration config) {
+      warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
+    }
+
+    @Override
+    public void registerMetrics(MeterRegistry registry) {
+      Gauge.builder(METRICS_TSERVER_MAJC_STUCK, this, CompactionProgress::getNumStuck)

Review comment:
       So, each Compactor runs one majc at a time, but the TabletServer can run many of them at once. I was approaching this from the perspective that a user could just plot stuck majc's over time and be able to drill down to one of the processes. For your suggestion to work, I think I would need to emit multiple Timers , one per stuck extent?




-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org