You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ji...@apache.org on 2019/06/11 20:08:00 UTC

[incubator-pinot] branch master updated: [TE] Detection model maintenance flow (#4266)

This is an automated email from the ASF dual-hosted git repository.

jihao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 6c4138f  [TE] Detection model maintenance flow (#4266)
6c4138f is described below

commit 6c4138fc98ba11d2ad82816581f008197679bd8f
Author: Jihao Zhang <ji...@linkedin.com>
AuthorDate: Tue Jun 11 13:07:53 2019 -0700

    [TE] Detection model maintenance flow (#4266)
    
    - implement detection model maintenance flow to check the status of detection and re-tunes the model automatically
    - unit tests
---
 .../thirdeye/datalayer/dto/EvaluationDTO.java      |   9 +-
 .../datalayer/pojo/DetectionConfigBean.java        |   9 ++
 .../thirdeye/datalayer/pojo/EvaluationBean.java    |   6 +-
 .../detection/DetectionPipelineTaskRunner.java     |  10 +-
 .../thirdeye/detection/ModelMaintenanceFlow.java   |  41 ++++++++
 .../pinot/thirdeye/detection/ModelRetuneFlow.java  | 110 ++++++++++++++++++++
 .../annotation/registry/DetectionRegistry.java     |  21 ++++
 .../MapeAveragePercentageChangeModelEvaluator.java |   4 +-
 .../detection/DefaultModelMaintenanceFlowTest.java | 111 +++++++++++++++++++++
 .../MapePercentageChangeModelEvaluatorTest.java    |   2 +-
 .../detection/components/MockModelEvaluator.java   |  45 +++++++++
 .../detection/components/MockTunableDetector.java  |  60 +++++++++++
 .../detection/spec/MockModelEvaluatorSpec.java     |  38 +++++++
 .../thirdeye/detection/spec/MockTunableSpec.java   |  26 +++++
 14 files changed, 484 insertions(+), 8 deletions(-)

diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dto/EvaluationDTO.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dto/EvaluationDTO.java
index 19d0cac..90a5b01 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dto/EvaluationDTO.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dto/EvaluationDTO.java
@@ -46,10 +46,15 @@ public class EvaluationDTO extends EvaluationBean {
     return evaluation;
   }
 
-  private static double getMape(PredictionResult result) {
+  private static Double getMape(PredictionResult result) {
     DataFrame df = result.getPredictedTimeSeries();
     // drop zero current value for mape calculation
     df = df.filter(df.getDoubles(COL_CURRENT).ne(0.0)).dropNull(COL_CURRENT, COL_VALUE);
-    return Evaluation.calculateMape(df.getDoubles(COL_CURRENT), df.getDoubles(COL_VALUE));
+    Double mape = Evaluation.calculateMape(df.getDoubles(COL_CURRENT), df.getDoubles(COL_VALUE));
+    if (Double.isNaN(mape)) {
+      // explicitly swap NaN to null values because mysql doesn't support storing NaN and will throw an exception.
+      mape = null;
+    }
+    return mape;
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/pojo/DetectionConfigBean.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/pojo/DetectionConfigBean.java
index 6294405..0be2cf1 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/pojo/DetectionConfigBean.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/pojo/DetectionConfigBean.java
@@ -41,6 +41,7 @@ public class DetectionConfigBean extends AbstractBean {
   boolean active;
   String yaml;
   Map<String, Object> componentSpecs;
+  long lastTuningTimestamp;
 
   public Map<String, Object> getComponentSpecs() {
     return componentSpecs;
@@ -98,6 +99,14 @@ public class DetectionConfigBean extends AbstractBean {
     this.lastTimestamp = lastTimestamp;
   }
 
+  public long getLastTuningTimestamp() {
+    return lastTuningTimestamp;
+  }
+
+  public void setLastTuningTimestamp(long lastTuningTimestamp) {
+    this.lastTuningTimestamp = lastTuningTimestamp;
+  }
+
   public boolean isActive() {
     return active;
   }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/pojo/EvaluationBean.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/pojo/EvaluationBean.java
index e141b23..5fe0a3c 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/pojo/EvaluationBean.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/pojo/EvaluationBean.java
@@ -34,7 +34,7 @@ public class EvaluationBean extends AbstractBean {
   private long startTime; // the start time for the detection window being monitored
   private long endTime; // the end time for the detection window being monitored
   private String detectorName; // the name for the detector
-  private double mape; //  the mean absolute percentage error (MAPE)
+  private Double mape; //  the mean absolute percentage error (MAPE)
   private String metricUrn; // the metric urn
 
   public long getDetectionConfigId() {
@@ -69,11 +69,11 @@ public class EvaluationBean extends AbstractBean {
     this.detectorName = detectorName;
   }
 
-  public double getMape() {
+  public Double getMape() {
     return mape;
   }
 
-  public void setMape(double mape) {
+  public void setMape(Double mape) {
     this.mape = mape;
   }
 
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionPipelineTaskRunner.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionPipelineTaskRunner.java
index 716917b..5a52a1b 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionPipelineTaskRunner.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionPipelineTaskRunner.java
@@ -41,6 +41,8 @@ import org.apache.pinot.thirdeye.datasource.loader.AggregationLoader;
 import org.apache.pinot.thirdeye.datasource.loader.DefaultAggregationLoader;
 import org.apache.pinot.thirdeye.datasource.loader.DefaultTimeSeriesLoader;
 import org.apache.pinot.thirdeye.datasource.loader.TimeSeriesLoader;
+import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionRegistry;
+import org.joda.time.Instant;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,6 +54,7 @@ public class DetectionPipelineTaskRunner implements TaskRunner {
   private final EvaluationManager evaluationDAO;
   private final DetectionPipelineLoader loader;
   private final DataProvider provider;
+  private final ModelMaintenanceFlow maintenanceFlow;
 
   /**
    * Default constructor for ThirdEye task execution framework.
@@ -80,6 +83,7 @@ public class DetectionPipelineTaskRunner implements TaskRunner {
 
     this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, this.anomalyDAO, this.evaluationDAO,
         timeseriesLoader, aggregationLoader, this.loader);
+    this.maintenanceFlow = new ModelRetuneFlow(this.provider, DetectionRegistry.getInstance());
   }
 
   /**
@@ -98,6 +102,7 @@ public class DetectionPipelineTaskRunner implements TaskRunner {
     this.evaluationDAO = evaluationDAO;
     this.loader = loader;
     this.provider = provider;
+    this.maintenanceFlow = new ModelRetuneFlow(this.provider, DetectionRegistry.getInstance());
   }
 
   @Override
@@ -133,6 +138,10 @@ public class DetectionPipelineTaskRunner implements TaskRunner {
         this.evaluationDAO.save(evaluationDTO);
       }
 
+      // run maintenance flow to update model
+      config = maintenanceFlow.maintain(config, Instant.now());
+      this.detectionDAO.update(config);
+
       return Collections.emptyList();
 
     } catch(Exception e) {
@@ -143,5 +152,4 @@ public class DetectionPipelineTaskRunner implements TaskRunner {
       ThirdeyeMetricsUtil.detectionTaskSuccessCounter.inc();
     }
   }
-
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/ModelMaintenanceFlow.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/ModelMaintenanceFlow.java
new file mode 100644
index 0000000..57a1437
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/ModelMaintenanceFlow.java
@@ -0,0 +1,41 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ *
+ */
+
+package org.apache.pinot.thirdeye.detection;
+
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.joda.time.Instant;
+
+
+/**
+ * The model maintenance flow. This flow re-tunes the detection model automatically if the the model's performance is bad.
+ */
+public interface ModelMaintenanceFlow {
+
+  /**
+   * Maintain the detection model
+   * @param detectionConfig the detection config to maintain
+   * @param timestamp the time stamp of this maintenance
+   * @return the maintained detection config
+   */
+  DetectionConfigDTO maintain(DetectionConfigDTO detectionConfig, Instant timestamp);
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/ModelRetuneFlow.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/ModelRetuneFlow.java
new file mode 100644
index 0000000..d3d4a96
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/ModelRetuneFlow.java
@@ -0,0 +1,110 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ *
+ */
+
+package org.apache.pinot.thirdeye.detection;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.stream.Collectors;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionRegistry;
+import org.apache.pinot.thirdeye.detection.components.MapeAveragePercentageChangeModelEvaluator;
+import org.apache.pinot.thirdeye.detection.spec.AbstractSpec;
+import org.apache.pinot.thirdeye.detection.spec.MapeAveragePercentageChangeModelEvaluatorSpec;
+import org.apache.pinot.thirdeye.detection.spi.components.ModelEvaluator;
+import org.apache.pinot.thirdeye.detection.spi.model.ModelStatus;
+import org.apache.pinot.thirdeye.detection.yaml.DetectionConfigTuner;
+import org.joda.time.Instant;
+
+
+/**
+ * The default model maintenance flow. If the model is tunable, this flow will run the configured model evaluators for
+ * the detection config and automatically re-tunes the model.
+ */
+public class ModelRetuneFlow implements ModelMaintenanceFlow {
+  private static final int DEFAULT_TUNING_WINDOW_DAYS = 28;
+
+  private final DataProvider provider;
+  private final DetectionRegistry detectionRegistry;
+
+  ModelRetuneFlow(DataProvider provider, DetectionRegistry detectionRegistry) {
+    this.provider = provider;
+    this.detectionRegistry = detectionRegistry;
+  }
+
+  public DetectionConfigDTO maintain(DetectionConfigDTO config, Instant timestamp) {
+    if (isTunable(config)) {
+      // if the pipeline is tunable, get the model evaluators
+      Collection<? extends ModelEvaluator<? extends AbstractSpec>> modelEvaluators = getModelEvaluators(config);
+      // check the status for model evaluators
+      for (ModelEvaluator<? extends AbstractSpec> modelEvaluator : modelEvaluators) {
+        // if returns bad model status, trigger model tuning
+        if (modelEvaluator.evaluateModel(timestamp).getStatus().equals(ModelStatus.BAD)) {
+          DetectionConfigTuner detectionConfigTuner = new DetectionConfigTuner(config, provider);
+          config = detectionConfigTuner.tune(timestamp.toDateTime().minusDays(DEFAULT_TUNING_WINDOW_DAYS).getMillis(),
+              timestamp.getMillis());
+          config.setLastTuningTimestamp(timestamp.getMillis());
+          break;
+        }
+      }
+    }
+    return config;
+  }
+
+  private Collection<? extends ModelEvaluator<? extends AbstractSpec>> getModelEvaluators(DetectionConfigDTO config) {
+    // get the model evaluator in the detection config
+    Collection<? extends ModelEvaluator<? extends AbstractSpec>> modelEvaluators = config.getComponents()
+        .values()
+        .stream()
+        .filter(component -> component instanceof ModelEvaluator)
+        .map(component -> (ModelEvaluator<? extends AbstractSpec>) component)
+        .collect(Collectors.toList());
+
+    if (modelEvaluators.isEmpty()) {
+      // if evaluators are not configured, use the default ones
+      modelEvaluators = instantiateDefaultEvaluators(config);
+    }
+
+    return modelEvaluators;
+  }
+
+  private Collection<ModelEvaluator<MapeAveragePercentageChangeModelEvaluatorSpec>> instantiateDefaultEvaluators(
+      DetectionConfigDTO config) {
+    ModelEvaluator<MapeAveragePercentageChangeModelEvaluatorSpec> evaluator =
+        new MapeAveragePercentageChangeModelEvaluator();
+    evaluator.init(new MapeAveragePercentageChangeModelEvaluatorSpec(),
+        new DefaultInputDataFetcher(this.provider, config.getId()));
+    return Collections.singleton(evaluator);
+  }
+
+  /**
+   * If the detection config contains a tunable component
+   * @param configDTO the detection config
+   * @return True if the detection config is contains tunable component
+   */
+  private boolean isTunable(DetectionConfigDTO configDTO) {
+    return configDTO.getComponents()
+        .values()
+        .stream()
+        .anyMatch(component -> this.detectionRegistry.isTunable(component.getClass().getName()));
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/annotation/registry/DetectionRegistry.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/annotation/registry/DetectionRegistry.java
index 4fe121c..c14ff15 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/annotation/registry/DetectionRegistry.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/annotation/registry/DetectionRegistry.java
@@ -115,6 +115,27 @@ public class DetectionRegistry {
     }
   }
 
+  public static void registerTunableComponent(String className, String tunable, String type) {
+    try {
+      Class<? extends BaseComponent> clazz = (Class<? extends BaseComponent>) Class.forName(className);
+      REGISTRY_MAP.put(type, ImmutableMap.of(KEY_CLASS_NAME, className, KEY_IS_BASELINE_PROVIDER, isBaselineProvider(clazz)));
+      Tune tune = new Tune(){
+        @Override
+        public String tunable() {
+          return tunable;
+        }
+
+        @Override
+        public Class<? extends Annotation> annotationType() {
+          return Tune.class;
+        }
+      };
+      TUNE_MAP.put(className, tune);
+    } catch (Exception e) {
+      LOG.warn("Encountered exception when registering component {}", className, e);
+    }
+  }
+
   public static void registerYamlConvertor(String className, String type) {
     YAML_MAP.put(type, className);
   }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/MapeAveragePercentageChangeModelEvaluator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/MapeAveragePercentageChangeModelEvaluator.java
index cff9d7d..5f132f3 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/MapeAveragePercentageChangeModelEvaluator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/MapeAveragePercentageChangeModelEvaluator.java
@@ -25,6 +25,7 @@ package org.apache.pinot.thirdeye.detection.components;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
+import java.util.Objects;
 import java.util.stream.Collectors;
 import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
 import org.apache.pinot.thirdeye.datalayer.pojo.EvaluationBean;
@@ -101,12 +102,13 @@ public class MapeAveragePercentageChangeModelEvaluator implements ModelEvaluator
   private Collection<EvaluationDTO> getEvaluationsWithinDays(Collection<EvaluationDTO> evaluations,
       Instant evaluationTimeStamp, int days) {
     return evaluations.stream()
+        .filter(eval -> Objects.nonNull(eval.getMape()))
         .filter(eval -> evaluationTimeStamp.toDateTime().minusDays(days).getMillis() < eval.getStartTime())
         .collect(Collectors.toSet());
   }
 
   /**
-   * calculate the mean MAPE for each metric urn based on the available evaluations over the past numbe of days
+   * calculate the mean MAPE for each metric urn based on the available evaluations over the past number of days
    * @param evaluations the available evaluations
    * @return the mean MAPE keyed by metric urns
    */
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DefaultModelMaintenanceFlowTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DefaultModelMaintenanceFlowTest.java
new file mode 100644
index 0000000..a862800
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DefaultModelMaintenanceFlowTest.java
@@ -0,0 +1,111 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ *
+ */
+
+package org.apache.pinot.thirdeye.detection;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.Collections;
+import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
+import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionRegistry;
+import org.apache.pinot.thirdeye.detection.components.MockModelEvaluator;
+import org.apache.pinot.thirdeye.detection.components.MockTunableDetector;
+import org.apache.pinot.thirdeye.detection.spec.MockModelEvaluatorSpec;
+import org.apache.pinot.thirdeye.detection.spi.components.ModelEvaluator;
+import org.apache.pinot.thirdeye.detection.spi.model.ModelStatus;
+import org.joda.time.Instant;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+
+public class DefaultModelMaintenanceFlowTest {
+  private static final String METRIC_NAME = "test_metric";
+  private static final String DATASET_NAME = "test_dataset";
+
+  private ModelMaintenanceFlow maintenanceFlow;
+  private DataProvider provider;
+  private InputDataFetcher dataFetcher;
+  private long configId;
+
+  @BeforeMethod
+  public void setUp() {
+    MetricConfigDTO metric = new MetricConfigDTO();
+    metric.setId(101L);
+    metric.setName(METRIC_NAME);
+    metric.setDataset(DATASET_NAME);
+    DatasetConfigDTO dataset = new DatasetConfigDTO();
+    dataset.setId(102L);
+    dataset.setDataset(DATASET_NAME);
+    this.provider = new MockDataProvider().setMetrics(Collections.singletonList(metric)).setDatasets(Collections.singletonList(dataset));
+    this.configId = 100L;
+    this.dataFetcher = new DefaultInputDataFetcher(this.provider, this.configId);
+    this.maintenanceFlow = new ModelRetuneFlow(this.provider, DetectionRegistry.getInstance());
+  }
+
+  @Test
+  public void testMaintainNotTunable() {
+    DetectionConfigDTO configDTO = new DetectionConfigDTO();
+    configDTO.setId(this.configId);
+    ModelEvaluator evaluator = new MockModelEvaluator();
+    MockModelEvaluatorSpec spec = new MockModelEvaluatorSpec();
+    spec.setMockModelStatus(ModelStatus.GOOD);
+    configDTO.setComponents(ImmutableMap.of("evaluator_1", evaluator));
+    DetectionConfigDTO maintainedConfig = this.maintenanceFlow.maintain(configDTO, Instant.now());
+    Assert.assertEquals(configDTO, maintainedConfig);
+  }
+
+  @Test
+  public void testMaintainTunableGood() {
+    DetectionConfigDTO configDTO = new DetectionConfigDTO();
+    configDTO.setId(this.configId);
+    ModelEvaluator evaluator = new MockModelEvaluator();
+    MockModelEvaluatorSpec spec = new MockModelEvaluatorSpec();
+    spec.setMockModelStatus(ModelStatus.GOOD);
+    evaluator.init(spec, this.dataFetcher);
+    MockTunableDetector detector = new MockTunableDetector();
+    configDTO.setComponents(ImmutableMap.of("evaluator_1", evaluator, "detector", detector));
+    DetectionConfigDTO maintainedConfig = this.maintenanceFlow.maintain(configDTO, Instant.now());
+    Assert.assertEquals(configDTO, maintainedConfig);
+  }
+
+  @Test
+  public void testMaintainTunableBad() {
+    DetectionRegistry.registerTunableComponent(MockTunableDetector.class.getName(), "MOCK_TUNABLE", "MOCK_TUNABLE");
+    DetectionConfigDTO configDTO = new DetectionConfigDTO();
+    configDTO.setId(this.configId);
+    configDTO.setYaml(String.format("metric: %s\ndataset: %s\n", METRIC_NAME, DATASET_NAME));
+    ModelEvaluator evaluator = new MockModelEvaluator();
+    MockModelEvaluatorSpec spec = new MockModelEvaluatorSpec();
+    spec.setMockModelStatus(ModelStatus.BAD);
+    evaluator.init(spec, this.dataFetcher);
+    MockTunableDetector tunableDetector = new MockTunableDetector();
+    configDTO.setComponents(ImmutableMap.of("evaluator_1", evaluator, "detector", tunableDetector));
+    configDTO.setLastTuningTimestamp(1559175301000L);
+    configDTO.setComponentSpecs(ImmutableMap.of("detector:MOCK_TUNABLE", ImmutableMap.of("className", MockTunableDetector.class.getName())));
+    Instant maintainTimestamp = Instant.now();
+    DetectionConfigDTO maintainedConfig = this.maintenanceFlow.maintain(configDTO, maintainTimestamp);
+    Assert.assertEquals(maintainedConfig.getLastTuningTimestamp(), maintainTimestamp.getMillis());
+  }
+
+}
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MapePercentageChangeModelEvaluatorTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MapePercentageChangeModelEvaluatorTest.java
index e7e0a55..3caff11 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MapePercentageChangeModelEvaluatorTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MapePercentageChangeModelEvaluatorTest.java
@@ -50,7 +50,7 @@ public class MapePercentageChangeModelEvaluatorTest {
     dataFetcher = new DefaultInputDataFetcher(dataProvider, mockDetectionConfigId);
   }
 
-  private EvaluationDTO makeMockEvaluationDTO(long mockDetectionConfigId, String mockMetricUrn, long start, long end, double mape) {
+  private EvaluationDTO makeMockEvaluationDTO(long mockDetectionConfigId, String mockMetricUrn, long start, long end, Double mape) {
     EvaluationDTO eval = new EvaluationDTO();
     eval.setStartTime(start);
     eval.setEndTime(end);
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MockModelEvaluator.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MockModelEvaluator.java
new file mode 100644
index 0000000..b08ddfe
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MockModelEvaluator.java
@@ -0,0 +1,45 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ *
+ */
+
+package org.apache.pinot.thirdeye.detection.components;
+
+import org.apache.pinot.thirdeye.detection.InputDataFetcher;
+import org.apache.pinot.thirdeye.detection.spec.MockModelEvaluatorSpec;
+import org.apache.pinot.thirdeye.detection.spi.components.ModelEvaluator;
+import org.apache.pinot.thirdeye.detection.spi.model.ModelEvaluationResult;
+import org.apache.pinot.thirdeye.detection.spi.model.ModelStatus;
+import org.joda.time.Instant;
+
+
+public class MockModelEvaluator implements ModelEvaluator<MockModelEvaluatorSpec> {
+  private ModelStatus status;
+
+  @Override
+  public ModelEvaluationResult evaluateModel(Instant evaluationTimeStamp) {
+    return new ModelEvaluationResult(status);
+  }
+
+  @Override
+  public void init(MockModelEvaluatorSpec spec, InputDataFetcher dataFetcher) {
+    this.status = spec.getMockModelStatus();
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MockTunableDetector.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MockTunableDetector.java
new file mode 100644
index 0000000..af3dbe3
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MockTunableDetector.java
@@ -0,0 +1,60 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ *
+ */
+
+package org.apache.pinot.thirdeye.detection.components;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.pinot.thirdeye.detection.InputDataFetcher;
+import org.apache.pinot.thirdeye.detection.annotation.Components;
+import org.apache.pinot.thirdeye.detection.annotation.Tune;
+import org.apache.pinot.thirdeye.detection.spec.MockTunableSpec;
+import org.apache.pinot.thirdeye.detection.spi.components.AnomalyDetector;
+import org.apache.pinot.thirdeye.detection.spi.components.Tunable;
+import org.apache.pinot.thirdeye.detection.spi.exception.DetectorException;
+import org.apache.pinot.thirdeye.detection.spi.model.DetectionResult;
+import org.joda.time.Interval;
+
+public class MockTunableDetector implements AnomalyDetector<MockTunableSpec>, Tunable<MockTunableSpec> {
+  private int tuneRunes = 0;
+
+  public int getTuneRuns() {
+    return tuneRunes;
+  }
+
+
+  @Override
+  public Map<String, Object> tune(Map<String, Object> currentSpec, Interval tuningWindow, String metricUrn) {
+    this.tuneRunes++;
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public DetectionResult runDetection(Interval window, String metricUrn) throws DetectorException {
+    return DetectionResult.empty();
+  }
+
+  @Override
+  public void init(MockTunableSpec spec, InputDataFetcher dataFetcher) {
+    // left empty
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/spec/MockModelEvaluatorSpec.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/spec/MockModelEvaluatorSpec.java
new file mode 100644
index 0000000..bd85902
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/spec/MockModelEvaluatorSpec.java
@@ -0,0 +1,38 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ *
+ */
+
+package org.apache.pinot.thirdeye.detection.spec;
+
+import org.apache.pinot.thirdeye.detection.spi.model.ModelStatus;
+
+
+public class MockModelEvaluatorSpec extends AbstractSpec {
+  private ModelStatus mockModelStatus;
+
+  public ModelStatus getMockModelStatus() {
+    return mockModelStatus;
+  }
+
+  public void setMockModelStatus(ModelStatus mockModelStatus) {
+    this.mockModelStatus = mockModelStatus;
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/spec/MockTunableSpec.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/spec/MockTunableSpec.java
new file mode 100644
index 0000000..d19be61
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/spec/MockTunableSpec.java
@@ -0,0 +1,26 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ *
+ */
+
+package org.apache.pinot.thirdeye.detection.spec;
+
+public class MockTunableSpec extends AbstractSpec {
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org