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/05/17 21:12:58 UTC

[incubator-pinot] branch master updated: [TE] Model evaluator interface & Implementation (#4209)

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 f50e87f  [TE] Model evaluator interface & Implementation (#4209)
f50e87f is described below

commit f50e87f14c3756f1673bc2c5d8f8b8d4cc1f74d1
Author: Jihao Zhang <ji...@linkedin.com>
AuthorDate: Fri May 17 14:12:52 2019 -0700

    [TE] Model evaluator interface & Implementation (#4209)
    
    - The boilerplate code to retrieve the evaluations in detection components, including dataProvider, InputDataFetcher, etc.
    - Define the model evaluator interface, which read the evaluation metrics and evaluate the detection model status.
    - Implement the MAPE percentage change evaluator, which compare the mean MAPE over past 7 days and past 30 days, if the up percentage change is more than a certain threshold, returns a bad detection status.
---
 .../dashboard/resources/v2/AnomaliesResource.java  |   6 +-
 .../pinot/thirdeye/detection/DataProvider.java     |  14 +++
 .../thirdeye/detection/DefaultDataProvider.java    |  31 +++++-
 .../detection/DefaultInputDataFetcher.java         |   6 +-
 .../detection/DetectionPipelineTaskRunner.java     |   7 +-
 .../thirdeye/detection/DetectionResource.java      |   5 +-
 .../detection/alert/DetectionAlertTaskFactory.java |   6 +-
 .../MapeAveragePercentageChangeModelEvaluator.java | 124 +++++++++++++++++++++
 .../finetune/GridSearchTuningAlgorithm.java        |  20 ++--
 .../onboard/YamlOnboardingTaskRunner.java          |   5 +-
 ...eAveragePercentageChangeModelEvaluatorSpec.java |  38 +++++++
 .../detection/spi/components/ModelEvaluator.java   |  41 +++++++
 .../detection/spi/model/EvaluationSlice.java       |  68 +++++++++++
 .../thirdeye/detection/spi/model/InputData.java    |  12 +-
 .../detection/spi/model/InputDataSpec.java         |  34 ++++--
 .../detection/spi/model/ModelEvaluationResult.java |  39 +++++++
 .../thirdeye/detection/spi/model/ModelStatus.java  |  30 +++++
 .../validators/DetectionConfigValidator.java       |   1 +
 .../thirdeye/detection/yaml/YamlResource.java      |   5 +-
 .../pinot/thirdeye/detection/DataProviderTest.java |   6 +-
 .../pinot/thirdeye/detection/MockDataProvider.java |  25 +++++
 .../MapePercentageChangeModelEvaluatorTest.java    |  82 ++++++++++++++
 22 files changed, 571 insertions(+), 34 deletions(-)

diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/AnomaliesResource.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/AnomaliesResource.java
index 461ffe3..fbc7920 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/AnomaliesResource.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/AnomaliesResource.java
@@ -82,6 +82,7 @@ import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.bao.AnomalyFunctionManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.EvaluationManager;
 import org.apache.pinot.thirdeye.datalayer.bao.EventManager;
 import org.apache.pinot.thirdeye.datalayer.bao.GroupedAnomalyResultsManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
@@ -155,6 +156,7 @@ public class AnomaliesResource {
   private final DetectionConfigManager detectionDAO;
   private final EventManager eventDAO;
   private final MergedAnomalyResultManager anomalyDAO;
+  private final EvaluationManager evaluationDAO;
 
   private final ExecutorService threadPool;
   private final AlertFilterFactory alertFilterFactory;
@@ -177,6 +179,7 @@ public class AnomaliesResource {
     this.anomalyFunctionFactory = anomalyFunctionFactory;
     this.eventDAO = DAORegistry.getInstance().getEventDAO();
     this.anomalyDAO = DAORegistry.getInstance().getMergedAnomalyResultDAO();
+    this.evaluationDAO = DAO_REGISTRY.getEvaluationManager();
 
     QueryCache queryCache = ThirdEyeCacheRegistry.getInstance().getQueryCache();
     LoadingCache<String, Long> maxTimeCache = ThirdEyeCacheRegistry.getInstance().getDatasetMaxDataTimeCache();
@@ -185,7 +188,8 @@ public class AnomaliesResource {
     this.aggregationLoader = new DefaultAggregationLoader(this.metricConfigDAO, this.datasetConfigDAO, queryCache, maxTimeCache);
     this.loader = new DetectionPipelineLoader();
 
-    this.provider = new DefaultDataProvider(this.metricConfigDAO, this.datasetConfigDAO, this.eventDAO, this.anomalyDAO, this.timeSeriesLoader, this.aggregationLoader, this.loader);
+    this.provider = new DefaultDataProvider(this.metricConfigDAO, this.datasetConfigDAO, this.eventDAO, this.anomalyDAO, this.evaluationDAO,
+        this.timeSeriesLoader, this.aggregationLoader, this.loader);
   }
 
   @GET
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DataProvider.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DataProvider.java
index 4d80598..fb6e902 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DataProvider.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DataProvider.java
@@ -23,14 +23,17 @@ import com.google.common.collect.Multimap;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import org.apache.commons.lang.NotImplementedException;
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.EventDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
 import org.apache.pinot.thirdeye.detection.spi.model.AnomalySlice;
+import org.apache.pinot.thirdeye.detection.spi.model.EvaluationSlice;
 import org.apache.pinot.thirdeye.detection.spi.model.EventSlice;
 
 
@@ -136,4 +139,15 @@ public interface DataProvider {
    * @throws Exception
    */
   DetectionPipeline loadPipeline(DetectionConfigDTO config, long start, long end) throws Exception;
+
+  /**
+   * Returns a multimap of evaluations (keyed by the evaluations slice) for a given set of evaluations slices.
+   *
+   * @see Evaluation
+   *
+   * @param evaluationSlices the evaluation slices
+   * @param configId configId
+   * @return a multimap of evaluations (keyed by the evaluations slice)
+   */
+  Multimap<EvaluationSlice, EvaluationDTO> fetchEvaluations(Collection<EvaluationSlice> evaluationSlices, long configId);
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DefaultDataProvider.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DefaultDataProvider.java
index 81b5bd1..4d890bf 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DefaultDataProvider.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DefaultDataProvider.java
@@ -45,11 +45,13 @@ import org.apache.pinot.thirdeye.dataframe.DataFrame;
 import org.apache.pinot.thirdeye.dataframe.LongSeries;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.EvaluationManager;
 import org.apache.pinot.thirdeye.datalayer.bao.EventManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.EventDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
@@ -58,6 +60,7 @@ import org.apache.pinot.thirdeye.datasource.comparison.Row;
 import org.apache.pinot.thirdeye.datasource.loader.AggregationLoader;
 import org.apache.pinot.thirdeye.datasource.loader.TimeSeriesLoader;
 import org.apache.pinot.thirdeye.detection.spi.model.AnomalySlice;
+import org.apache.pinot.thirdeye.detection.spi.model.EvaluationSlice;
 import org.apache.pinot.thirdeye.detection.spi.model.EventSlice;
 import org.joda.time.DateTimeZone;
 import org.slf4j.Logger;
@@ -76,19 +79,20 @@ public class DefaultDataProvider implements DataProvider {
   private final DatasetConfigManager datasetDAO;
   private final EventManager eventDAO;
   private final MergedAnomalyResultManager anomalyDAO;
+  private final EvaluationManager evaluationDAO;
   private final TimeSeriesLoader timeseriesLoader;
   private final AggregationLoader aggregationLoader;
   private final DetectionPipelineLoader loader;
   private static LoadingCache<MetricSlice, DataFrame> DETECTION_TIME_SERIES_CACHE;
 
-
   public DefaultDataProvider(MetricConfigManager metricDAO, DatasetConfigManager datasetDAO, EventManager eventDAO,
-      MergedAnomalyResultManager anomalyDAO, TimeSeriesLoader timeseriesLoader, AggregationLoader aggregationLoader,
-      DetectionPipelineLoader loader) {
+      MergedAnomalyResultManager anomalyDAO, EvaluationManager evaluationDAO, TimeSeriesLoader timeseriesLoader,
+      AggregationLoader aggregationLoader, DetectionPipelineLoader loader) {
     this.metricDAO = metricDAO;
     this.datasetDAO = datasetDAO;
     this.eventDAO = eventDAO;
     this.anomalyDAO = anomalyDAO;
+    this.evaluationDAO = evaluationDAO;
     this.timeseriesLoader = timeseriesLoader;
     this.aggregationLoader = aggregationLoader;
     this.loader = loader;
@@ -316,6 +320,27 @@ public class DefaultDataProvider implements DataProvider {
     return this.metricDAO.findByMetricAndDataset(metricName, datasetName);
   }
 
+  @Override
+  public Multimap<EvaluationSlice, EvaluationDTO> fetchEvaluations(Collection<EvaluationSlice> slices, long configId) {
+    Multimap<EvaluationSlice, EvaluationDTO> output = ArrayListMultimap.create();
+    for (EvaluationSlice slice : slices) {
+      List<Predicate> predicates = new ArrayList<>();
+      if (slice.getEnd() >= 0)
+        predicates.add(Predicate.LT("startTime", slice.getEnd()));
+      if (slice.getStart() >= 0)
+        predicates.add(Predicate.GT("endTime", slice.getStart()));
+      if (predicates.isEmpty())
+        throw new IllegalArgumentException("Must provide at least one of start, or end");
+
+      if (configId >= 0) {
+        predicates.add(Predicate.EQ("detectionConfigId", configId));
+      }
+      List<EvaluationDTO> evaluations = this.evaluationDAO.findByPredicate(AND(predicates));
+      output.putAll(slice, evaluations.stream().filter(slice::match).collect(Collectors.toList()));
+    }
+    return output;
+  }
+
   private static Predicate AND(Collection<Predicate> predicates) {
     return Predicate.AND(predicates.toArray(new Predicate[predicates.size()]));
   }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DefaultInputDataFetcher.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DefaultInputDataFetcher.java
index 43c3db4..3c72011 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DefaultInputDataFetcher.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DefaultInputDataFetcher.java
@@ -23,10 +23,12 @@ import com.google.common.collect.Multimap;
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.EventDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
 import org.apache.pinot.thirdeye.detection.spi.model.AnomalySlice;
+import org.apache.pinot.thirdeye.detection.spi.model.EvaluationSlice;
 import org.apache.pinot.thirdeye.detection.spi.model.EventSlice;
 import org.apache.pinot.thirdeye.detection.spi.model.InputData;
 import org.apache.pinot.thirdeye.detection.spi.model.InputDataSpec;
@@ -59,10 +61,10 @@ public class DefaultInputDataFetcher implements InputDataFetcher {
     Multimap<EventSlice, EventDTO> events = provider.fetchEvents(inputDataSpec.getEventSlices());
     Map<Long, MetricConfigDTO> metrics = provider.fetchMetrics(inputDataSpec.getMetricIds());
     Map<String, DatasetConfigDTO> datasets = provider.fetchDatasets(inputDataSpec.getDatasetNames());
-
+    Multimap<EvaluationSlice, EvaluationDTO> evaluations = provider.fetchEvaluations(inputDataSpec.getEvaluationSlices(), configId);
     Map<Long, DatasetConfigDTO> datasetForMetricId = fetchDatasetForMetricId(inputDataSpec.getMetricIdsForDatasets());
     Map<InputDataSpec.MetricAndDatasetName, MetricConfigDTO> metricForMetricAndDatasetName = fetchMetricForDatasetAndMetricNames(inputDataSpec.getMetricAndDatasetNames());
-    return new InputData(inputDataSpec, timeseries, aggregates, existingAnomalies, events, metrics, datasets, datasetForMetricId, metricForMetricAndDatasetName);
+    return new InputData(inputDataSpec, timeseries, aggregates, existingAnomalies, events, metrics, datasets, evaluations, datasetForMetricId, metricForMetricAndDatasetName);
   }
 
   private Map<InputDataSpec.MetricAndDatasetName, MetricConfigDTO> fetchMetricForDatasetAndMetricNames(Collection<InputDataSpec.MetricAndDatasetName> metricNameAndDatasetNames){
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 42dec01..716917b 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
@@ -19,6 +19,8 @@
 
 package org.apache.pinot.thirdeye.detection;
 
+import java.util.Collections;
+import java.util.List;
 import org.apache.pinot.thirdeye.anomaly.task.TaskContext;
 import org.apache.pinot.thirdeye.anomaly.task.TaskInfo;
 import org.apache.pinot.thirdeye.anomaly.task.TaskResult;
@@ -39,8 +41,6 @@ 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 java.util.Collections;
-import java.util.List;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -78,7 +78,7 @@ public class DetectionPipelineTaskRunner implements TaskRunner {
             ThirdEyeCacheRegistry.getInstance().getQueryCache(),
             ThirdEyeCacheRegistry.getInstance().getDatasetMaxDataTimeCache());
 
-    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, this.anomalyDAO,
+    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, this.anomalyDAO, this.evaluationDAO,
         timeseriesLoader, aggregationLoader, this.loader);
   }
 
@@ -143,4 +143,5 @@ public class DetectionPipelineTaskRunner implements TaskRunner {
       ThirdeyeMetricsUtil.detectionTaskSuccessCounter.inc();
     }
   }
+
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionResource.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionResource.java
index 9558aad..cb7e761 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionResource.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionResource.java
@@ -55,6 +55,7 @@ import org.apache.pinot.thirdeye.dashboard.resources.v2.rootcause.AnomalyEventFo
 import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionAlertConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.EvaluationManager;
 import org.apache.pinot.thirdeye.datalayer.bao.EventManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
@@ -98,6 +99,7 @@ public class DetectionResource {
   private final DetectionPipelineLoader loader;
   private final DataProvider provider;
   private final DetectionConfigManager configDAO;
+  private final EvaluationManager evaluationDAO;
   private final DetectionAlertConfigManager detectionAlertConfigDAO;
 
   public DetectionResource() {
@@ -107,6 +109,7 @@ public class DetectionResource {
     this.anomalyDAO = DAORegistry.getInstance().getMergedAnomalyResultDAO();
     this.configDAO = DAORegistry.getInstance().getDetectionConfigManager();
     this.detectionAlertConfigDAO = DAORegistry.getInstance().getDetectionAlertConfigManager();
+    this.evaluationDAO = DAORegistry.getInstance().getEvaluationManager();
 
     TimeSeriesLoader timeseriesLoader =
         new DefaultTimeSeriesLoader(metricDAO, datasetDAO, ThirdEyeCacheRegistry.getInstance().getQueryCache());
@@ -117,7 +120,7 @@ public class DetectionResource {
 
     this.loader = new DetectionPipelineLoader();
 
-    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, anomalyDAO, timeseriesLoader, aggregationLoader, loader);
+    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, anomalyDAO, evaluationDAO, timeseriesLoader, aggregationLoader, loader);
   }
 
   @Path("/{id}")
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/DetectionAlertTaskFactory.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/DetectionAlertTaskFactory.java
index 0b752c6..0737ab3 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/DetectionAlertTaskFactory.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/DetectionAlertTaskFactory.java
@@ -22,10 +22,12 @@ package org.apache.pinot.thirdeye.detection.alert;
 import com.google.common.base.Preconditions;
 import org.apache.pinot.thirdeye.anomaly.ThirdEyeAnomalyConfiguration;
 import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.EvaluationManager;
 import org.apache.pinot.thirdeye.datalayer.bao.EventManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
 import org.apache.pinot.thirdeye.datasource.DAORegistry;
 import org.apache.pinot.thirdeye.datasource.ThirdEyeCacheRegistry;
 import org.apache.pinot.thirdeye.datasource.loader.AggregationLoader;
@@ -61,13 +63,13 @@ public class DetectionAlertTaskFactory {
     MetricConfigManager metricDAO = DAO_REGISTRY.getMetricConfigDAO();
     DatasetConfigManager datasetDAO = DAO_REGISTRY.getDatasetConfigDAO();
     MergedAnomalyResultManager anomalyMergedResultDAO = DAO_REGISTRY.getMergedAnomalyResultDAO();
-
+    EvaluationManager evaluationDAO = DAO_REGISTRY.getEvaluationManager();
     TimeSeriesLoader timeseriesLoader = new DefaultTimeSeriesLoader(metricDAO, datasetDAO,
         ThirdEyeCacheRegistry.getInstance().getQueryCache());
     AggregationLoader aggregationLoader = new DefaultAggregationLoader(metricDAO, datasetDAO,
         ThirdEyeCacheRegistry.getInstance().getQueryCache(),
         ThirdEyeCacheRegistry.getInstance().getDatasetMaxDataTimeCache());
-    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, anomalyMergedResultDAO,
+    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, anomalyMergedResultDAO, evaluationDAO,
         timeseriesLoader, aggregationLoader, new DetectionPipelineLoader());
   }
 
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
new file mode 100644
index 0000000..cff9d7d
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/MapeAveragePercentageChangeModelEvaluator.java
@@ -0,0 +1,124 @@
+/*
+ *
+ *  * 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.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
+import org.apache.pinot.thirdeye.datalayer.pojo.EvaluationBean;
+import org.apache.pinot.thirdeye.detection.InputDataFetcher;
+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.EvaluationSlice;
+import org.apache.pinot.thirdeye.detection.spi.model.InputDataSpec;
+import org.apache.pinot.thirdeye.detection.spi.model.ModelEvaluationResult;
+import org.apache.pinot.thirdeye.detection.spi.model.ModelStatus;
+import org.joda.time.Instant;
+
+
+/**
+ *  Monitor the recent mean MAPE in last 7 days, and compare that with the mean MAPE for the last 30 days.
+ *  If the percentage change dropped to a certain threshold for a metric urn, return a bad model status to trigger
+ *  auto configuration.
+ */
+public class MapeAveragePercentageChangeModelEvaluator implements ModelEvaluator<MapeAveragePercentageChangeModelEvaluatorSpec> {
+  private static final int MAPE_LOOK_BACK_DAYS_RECENT = 7;
+  private static final int MAPE_LOOK_BACK_DAYS_BASELINE = 30;
+
+  private InputDataFetcher dataFetcher;
+  private double threshold;
+
+  @Override
+  public ModelEvaluationResult evaluateModel(Instant evaluationTimeStamp) {
+    EvaluationSlice evaluationSlice =
+        new EvaluationSlice().withStartTime(evaluationTimeStamp.toDateTime().minusDays(MAPE_LOOK_BACK_DAYS_BASELINE).getMillis())
+            .withEndTime(evaluationTimeStamp.getMillis());
+    // fetch evaluations
+    Collection<EvaluationDTO> evaluations =
+        this.dataFetcher.fetchData(new InputDataSpec().withEvaluationSlices(Collections.singleton(evaluationSlice)))
+            .getEvaluations()
+            .get(evaluationSlice);
+
+    Collection<EvaluationDTO> recentEvaluations = getEvaluationsWithinDays(evaluations, evaluationTimeStamp,
+        MAPE_LOOK_BACK_DAYS_RECENT);
+    Collection<EvaluationDTO> baselineEvaluations = getEvaluationsWithinDays(evaluations, evaluationTimeStamp,
+        MAPE_LOOK_BACK_DAYS_BASELINE);
+
+    if (recentEvaluations.isEmpty() || recentEvaluations.containsAll(baselineEvaluations)) {
+      // data is insufficient for performing evaluations
+      return new ModelEvaluationResult(ModelStatus.UNKNOWN);
+    }
+
+    // calculate past 7 day mean MAPE for each metric urn and rules
+    Map<String, Double> recentMeanMapeForMetricUrnsAndRules = getMeanMapeForEachMetricUrnAndRule(recentEvaluations);
+
+    // calculate past 30 day mean MAPE for each metric urn and rules
+    Map<String, Double> baselineMeanMapeForMetricUrnsAndRules = getMeanMapeForEachMetricUrnAndRule(baselineEvaluations);
+
+    // evaluate for each metric urn
+    Map<String, Boolean> evaluationResultForMetricUrnsAndRules = recentMeanMapeForMetricUrnsAndRules.entrySet()
+        .stream()
+        .collect(Collectors.toMap(Map.Entry::getKey,
+            // compare the MAPE percentage change to threshold
+            recentMeanMape -> recentMeanMape.getValue() / baselineMeanMapeForMetricUrnsAndRules.get(recentMeanMape.getKey()) - 1 <= threshold));
+
+    if (evaluationResultForMetricUrnsAndRules.values().stream().allMatch(result -> result)) {
+      // if all metric urn's status is good, return overall good status
+      return new ModelEvaluationResult(ModelStatus.GOOD);
+    }
+    return new ModelEvaluationResult(ModelStatus.BAD);
+  }
+
+  /**
+   * Filter the evaluations to return only the past number days.
+   * @param evaluations evaluations
+   * @param evaluationTimeStamp the time stamp for evaluations
+   * @param days look back number of days
+   * @return the filtered collection of evaluationDTOs
+   */
+  private Collection<EvaluationDTO> getEvaluationsWithinDays(Collection<EvaluationDTO> evaluations,
+      Instant evaluationTimeStamp, int days) {
+    return evaluations.stream()
+        .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
+   * @param evaluations the available evaluations
+   * @return the mean MAPE keyed by metric urns
+   */
+  private Map<String, Double> getMeanMapeForEachMetricUrnAndRule(Collection<EvaluationDTO> evaluations) {
+    return
+        evaluations.stream().collect(
+            Collectors.groupingBy(e -> String.format("%s:%s", e.getMetricUrn(), e.getDetectorName()), Collectors.averagingDouble(EvaluationBean::getMape)));
+  }
+
+  @Override
+  public void init(MapeAveragePercentageChangeModelEvaluatorSpec spec, InputDataFetcher dataFetcher) {
+    this.dataFetcher = dataFetcher;
+    this.threshold = spec.getThreshold();
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/finetune/GridSearchTuningAlgorithm.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/finetune/GridSearchTuningAlgorithm.java
index 30d67ac..a574b24 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/finetune/GridSearchTuningAlgorithm.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/finetune/GridSearchTuningAlgorithm.java
@@ -24,7 +24,15 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.Iterables;
 import com.jayway.jsonpath.DocumentContext;
 import com.jayway.jsonpath.JsonPath;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
 import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.EvaluationManager;
 import org.apache.pinot.thirdeye.datalayer.bao.EventManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
@@ -36,19 +44,12 @@ 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.spi.model.AnomalySlice;
 import org.apache.pinot.thirdeye.detection.DataProvider;
 import org.apache.pinot.thirdeye.detection.DefaultDataProvider;
 import org.apache.pinot.thirdeye.detection.DetectionPipeline;
 import org.apache.pinot.thirdeye.detection.DetectionPipelineLoader;
 import org.apache.pinot.thirdeye.detection.DetectionPipelineResult;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
+import org.apache.pinot.thirdeye.detection.spi.model.AnomalySlice;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -80,6 +81,7 @@ public class GridSearchTuningAlgorithm implements TuningAlgorithm {
     MetricConfigManager metricDAO = DAORegistry.getInstance().getMetricConfigDAO();
     DatasetConfigManager datasetDAO = DAORegistry.getInstance().getDatasetConfigDAO();
     EventManager eventDAO = DAORegistry.getInstance().getEventDAO();
+    EvaluationManager evaluationDAO = DAORegistry.getInstance().getEvaluationManager();
     this.anomalyDAO = DAORegistry.getInstance().getMergedAnomalyResultDAO();
 
     TimeSeriesLoader timeseriesLoader =
@@ -89,7 +91,7 @@ public class GridSearchTuningAlgorithm implements TuningAlgorithm {
         new DefaultAggregationLoader(metricDAO, datasetDAO, ThirdEyeCacheRegistry.getInstance().getQueryCache(),
             ThirdEyeCacheRegistry.getInstance().getDatasetMaxDataTimeCache());
 
-    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, anomalyDAO, timeseriesLoader, aggregationLoader, loader);
+    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, anomalyDAO, evaluationDAO, timeseriesLoader, aggregationLoader, loader);
     this.scores = new HashMap<>();
     this.results = new LinkedHashMap<>();
     this.scoreFunction = new TimeBucketF1ScoreFunction();
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskRunner.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskRunner.java
index 5730b41..5835238 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskRunner.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskRunner.java
@@ -30,6 +30,7 @@ import org.apache.pinot.thirdeye.anomaly.task.TaskRunner;
 import org.apache.pinot.thirdeye.constant.AnomalyResultSource;
 import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.EvaluationManager;
 import org.apache.pinot.thirdeye.datalayer.bao.EventManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
@@ -62,6 +63,7 @@ public class YamlOnboardingTaskRunner implements TaskRunner {
   private static final Logger LOG = LoggerFactory.getLogger(YamlOnboardingTaskRunner.class);
   private final DetectionConfigManager detectionDAO;
   private final MergedAnomalyResultManager anomalyDAO;
+  private final EvaluationManager evaluationDAO;
   private final DetectionPipelineLoader loader;
   private final DataProvider provider;
   private final YamlDetectionTranslatorLoader translatorLoader;
@@ -72,6 +74,7 @@ public class YamlOnboardingTaskRunner implements TaskRunner {
     this.loader = new DetectionPipelineLoader();
     this.detectionDAO = DAORegistry.getInstance().getDetectionConfigManager();
     this.anomalyDAO = DAORegistry.getInstance().getMergedAnomalyResultDAO();
+    this.evaluationDAO = DAORegistry.getInstance().getEvaluationManager();
     this.translatorLoader = new YamlDetectionTranslatorLoader();
     this.yaml = new Yaml();
 
@@ -88,7 +91,7 @@ public class YamlOnboardingTaskRunner implements TaskRunner {
             ThirdEyeCacheRegistry.getInstance().getQueryCache(),
             ThirdEyeCacheRegistry.getInstance().getDatasetMaxDataTimeCache());
 
-    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, this.anomalyDAO,
+    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, this.anomalyDAO, this.evaluationDAO,
         timeseriesLoader, aggregationLoader, this.loader);
   }
 
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spec/MapeAveragePercentageChangeModelEvaluatorSpec.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spec/MapeAveragePercentageChangeModelEvaluatorSpec.java
new file mode 100644
index 0000000..f1edea1
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spec/MapeAveragePercentageChangeModelEvaluatorSpec.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;
+
+/**
+ * The spec class for MAPE change evaluator
+ */
+public class MapeAveragePercentageChangeModelEvaluatorSpec extends AbstractSpec {
+  private double threshold = 0.1; // default threshold to 10%
+
+  public double getThreshold() {
+    return threshold;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/components/ModelEvaluator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/components/ModelEvaluator.java
new file mode 100644
index 0000000..1cb20d9
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/components/ModelEvaluator.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.spi.components;
+
+import org.apache.pinot.thirdeye.detection.spec.AbstractSpec;
+import org.apache.pinot.thirdeye.detection.spi.model.ModelEvaluationResult;
+import org.joda.time.Instant;
+
+
+/**
+ * The interface for model evaluator.
+ * @param <T> the spec class for this model evaluator
+ */
+public interface ModelEvaluator<T extends AbstractSpec> extends BaseComponent<T> {
+  /**
+   * Evaluate the current detection model.
+   * @param evaluationTimeStamp the time stamp when the evaluation is run.
+   * @return a model evaluation result
+   */
+  ModelEvaluationResult evaluateModel(Instant evaluationTimeStamp);
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/EvaluationSlice.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/EvaluationSlice.java
new file mode 100644
index 0000000..318f214
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/EvaluationSlice.java
@@ -0,0 +1,68 @@
+/*
+ *
+ *  * 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.spi.model;
+
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
+
+
+/**
+ * Selector for evaluations based on (optionally) start time and end time.
+ */
+public class EvaluationSlice {
+  private final long start;
+  private final long end;
+
+  private EvaluationSlice(long start, long end) {
+    this.start = start;
+    this.end = end;
+  }
+
+  public EvaluationSlice() {
+    // -1 means match any
+    this(-1, -1);
+  }
+
+  public EvaluationSlice withStartTime(long startTime) {
+    return new EvaluationSlice(startTime, this.end);
+  }
+
+  public EvaluationSlice withEndTime(long endTime) {
+    return new EvaluationSlice(this.start, endTime);
+  }
+
+  public long getStart() {
+    return start;
+  }
+
+  public long getEnd() {
+    return end;
+  }
+
+  public boolean match(EvaluationDTO evaluationDTO) {
+    if (this.start >= 0 && evaluationDTO.getEndTime() <= this.start)
+      return false;
+    if (this.end >= 0 && evaluationDTO.getStartTime() >= this.end)
+      return false;
+    return true;
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/InputData.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/InputData.java
index 1f8e4af..1642fa7 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/InputData.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/InputData.java
@@ -19,10 +19,12 @@
 
 package org.apache.pinot.thirdeye.detection.spi.model;
 
+import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.EventDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
@@ -42,6 +44,7 @@ public class InputData {
   final Multimap<EventSlice, EventDTO> events;
   final Map<Long, MetricConfigDTO> metrics;
   final Map<String, DatasetConfigDTO> datasets;
+  final Multimap<EvaluationSlice, EvaluationDTO> evaluations;
 
   /**
    * The data set config dtos for metric ids
@@ -66,11 +69,13 @@ public class InputData {
     this.datasets = Collections.emptyMap();
     this.datasetForMetricId = Collections.emptyMap();
     this.metricForMetricAndDatasetNames = Collections.emptyMap();
+    this.evaluations = ArrayListMultimap.create();
   }
 
   public InputData(InputDataSpec spec, Map<MetricSlice, DataFrame> timeseries, Map<MetricSlice, DataFrame> aggregates,
       Multimap<AnomalySlice, MergedAnomalyResultDTO> anomalies, Multimap<EventSlice, EventDTO> events,
-      Map<Long, MetricConfigDTO> metrics, Map<String, DatasetConfigDTO> datasets, Map<Long, DatasetConfigDTO> datasetForMetricId,
+      Map<Long, MetricConfigDTO> metrics, Map<String, DatasetConfigDTO> datasets,
+      Multimap<EvaluationSlice, EvaluationDTO> evaluations, Map<Long, DatasetConfigDTO> datasetForMetricId,
       Map<InputDataSpec.MetricAndDatasetName, MetricConfigDTO> metricForMetricAndDatasetNames) {
     this.dataSpec = spec;
     this.timeseries = timeseries;
@@ -79,6 +84,7 @@ public class InputData {
     this.events = events;
     this.metrics = metrics;
     this.datasets = datasets;
+    this.evaluations = evaluations;
     this.datasetForMetricId = datasetForMetricId;
     this.metricForMetricAndDatasetNames = metricForMetricAndDatasetNames;
   }
@@ -118,4 +124,8 @@ public class InputData {
   public Map<InputDataSpec.MetricAndDatasetName, MetricConfigDTO> getMetricForMetricAndDatasetNames() {
     return metricForMetricAndDatasetNames;
   }
+
+  public Multimap<EvaluationSlice, EvaluationDTO> getEvaluations() {
+    return evaluations;
+  }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/InputDataSpec.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/InputDataSpec.java
index f77265e..c4ba20b 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/InputDataSpec.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/InputDataSpec.java
@@ -53,6 +53,12 @@ public class InputDataSpec {
   final Collection<EventSlice> eventSlices;
 
   /*
+    Specs for evaluations. Describe what evaluations to fetch.
+    Each slice defines the time range and detection config id of the evaluations to fetch.
+  */
+  final Collection<EvaluationSlice> evaluationSlices;
+
+  /*
     Metric ids to fetch the MetricConfigDTO for.
    */
   final Collection<Long> metricIds;
@@ -91,23 +97,27 @@ public class InputDataSpec {
   }
 
   public InputDataSpec() {
-    this.timeseriesSlices = Collections.emptyList();
-    this.aggregateSlices = Collections.emptyList();
-    this.anomalySlices = Collections.emptyList();
-    this.eventSlices = Collections.emptyList();
-    this.metricIds = Collections.emptyList();
-    this.datasetNames = Collections.emptyList();
-    this.metricIdsForDatasets = Collections.emptyList();
-    this.metricAndDatasetNames = Collections.emptyList();
+    this(Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), Collections.emptyList(),
+        Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), Collections.emptyList(),
+        Collections.emptyList());
   }
 
   public InputDataSpec(Collection<MetricSlice> timeseriesSlices, Collection<MetricSlice> aggregateSlices,
       Collection<AnomalySlice> anomalySlices, Collection<EventSlice> eventSlices, Collection<Long> metricIds, Collection<String> datasetNames,
       Collection<Long> metricIdsForDatasets, Collection<MetricAndDatasetName> metricAndDatasetNames) {
+    this(timeseriesSlices, aggregateSlices, anomalySlices, eventSlices, Collections.emptyList(), metricIds,
+        datasetNames, metricIdsForDatasets, metricAndDatasetNames);
+  }
+
+  public InputDataSpec(Collection<MetricSlice> timeseriesSlices, Collection<MetricSlice> aggregateSlices,
+      Collection<AnomalySlice> anomalySlices, Collection<EventSlice> eventSlices,
+      Collection<EvaluationSlice> evaluationSlices, Collection<Long> metricIds, Collection<String> datasetNames,
+      Collection<Long> metricIdsForDatasets, Collection<MetricAndDatasetName> metricAndDatasetNames) {
     this.timeseriesSlices = timeseriesSlices;
     this.aggregateSlices = aggregateSlices;
     this.anomalySlices = anomalySlices;
     this.eventSlices = eventSlices;
+    this.evaluationSlices = evaluationSlices;
     this.metricIds = metricIds;
     this.datasetNames = datasetNames;
     this.metricIdsForDatasets = metricIdsForDatasets;
@@ -138,6 +148,10 @@ public class InputDataSpec {
     return datasetNames;
   }
 
+  public Collection<EvaluationSlice> getEvaluationSlices() {
+    return evaluationSlices;
+  }
+
   public Collection<Long> getMetricIdsForDatasets() {
     return metricIdsForDatasets;
   }
@@ -162,6 +176,10 @@ public class InputDataSpec {
     return new InputDataSpec(this.timeseriesSlices, this.aggregateSlices, this.anomalySlices, eventSlices, this.metricIds, this.datasetNames, this.metricIdsForDatasets, this.metricAndDatasetNames);
   }
 
+  public InputDataSpec withEvaluationSlices(Collection<EvaluationSlice> evaluationSlices) {
+    return new InputDataSpec(this.timeseriesSlices, this.aggregateSlices, this.anomalySlices, this.eventSlices, evaluationSlices, this.metricIds, this.datasetNames, this.metricIdsForDatasets, this.metricAndDatasetNames);
+  }
+
   public InputDataSpec withMetricIds(Collection<Long> metricIds) {
     return new InputDataSpec(this.timeseriesSlices, this.aggregateSlices, this.anomalySlices, eventSlices, metricIds, this.datasetNames, this.metricIdsForDatasets, this.metricAndDatasetNames);
   }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/ModelEvaluationResult.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/ModelEvaluationResult.java
new file mode 100644
index 0000000..851b2ac
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/ModelEvaluationResult.java
@@ -0,0 +1,39 @@
+/*
+ *
+ *  * 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.spi.model;
+
+/**
+ * The model valuation result
+ */
+public class ModelEvaluationResult {
+  // the overall model status
+  private final ModelStatus status;
+
+  public ModelStatus getStatus() {
+    return status;
+  }
+
+  public ModelEvaluationResult(ModelStatus status) {
+    this.status = status;
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/ModelStatus.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/ModelStatus.java
new file mode 100644
index 0000000..b823e0a
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/ModelStatus.java
@@ -0,0 +1,30 @@
+/*
+ *
+ *  * 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.spi.model;
+
+/**
+ * The detection model status.
+ */
+public enum ModelStatus {
+  GOOD, BAD, UNKNOWN
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidator.java
index fef9934..94ba5ff 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidator.java
@@ -57,6 +57,7 @@ public class DetectionConfigValidator implements ConfigValidator<DetectionConfig
     this.provider = new DefaultDataProvider(metricDAO, datasetDAO,
         DAORegistry.getInstance().getEventDAO(),
         DAORegistry.getInstance().getMergedAnomalyResultDAO(),
+        DAORegistry.getInstance().getEvaluationManager(),
         timeseriesLoader, aggregationLoader, loader);
   }
 
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlResource.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlResource.java
index 64f6577..db765fd 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlResource.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlResource.java
@@ -54,6 +54,7 @@ import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionAlertConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.EvaluationManager;
 import org.apache.pinot.thirdeye.datalayer.bao.EventManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
@@ -110,6 +111,7 @@ public class YamlResource {
   private final DatasetConfigManager datasetDAO;
   private final EventManager eventDAO;
   private final MergedAnomalyResultManager anomalyDAO;
+  private final EvaluationManager evaluationDAO;
   private final TaskManager taskDAO;
   private final DetectionPipelineLoader loader;
   private final Yaml yaml;
@@ -126,6 +128,7 @@ public class YamlResource {
     this.eventDAO = DAORegistry.getInstance().getEventDAO();
     this.anomalyDAO = DAORegistry.getInstance().getMergedAnomalyResultDAO();
     this.taskDAO = DAORegistry.getInstance().getTaskDAO();
+    this.evaluationDAO = DAORegistry.getInstance().getEvaluationManager();
     this.yaml = new Yaml();
 
     TimeSeriesLoader timeseriesLoader =
@@ -137,7 +140,7 @@ public class YamlResource {
 
     this.loader = new DetectionPipelineLoader();
 
-    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, anomalyDAO, timeseriesLoader, aggregationLoader, loader);
+    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, anomalyDAO, evaluationDAO, timeseriesLoader, aggregationLoader, loader);
   }
 
   public DetectionConfigDTO translateToDetectionConfig(Map<String, Object> yamlConfig) throws Exception {
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DataProviderTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DataProviderTest.java
index e12c4ba..d3a65d7 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DataProviderTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DataProviderTest.java
@@ -33,6 +33,7 @@ import org.apache.pinot.thirdeye.common.dimension.DimensionMap;
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
 import org.apache.pinot.thirdeye.datalayer.bao.DAOTestBase;
 import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.EvaluationManager;
 import org.apache.pinot.thirdeye.datalayer.bao.EventManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
@@ -65,6 +66,7 @@ public class DataProviderTest {
   private MergedAnomalyResultManager anomalyDAO;
   private MetricConfigManager metricDAO;
   private DatasetConfigManager datasetDAO;
+  private EvaluationManager evaluationDAO;
   private QueryCache cache;
   private TimeSeriesLoader timeseriesLoader;
 
@@ -86,7 +88,7 @@ public class DataProviderTest {
     this.anomalyDAO = reg.getMergedAnomalyResultDAO();
     this.metricDAO = reg.getMetricConfigDAO();
     this.datasetDAO = reg.getDatasetConfigDAO();
-
+    this.evaluationDAO = reg.getEvaluationManager();
     // events
     this.eventIds = new ArrayList<>();
     this.eventIds.add(this.eventDAO.save(makeEvent(3600000L, 7200000L)));
@@ -140,7 +142,7 @@ public class DataProviderTest {
     this.timeseriesLoader = new DefaultTimeSeriesLoader(this.metricDAO, this.datasetDAO, this.cache);
 
     // provider
-    this.provider = new DefaultDataProvider(this.metricDAO, this.datasetDAO, this.eventDAO, this.anomalyDAO,
+    this.provider = new DefaultDataProvider(this.metricDAO, this.datasetDAO, this.eventDAO, this.anomalyDAO, this.evaluationDAO,
         this.timeseriesLoader, null, null);
   }
 
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/MockDataProvider.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/MockDataProvider.java
index 27483b3..3af49e8 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/MockDataProvider.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/MockDataProvider.java
@@ -28,10 +28,12 @@ import org.apache.pinot.thirdeye.dataframe.Series;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.EventDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
 import org.apache.pinot.thirdeye.detection.spi.model.AnomalySlice;
+import org.apache.pinot.thirdeye.detection.spi.model.EvaluationSlice;
 import org.apache.pinot.thirdeye.detection.spi.model.EventSlice;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -54,6 +56,7 @@ public class MockDataProvider implements DataProvider {
   private List<MergedAnomalyResultDTO> anomalies;
   private List<MetricConfigDTO> metrics;
   private List<DatasetConfigDTO> datasets;
+  private List<EvaluationDTO>  evaluations;
   private DetectionPipelineLoader loader;
 
   public MockDataProvider() {
@@ -168,6 +171,20 @@ public class MockDataProvider implements DataProvider {
   }
 
   @Override
+  public Multimap<EvaluationSlice, EvaluationDTO> fetchEvaluations(Collection<EvaluationSlice> slices,
+      long configId) {
+    Multimap<EvaluationSlice, EvaluationDTO> result = ArrayListMultimap.create();
+    for (EvaluationSlice slice : slices) {
+      for (EvaluationDTO evaluation  :this.evaluations) {
+        if (slice.match(evaluation) && evaluation.getDetectionConfigId() == configId) {
+          result.put(slice, evaluation);
+        }
+      }
+    }
+    return result;
+  }
+
+  @Override
   public Map<Long, MetricConfigDTO> fetchMetrics(Collection<Long> ids) {
     Map<Long, MetricConfigDTO> result = new HashMap<>();
     for (Long id : ids) {
@@ -271,6 +288,14 @@ public class MockDataProvider implements DataProvider {
     return this;
   }
 
+  public List<EvaluationDTO> getEvaluations() {
+    return evaluations;
+  }
+
+  public void setEvaluations(List<EvaluationDTO> evaluations) {
+    this.evaluations = evaluations;
+  }
+
   @Override
   public boolean equals(Object o) {
     if (this == o) {
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
new file mode 100644
index 0000000..e7e0a55
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MapePercentageChangeModelEvaluatorTest.java
@@ -0,0 +1,82 @@
+/*
+ *
+ *  * 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.Arrays;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
+import org.apache.pinot.thirdeye.detection.DefaultInputDataFetcher;
+import org.apache.pinot.thirdeye.detection.InputDataFetcher;
+import org.apache.pinot.thirdeye.detection.MockDataProvider;
+import org.apache.pinot.thirdeye.detection.spec.MapeAveragePercentageChangeModelEvaluatorSpec;
+import org.apache.pinot.thirdeye.detection.spi.model.ModelEvaluationResult;
+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 MapePercentageChangeModelEvaluatorTest {
+  private InputDataFetcher dataFetcher;
+
+  @BeforeMethod
+  public void setUp() {
+    MockDataProvider dataProvider = new MockDataProvider();
+    long mockDetectionConfigId = 100L;
+    String mockMetricUrn = "thirdeye:metric:1";
+    EvaluationDTO eval1 = makeMockEvaluationDTO(mockDetectionConfigId, mockMetricUrn, 1557187200000L,1557273600000L, 0.06);
+    EvaluationDTO eval2 = makeMockEvaluationDTO(mockDetectionConfigId, mockMetricUrn, 1555368321000L,1555454721000L, 0.055);
+    dataProvider.setEvaluations(Arrays.asList(eval1, eval2));
+    dataFetcher = new DefaultInputDataFetcher(dataProvider, mockDetectionConfigId);
+  }
+
+  private EvaluationDTO makeMockEvaluationDTO(long mockDetectionConfigId, String mockMetricUrn, long start, long end, double mape) {
+    EvaluationDTO eval = new EvaluationDTO();
+    eval.setStartTime(start);
+    eval.setEndTime(end);
+    eval.setMetricUrn(mockMetricUrn);
+    eval.setMape(mape);
+    eval.setDetectionConfigId(mockDetectionConfigId);
+    return eval;
+  }
+
+  @Test
+  public void testEvaluateModelGood() {
+    MapeAveragePercentageChangeModelEvaluatorSpec spec = new MapeAveragePercentageChangeModelEvaluatorSpec();
+    spec.setThreshold(0.1);
+    MapeAveragePercentageChangeModelEvaluator evaluator = new MapeAveragePercentageChangeModelEvaluator();
+    evaluator.init(spec, dataFetcher);
+    ModelEvaluationResult result = evaluator.evaluateModel(Instant.parse("2019-05-08T20:00:00.000Z"));
+    Assert.assertEquals(result.getStatus(), ModelStatus.GOOD);
+  }
+
+  @Test
+  public void testEvaluateModelBad() {
+    MapeAveragePercentageChangeModelEvaluatorSpec spec = new MapeAveragePercentageChangeModelEvaluatorSpec();
+    spec.setThreshold(0.01);
+    MapeAveragePercentageChangeModelEvaluator evaluator = new MapeAveragePercentageChangeModelEvaluator();
+    evaluator.init(spec, dataFetcher);
+    ModelEvaluationResult result = evaluator.evaluateModel(Instant.parse("2019-05-08T20:00:00.000Z"));
+    Assert.assertEquals(result.getStatus(), ModelStatus.BAD);
+  }
+}


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