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/14 21:46:40 UTC

[incubator-pinot] branch master updated: [TE] Evaluation metrics calculation & store (#4202)

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 e279dc9  [TE] Evaluation metrics calculation & store (#4202)
e279dc9 is described below

commit e279dc940066bdae9728b1fa7800910ce43271a4
Author: Jihao Zhang <ji...@linkedin.com>
AuthorDate: Tue May 14 14:46:34 2019 -0700

    [TE] Evaluation metrics calculation & store (#4202)
    
    The performance metrics need to be stored as a by-product of each detection. Later this will be read by the model evaluators to monitor the detection models. This PR creates the table and adds the necessary boilerplate codes for storing the Evaluation Metrics. The PR also adds the MAPE calculation and collect the metric.
---
 .../pinot/thirdeye/dataframe/DoubleSeries.java     |   4 +
 .../thirdeye/datalayer/bao/EvaluationManager.java  |  33 ++++++
 .../datalayer/bao/jdbc/EvaluationManagerImpl.java  |  36 +++++++
 .../thirdeye/datalayer/dao/GenericPojoDao.java     |   4 +
 .../thirdeye/datalayer/dto/EvaluationDTO.java      |  55 ++++++++++
 .../thirdeye/datalayer/entity/EvaluationIndex.java |  71 +++++++++++++
 .../thirdeye/datalayer/pojo/EvaluationBean.java    | 113 +++++++++++++++++++++
 .../thirdeye/datalayer/util/DaoProviderUtil.java   |   4 +-
 .../pinot/thirdeye/datasource/DAORegistry.java     |   7 ++
 .../detection/DetectionPipelineResult.java         |  15 ++-
 .../detection/DetectionPipelineTaskRunner.java     |  15 ++-
 .../pinot/thirdeye/detection/Evaluation.java       |  49 +++++++++
 .../detection/algorithm/DimensionWrapper.java      |  11 +-
 .../thirdeye/detection/algorithm/MergeWrapper.java |  10 +-
 .../src/main/resources/schema/create-schema.sql    |  18 +++-
 .../detection/DetectionPipelineTaskRunnerTest.java |   4 +
 .../pinot/thirdeye/detection/EvaluationTest.java   |  45 ++++++++
 17 files changed, 485 insertions(+), 9 deletions(-)

diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dataframe/DoubleSeries.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dataframe/DoubleSeries.java
index 33e574d..7a6cd2b 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dataframe/DoubleSeries.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dataframe/DoubleSeries.java
@@ -753,6 +753,10 @@ public final class DoubleSeries extends TypedSeries<DoubleSeries> {
     }, this, other);
   }
 
+  public BooleanSeries ne(final double constant) {
+    return this.eq(constant).not();
+  }
+
   public BooleanSeries gt(final double constant) {
     if(isNull(constant))
       return BooleanSeries.nulls(this.size());
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/EvaluationManager.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/EvaluationManager.java
new file mode 100644
index 0000000..282d4ac
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/EvaluationManager.java
@@ -0,0 +1,33 @@
+/*
+ *
+ *  * 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.datalayer.bao;
+
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
+
+
+/**
+ * The DAO for evaluations
+ */
+public interface EvaluationManager extends AbstractManager<EvaluationDTO> {
+
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/jdbc/EvaluationManagerImpl.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/jdbc/EvaluationManagerImpl.java
new file mode 100644
index 0000000..75dc951
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/jdbc/EvaluationManagerImpl.java
@@ -0,0 +1,36 @@
+/*
+ *
+ *  * 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.datalayer.bao.jdbc;
+
+import com.google.inject.Singleton;
+import org.apache.pinot.thirdeye.datalayer.bao.EvaluationManager;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
+import org.apache.pinot.thirdeye.datalayer.pojo.EvaluationBean;
+
+
+@Singleton
+public class EvaluationManagerImpl extends AbstractManagerImpl<EvaluationDTO> implements EvaluationManager {
+  public EvaluationManagerImpl() {
+    super(EvaluationDTO.class, EvaluationBean.class);
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dao/GenericPojoDao.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dao/GenericPojoDao.java
index 261a03d..a445665 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dao/GenericPojoDao.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dao/GenericPojoDao.java
@@ -43,6 +43,7 @@ import org.apache.pinot.thirdeye.datalayer.entity.DetectionAlertConfigIndex;
 import org.apache.pinot.thirdeye.datalayer.entity.DetectionConfigIndex;
 import org.apache.pinot.thirdeye.datalayer.entity.DetectionStatusIndex;
 import org.apache.pinot.thirdeye.datalayer.entity.EntityToEntityMappingIndex;
+import org.apache.pinot.thirdeye.datalayer.entity.EvaluationIndex;
 import org.apache.pinot.thirdeye.datalayer.entity.EventIndex;
 import org.apache.pinot.thirdeye.datalayer.entity.GenericJsonEntity;
 import org.apache.pinot.thirdeye.datalayer.entity.GroupedAnomalyResultsIndex;
@@ -70,6 +71,7 @@ import org.apache.pinot.thirdeye.datalayer.pojo.DetectionAlertConfigBean;
 import org.apache.pinot.thirdeye.datalayer.pojo.DetectionConfigBean;
 import org.apache.pinot.thirdeye.datalayer.pojo.DetectionStatusBean;
 import org.apache.pinot.thirdeye.datalayer.pojo.EntityToEntityMappingBean;
+import org.apache.pinot.thirdeye.datalayer.pojo.EvaluationBean;
 import org.apache.pinot.thirdeye.datalayer.pojo.EventBean;
 import org.apache.pinot.thirdeye.datalayer.pojo.GroupedAnomalyResultsBean;
 import org.apache.pinot.thirdeye.datalayer.pojo.JobBean;
@@ -163,6 +165,8 @@ public class GenericPojoDao {
         newPojoInfo(DEFAULT_BASE_TABLE_NAME, DetectionConfigIndex.class));
     pojoInfoMap.put(DetectionAlertConfigBean.class,
         newPojoInfo(DEFAULT_BASE_TABLE_NAME, DetectionAlertConfigIndex.class));
+    pojoInfoMap.put(EvaluationBean.class,
+        newPojoInfo(DEFAULT_BASE_TABLE_NAME, EvaluationIndex.class));
   }
 
   private static PojoInfo newPojoInfo(String baseTableName,
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
new file mode 100644
index 0000000..19d0cac
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dto/EvaluationDTO.java
@@ -0,0 +1,55 @@
+/*
+ *
+ *  * 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.datalayer.dto;
+
+import org.apache.pinot.thirdeye.dataframe.DataFrame;
+import org.apache.pinot.thirdeye.datalayer.pojo.EvaluationBean;
+import org.apache.pinot.thirdeye.detection.Evaluation;
+import org.apache.pinot.thirdeye.detection.PredictionResult;
+
+import static org.apache.pinot.thirdeye.dataframe.util.DataFrameUtils.*;
+
+
+/**
+ * The evaluation DTO
+ */
+public class EvaluationDTO extends EvaluationBean {
+  public static EvaluationDTO fromPredictionResult(PredictionResult predictionResult, long startTime, long endTime,
+      long detectionConfigId) {
+    EvaluationDTO evaluation = new EvaluationDTO();
+    evaluation.setDetectionConfigId(detectionConfigId);
+    evaluation.setStartTime(startTime);
+    evaluation.setEndTime(endTime);
+    evaluation.setDetectorName(predictionResult.getDetectorName());
+    evaluation.setMetricUrn(predictionResult.getMetricUrn());
+    evaluation.setMape(getMape(predictionResult));
+    return evaluation;
+  }
+
+  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));
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/entity/EvaluationIndex.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/entity/EvaluationIndex.java
new file mode 100644
index 0000000..207caf7
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/entity/EvaluationIndex.java
@@ -0,0 +1,71 @@
+/*
+ *
+ *  * 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.datalayer.entity;
+
+public class EvaluationIndex extends AbstractIndexEntity {
+  private long detectionConfigId;
+  private long startTime;
+  private long endTime;
+  private String detectorName;
+  private double mape;
+
+  public long getDetectionConfigId() {
+    return detectionConfigId;
+  }
+
+  public void setDetectionConfigId(long detectionConfigId) {
+    this.detectionConfigId = detectionConfigId;
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public void setStartTime(long startTime) {
+    this.startTime = startTime;
+  }
+
+  public long getEndTime() {
+    return endTime;
+  }
+
+  public void setEndTime(long endTime) {
+    this.endTime = endTime;
+  }
+
+  public String getDetectorName() {
+    return detectorName;
+  }
+
+  public void setDetectorName(String detectorName) {
+    this.detectorName = detectorName;
+  }
+
+  public double getMape() {
+    return mape;
+  }
+
+  public void setMape(double mape) {
+    this.mape = mape;
+  }
+}
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
new file mode 100644
index 0000000..e141b23
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/pojo/EvaluationBean.java
@@ -0,0 +1,113 @@
+/*
+ *
+ *  * 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.datalayer.pojo;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import java.util.Objects;
+
+/**
+ * The class for evaluation metrics.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class EvaluationBean extends AbstractBean {
+  private long detectionConfigId; // the detection config id
+  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 String metricUrn; // the metric urn
+
+  public long getDetectionConfigId() {
+    return detectionConfigId;
+  }
+
+  public void setDetectionConfigId(long detectionConfigId) {
+    this.detectionConfigId = detectionConfigId;
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public void setStartTime(long startTime) {
+    this.startTime = startTime;
+  }
+
+  public long getEndTime() {
+    return endTime;
+  }
+
+  public void setEndTime(long endTime) {
+    this.endTime = endTime;
+  }
+
+  public String getDetectorName() {
+    return detectorName;
+  }
+
+  public void setDetectorName(String detectorName) {
+    this.detectorName = detectorName;
+  }
+
+  public double getMape() {
+    return mape;
+  }
+
+  public void setMape(double mape) {
+    this.mape = mape;
+  }
+
+  public String getMetricUrn() {
+    return metricUrn;
+  }
+
+  public void setMetricUrn(String metricUrn) {
+    this.metricUrn = metricUrn;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof EvaluationBean)) {
+      return false;
+    }
+    EvaluationBean that = (EvaluationBean) o;
+    return detectionConfigId == that.detectionConfigId && startTime == that.startTime && endTime == that.endTime
+        && Double.compare(that.mape, mape) == 0 && Objects.equals(detectorName, that.detectorName) && Objects.equals(
+        metricUrn, that.metricUrn);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(detectionConfigId, startTime, endTime, detectorName, mape, metricUrn);
+  }
+
+  @Override
+  public String toString() {
+    return "EvaluationBean{" + "detectionConfigId=" + detectionConfigId + ", startTime=" + startTime + ", endTime="
+        + endTime + ", detectorName='" + detectorName + '\'' + ", mape=" + mape + ", metricUrn='" + metricUrn + '\''
+        + '}';
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/util/DaoProviderUtil.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/util/DaoProviderUtil.java
index 31b974d..aaa3253 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/util/DaoProviderUtil.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/util/DaoProviderUtil.java
@@ -39,6 +39,7 @@ import org.apache.pinot.thirdeye.datalayer.entity.DetectionAlertConfigIndex;
 import org.apache.pinot.thirdeye.datalayer.entity.DetectionConfigIndex;
 import org.apache.pinot.thirdeye.datalayer.entity.DetectionStatusIndex;
 import org.apache.pinot.thirdeye.datalayer.entity.EntityToEntityMappingIndex;
+import org.apache.pinot.thirdeye.datalayer.entity.EvaluationIndex;
 import org.apache.pinot.thirdeye.datalayer.entity.EventIndex;
 import org.apache.pinot.thirdeye.datalayer.entity.GenericJsonEntity;
 import org.apache.pinot.thirdeye.datalayer.entity.GroupedAnomalyResultsIndex;
@@ -203,7 +204,8 @@ public abstract class DaoProviderUtil {
             convertCamelCaseToUnderscore(DetectionConfigIndex.class.getSimpleName()));
         entityMappingHolder.register(conn, DetectionAlertConfigIndex.class,
             convertCamelCaseToUnderscore(DetectionAlertConfigIndex.class.getSimpleName()));
-
+        entityMappingHolder.register(conn, EvaluationIndex.class,
+            convertCamelCaseToUnderscore(EvaluationIndex.class.getSimpleName()));
       } catch (Exception e) {
         throw new RuntimeException(e);
       }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datasource/DAORegistry.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datasource/DAORegistry.java
index 2b35a1c..6a28f5f 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datasource/DAORegistry.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datasource/DAORegistry.java
@@ -32,6 +32,7 @@ import org.apache.pinot.thirdeye.datalayer.bao.DetectionAlertConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionStatusManager;
 import org.apache.pinot.thirdeye.datalayer.bao.EntityToEntityMappingManager;
+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.JobManager;
@@ -56,6 +57,7 @@ import org.apache.pinot.thirdeye.datalayer.bao.jdbc.DetectionAlertConfigManagerI
 import org.apache.pinot.thirdeye.datalayer.bao.jdbc.DetectionConfigManagerImpl;
 import org.apache.pinot.thirdeye.datalayer.bao.jdbc.DetectionStatusManagerImpl;
 import org.apache.pinot.thirdeye.datalayer.bao.jdbc.EntityToEntityMappingManagerImpl;
+import org.apache.pinot.thirdeye.datalayer.bao.jdbc.EvaluationManagerImpl;
 import org.apache.pinot.thirdeye.datalayer.bao.jdbc.EventManagerImpl;
 import org.apache.pinot.thirdeye.datalayer.bao.jdbc.GroupedAnomalyResultsManagerImpl;
 import org.apache.pinot.thirdeye.datalayer.bao.jdbc.JobManagerImpl;
@@ -189,4 +191,9 @@ public class DAORegistry {
   public DetectionAlertConfigManager getDetectionAlertConfigManager() {
     return DaoProviderUtil.getInstance(DetectionAlertConfigManagerImpl.class);
   }
+
+  public EvaluationManager getEvaluationManager() {
+    return DaoProviderUtil.getInstance(EvaluationManagerImpl.class);
+  }
+
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionPipelineResult.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionPipelineResult.java
index 7a5c934..713227a 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionPipelineResult.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionPipelineResult.java
@@ -20,6 +20,7 @@
 package org.apache.pinot.thirdeye.detection;
 
 import java.util.Collections;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import java.util.HashMap;
 import java.util.List;
@@ -37,9 +38,11 @@ public class DetectionPipelineResult {
   private final long lastTimestamp;
   // predicted baselines result
   private final List<PredictionResult> predictions;
+  // evaluation metrics
+  private final List<EvaluationDTO> evaluations;
 
   public DetectionPipelineResult(List<MergedAnomalyResultDTO> anomalies) {
-    this(anomalies, getMaxTime(anomalies), Collections.emptyList());
+    this(anomalies, getMaxTime(anomalies));
     this.diagnostics = new HashMap<>();
   }
 
@@ -50,12 +53,19 @@ public class DetectionPipelineResult {
 
   public DetectionPipelineResult(List<MergedAnomalyResultDTO> anomalies, long lastTimestamp,
       List<PredictionResult> predictedTimeSeries) {
+    this(anomalies, lastTimestamp, predictedTimeSeries, Collections.emptyList());
+  }
+
+  public DetectionPipelineResult(List<MergedAnomalyResultDTO> anomalies, long lastTimestamp,
+      List<PredictionResult> predictedTimeSeries, List<EvaluationDTO> evaluations) {
     this.anomalies = anomalies;
     this.lastTimestamp = lastTimestamp;
     this.predictions = predictedTimeSeries;
+    this.evaluations = evaluations;
     this.diagnostics = new HashMap<>();
   }
 
+
   public List<PredictionResult> getPredictions() {
     return predictions;
   }
@@ -69,6 +79,9 @@ public class DetectionPipelineResult {
     return lastTimestamp;
   }
 
+  public List<EvaluationDTO> getEvaluations() {
+    return evaluations;
+  }
 
   public Map<String, Object> getDiagnostics() {
     return diagnostics;
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 559358e..42dec01 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
@@ -26,10 +26,12 @@ import org.apache.pinot.thirdeye.anomaly.task.TaskRunner;
 import org.apache.pinot.thirdeye.anomaly.utils.ThirdeyeMetricsUtil;
 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;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import org.apache.pinot.thirdeye.datasource.DAORegistry;
 import org.apache.pinot.thirdeye.datasource.ThirdEyeCacheRegistry;
@@ -47,6 +49,7 @@ public class DetectionPipelineTaskRunner implements TaskRunner {
   private static final Logger LOG = LoggerFactory.getLogger(DetectionPipelineTaskRunner.class);
   private final DetectionConfigManager detectionDAO;
   private final MergedAnomalyResultManager anomalyDAO;
+  private final EvaluationManager evaluationDAO;
   private final DetectionPipelineLoader loader;
   private final DataProvider provider;
 
@@ -61,7 +64,7 @@ public class DetectionPipelineTaskRunner implements TaskRunner {
     this.loader = new DetectionPipelineLoader();
     this.detectionDAO = DAORegistry.getInstance().getDetectionConfigManager();
     this.anomalyDAO = DAORegistry.getInstance().getMergedAnomalyResultDAO();
-
+    this.evaluationDAO = DAORegistry.getInstance().getEvaluationManager();
     MetricConfigManager metricDAO = DAORegistry.getInstance().getMetricConfigDAO();
     DatasetConfigManager datasetDAO = DAORegistry.getInstance().getDatasetConfigDAO();
     EventManager eventDAO = DAORegistry.getInstance().getEventDAO();
@@ -84,13 +87,15 @@ public class DetectionPipelineTaskRunner implements TaskRunner {
    *
    * @param detectionDAO detection config DAO
    * @param anomalyDAO merged anomaly DAO
+   * @param evaluationDAO the evaluation DAO
    * @param loader pipeline loader
    * @param provider pipeline data provider
    */
-  public DetectionPipelineTaskRunner(DetectionConfigManager detectionDAO, MergedAnomalyResultManager anomalyDAO,
-      DetectionPipelineLoader loader, DataProvider provider) {
+  DetectionPipelineTaskRunner(DetectionConfigManager detectionDAO, MergedAnomalyResultManager anomalyDAO,
+      EvaluationManager evaluationDAO, DetectionPipelineLoader loader, DataProvider provider) {
     this.detectionDAO = detectionDAO;
     this.anomalyDAO = anomalyDAO;
+    this.evaluationDAO = evaluationDAO;
     this.loader = loader;
     this.provider = provider;
   }
@@ -124,6 +129,10 @@ public class DetectionPipelineTaskRunner implements TaskRunner {
         }
       }
 
+      for (EvaluationDTO evaluationDTO : result.getEvaluations()) {
+        this.evaluationDAO.save(evaluationDTO);
+      }
+
       return Collections.emptyList();
 
     } catch(Exception e) {
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/Evaluation.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/Evaluation.java
new file mode 100644
index 0000000..9109fe5
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/Evaluation.java
@@ -0,0 +1,49 @@
+/*
+ *
+ *  * 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.dataframe.DoubleSeries;
+
+
+/**
+ * The util class for model evaluation
+ */
+public class Evaluation {
+  // Suppresses default constructor, ensuring non-instantiability.
+  private Evaluation() {
+  }
+
+  /**
+   * Calculate the mean absolute percentage error (MAPE).
+   * See https://en.wikipedia.org/wiki/Mean_absolute_percentage_error
+   * @param current current time series
+   * @param predicted baseline time series
+   * @return the mape value
+   */
+  public static double calculateMape(DoubleSeries current, DoubleSeries predicted) {
+    if (current.contains(0.0)) {
+      return Double.POSITIVE_INFINITY;
+    }
+    return predicted.divide(current).subtract(1).abs().mean().value();
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/algorithm/DimensionWrapper.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/algorithm/DimensionWrapper.java
index b5f929b..769c978 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/algorithm/DimensionWrapper.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/algorithm/DimensionWrapper.java
@@ -27,6 +27,7 @@ import java.util.Set;
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import org.apache.pinot.thirdeye.detection.ConfigUtils;
 import org.apache.pinot.thirdeye.detection.DataProvider;
@@ -252,10 +253,18 @@ public class DimensionWrapper extends DetectionPipeline {
     }
 
     checkNestedMetricsStatus(totalNestedMetrics, successNestedMetrics, lastException);
-    return new DetectionPipelineResult(anomalies, DetectionUtils.consolidateNestedLastTimeStamps(lastTimeStamps), predictionResults)
+    return new DetectionPipelineResult(anomalies, DetectionUtils.consolidateNestedLastTimeStamps(lastTimeStamps), predictionResults
+    , calculateEvaluationMetrics(predictionResults))
         .setDiagnostics(diagnostics);
   }
 
+  private List<EvaluationDTO> calculateEvaluationMetrics(List<PredictionResult> predictionResults) {
+    return predictionResults.stream()
+        .map(prediction -> EvaluationDTO.fromPredictionResult(prediction, this.startTime, this.endTime,
+            this.config.getId()))
+        .collect(Collectors.toList());
+  }
+
   private void checkEarlyStop(long totalNestedMetrics, long successNestedMetrics, int i, Exception lastException) throws DetectionPipelineException {
     // if the first certain number of dimensions all failed, throw an exception
     if (i == EARLY_STOP_THRESHOLD && successNestedMetrics == 0) {
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/algorithm/MergeWrapper.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/algorithm/MergeWrapper.java
index 1535ae7..fd5a8cb 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/algorithm/MergeWrapper.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/algorithm/MergeWrapper.java
@@ -35,6 +35,7 @@ import java.util.stream.Collectors;
 import org.apache.commons.collections.MapUtils;
 import org.apache.pinot.thirdeye.common.dimension.DimensionMap;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import org.apache.pinot.thirdeye.detection.ConfigUtils;
 import org.apache.pinot.thirdeye.detection.DataProvider;
@@ -106,9 +107,12 @@ public class MergeWrapper extends DetectionPipeline {
   public DetectionPipelineResult run() throws Exception {
     Map<String, Object> diagnostics = new HashMap<>();
 
-    // generate anomalies
+    // generated anomalies
     List<MergedAnomalyResultDTO> generated = new ArrayList<>();
+    // predicted time series results
     List<PredictionResult> predictionResults = new ArrayList<>();
+    // evaluation for the predictions
+    List<EvaluationDTO> evaluations = new ArrayList<>();
     int i = 0;
     Set<Long> lastTimeStamps = new HashSet<>();
     for (Map<String, Object> properties : this.nestedProperties) {
@@ -128,6 +132,7 @@ public class MergeWrapper extends DetectionPipeline {
 
       generated.addAll(intermediate.getAnomalies());
       predictionResults.addAll(intermediate.getPredictions());
+      evaluations.addAll(intermediate.getEvaluations());
       diagnostics.put(String.valueOf(i), intermediate.getDiagnostics());
 
       i++;
@@ -138,7 +143,8 @@ public class MergeWrapper extends DetectionPipeline {
     all.addAll(retrieveAnomaliesFromDatabase(generated));
     all.addAll(generated);
 
-    return new DetectionPipelineResult(this.merge(all), DetectionUtils.consolidateNestedLastTimeStamps(lastTimeStamps), predictionResults).setDiagnostics(diagnostics);
+    return new DetectionPipelineResult(this.merge(all), DetectionUtils.consolidateNestedLastTimeStamps(lastTimeStamps),
+        predictionResults, evaluations).setDiagnostics(diagnostics);
   }
 
   protected List<MergedAnomalyResultDTO> retrieveAnomaliesFromDatabase(List<MergedAnomalyResultDTO> generated) {
diff --git a/thirdeye/thirdeye-pinot/src/main/resources/schema/create-schema.sql b/thirdeye/thirdeye-pinot/src/main/resources/schema/create-schema.sql
index 7952f74..260262f 100644
--- a/thirdeye/thirdeye-pinot/src/main/resources/schema/create-schema.sql
+++ b/thirdeye/thirdeye-pinot/src/main/resources/schema/create-schema.sql
@@ -389,4 +389,20 @@ create table if not exists detection_alert_config_index (
 ALTER TABLE `detection_alert_config_index` ADD UNIQUE `detection_alert_config_unique_index`(`name`);
 create index detection_alert_config_base_id_idx ON detection_alert_config_index(base_id);
 create index detection_alert_config_name_idx ON detection_alert_config_index(`name`);
-create index detection_alert_config_application_idx ON detection_alert_config_index(`application`);
\ No newline at end of file
+create index detection_alert_config_application_idx ON detection_alert_config_index(`application`);
+
+create table if not exists evaluation_index (
+    base_id bigint(20) not null,
+    detection_config_id bigint(20) not null,
+    start_time bigint(20) not null,
+    end_time bigint(20) not null,
+    detectorName VARCHAR(128),
+    mape double,
+    create_time timestamp,
+    update_time timestamp default current_timestamp,
+    version int(10)
+) ENGINE=InnoDB;
+ALTER TABLE `evaluation_index` ADD UNIQUE `evaluation_index`(`detection_config_id`, `start_time`, `end_time`);
+create index evaluation_base_id_idx ON evaluation_index(base_id);
+create index evaluation_detection_config_id_idx ON evaluation_index(detection_config_id);
+create index evaluation_detection_start_time_idx on evaluation_index(start_time);
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DetectionPipelineTaskRunnerTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DetectionPipelineTaskRunnerTest.java
index ba43539..a1db5b9 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DetectionPipelineTaskRunnerTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DetectionPipelineTaskRunnerTest.java
@@ -23,6 +23,7 @@ import org.apache.pinot.thirdeye.anomaly.task.TaskContext;
 import org.apache.pinot.thirdeye.common.dimension.DimensionMap;
 import org.apache.pinot.thirdeye.datalayer.bao.DAOTestBase;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.EvaluationManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
@@ -49,6 +50,7 @@ public class DetectionPipelineTaskRunnerTest {
   private DAOTestBase testDAOProvider;
   private DetectionConfigManager detectionDAO;
   private MergedAnomalyResultManager anomalyDAO;
+  private EvaluationManager evaluationDAO;
   private DetectionPipelineLoader loader;
   private DataProvider provider;
   private Map<String, Object> properties;
@@ -64,6 +66,7 @@ public class DetectionPipelineTaskRunnerTest {
     this.testDAOProvider = DAOTestBase.getInstance();
     this.detectionDAO = DAORegistry.getInstance().getDetectionConfigManager();
     this.anomalyDAO = DAORegistry.getInstance().getMergedAnomalyResultDAO();
+    this.evaluationDAO = DAORegistry.getInstance().getEvaluationManager();
     this.loader = new MockPipelineLoader(this.runs, this.outputs);
     this.provider = new MockDataProvider();
 
@@ -81,6 +84,7 @@ public class DetectionPipelineTaskRunnerTest {
     this.runner = new DetectionPipelineTaskRunner(
         this.detectionDAO,
         this.anomalyDAO,
+        this.evaluationDAO,
         this.loader,
         this.provider
     );
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/EvaluationTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/EvaluationTest.java
new file mode 100644
index 0000000..c57f67c
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/EvaluationTest.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;
+
+import org.apache.pinot.thirdeye.dataframe.DoubleSeries;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.*;
+
+
+public class EvaluationTest {
+
+  @Test
+  public void testCalculateMape() {
+    double mape = Evaluation.calculateMape(DoubleSeries.buildFrom(10, 20), DoubleSeries.buildFrom(20, 30));
+    Assert.assertEquals(mape, 0.75);
+  }
+
+  @Test
+  public void testCalculateZero() {
+    double mape = Evaluation.calculateMape(DoubleSeries.buildFrom(0, 20), DoubleSeries.buildFrom(1, 30));
+    Assert.assertEquals(mape, Double.POSITIVE_INFINITY);
+  }
+}
\ No newline at end of file


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