You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@heron.apache.org by GitBox <gi...@apache.org> on 2018/04/04 22:11:20 UTC

[GitHub] ashvina commented on a change in pull request #2821: Update Dhalion dependency version

ashvina commented on a change in pull request #2821: Update Dhalion dependency version
URL: https://github.com/apache/incubator-heron/pull/2821#discussion_r179298554
 
 

 ##########
 File path: heron/healthmgr/src/java/com/twitter/heron/healthmgr/detectors/GrowingWaitQueueDetector.java
 ##########
 @@ -16,58 +16,89 @@
 package com.twitter.heron.healthmgr.detectors;
 
 import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
 import java.util.logging.Logger;
 
 import javax.inject.Inject;
 
-import com.microsoft.dhalion.api.IDetector;
-import com.microsoft.dhalion.detector.Symptom;
-import com.microsoft.dhalion.metrics.ComponentMetrics;
+import com.microsoft.dhalion.core.Measurement;
+import com.microsoft.dhalion.core.MeasurementsTable;
+import com.microsoft.dhalion.core.Symptom;
+
+import org.apache.commons.math3.stat.regression.SimpleRegression;
 
 import com.twitter.heron.healthmgr.HealthPolicyConfig;
-import com.twitter.heron.healthmgr.common.ComponentMetricsHelper;
-import com.twitter.heron.healthmgr.sensors.BufferSizeSensor;
 
-import static com.twitter.heron.healthmgr.detectors.BaseDetector.SymptomName.SYMPTOM_GROWING_WAIT_Q;
+import static com.twitter.heron.healthmgr.detectors.BaseDetector.SymptomType.SYMPTOM_GROWING_WAIT_Q;
+import static com.twitter.heron.healthmgr.sensors.BaseSensor.MetricName.METRIC_WAIT_Q_SIZE;
 
 
-public class GrowingWaitQueueDetector implements IDetector {
-  static final String CONF_LIMIT = GrowingWaitQueueDetector.class.getSimpleName() + ".limit";
+public class GrowingWaitQueueDetector extends BaseDetector {
+  static final String CONF_LIMIT
+      = GrowingWaitQueueDetector.class.getSimpleName() + ".limit";
 
   private static final Logger LOG = Logger.getLogger(GrowingWaitQueueDetector.class.getName());
-  private final BufferSizeSensor pendingBufferSensor;
   private final double rateLimit;
 
   @Inject
-  GrowingWaitQueueDetector(BufferSizeSensor pendingBufferSensor,
-                           HealthPolicyConfig policyConfig) {
-    this.pendingBufferSensor = pendingBufferSensor;
+  GrowingWaitQueueDetector(HealthPolicyConfig policyConfig) {
     rateLimit = (double) policyConfig.getConfig(CONF_LIMIT, 10.0);
   }
 
   /**
    * Detects all components unable to keep up with input load, hence having a growing pending buffer
    * or wait queue
    *
-   * @return A collection of all components executing slower than input rate.
+   * @return A collection of symptoms each one corresponding to a components executing slower
+   * than input rate.
    */
   @Override
-  public List<Symptom> detect() {
-    ArrayList<Symptom> result = new ArrayList<>();
-
-    Map<String, ComponentMetrics> bufferSizes = pendingBufferSensor.get();
-    for (ComponentMetrics compMetrics : bufferSizes.values()) {
-      ComponentMetricsHelper compStats = new ComponentMetricsHelper(compMetrics);
-      compStats.computeBufferSizeTrend();
-      if (compStats.getMaxBufferChangeRate() > rateLimit) {
+  public Collection<Symptom> detect(Collection<Measurement> measurements) {
+
+    Collection<Symptom> result = new ArrayList<>();
+
+    MeasurementsTable waitQueueMetrics = MeasurementsTable.of(measurements).type
+        (METRIC_WAIT_Q_SIZE.text());
+    for (String component : waitQueueMetrics.uniqueComponents()) {
+      Set<String> addresses = new HashSet<>();
+      double maxSlope = computeWaitQueueSizeTrend(waitQueueMetrics.component(component));
+      if (maxSlope > rateLimit) {
         LOG.info(String.format("Detected growing wait queues for %s, max rate %f",
-            compMetrics.getName(), compStats.getMaxBufferChangeRate()));
-        result.add(new Symptom(SYMPTOM_GROWING_WAIT_Q.text(), compMetrics));
+            component, maxSlope));
+        addresses.add(component);
+        result.add(new Symptom(SYMPTOM_GROWING_WAIT_Q.text(), context.checkpoint(), addresses));
       }
     }
-
     return result;
   }
+
+
+  private double computeWaitQueueSizeTrend(MeasurementsTable metrics) {
+    double maxSlope = 0;
+    for (String instance : metrics.uniqueInstances()) {
+
+      if (metrics.instance(instance) == null || metrics.instance(instance).size() < 3) {
+        // missing of insufficient data for creating a trend line
+        continue;
+      }
+
+      Collection<Measurement> measurements = metrics.instance(instance).sort(false,
+          MeasurementsTable.SortKey
+              .TIME_STAMP).get();
+      SimpleRegression simpleRegression = new SimpleRegression(true);
+
+      for (Measurement m : measurements) {
+        simpleRegression.addData(m.instant().getEpochSecond(), m.value());
 
 Review comment:
   Good point. This private is operating on a bounded set of metrics. So this should be ok. However the input set may be insufficient to compute the slope. I am thinking fo verifying and fixing (if required) in a different PR. As of now this detector is not used.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services