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/04/30 22:47:32 UTC

[incubator-pinot] branch master updated: [TE] anomaly detector interface change and implementation for rule-based detection (#4176)

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 1df6fb7  [TE] anomaly detector interface change and implementation for rule-based detection (#4176)
1df6fb7 is described below

commit 1df6fb7509901b892ddec0f82eb4292cedd0433e
Author: Jihao Zhang <ji...@linkedin.com>
AuthorDate: Tue Apr 30 15:47:27 2019 -0700

    [TE] anomaly detector interface change and implementation for rule-based detection (#4176)
    
    - Update the anomaly detector interface to return a detection result which includes anomalies and time series (containing time stamps, predicted baseline, current value, upper/lower bounds)
    - Implement the new detector interface in Threshold, Percentage Change, Absolute Change, and Holt-Winters detector.
    - Implement the baseline provider interfaces in Threshold, Percentage Change, Absolute Change classes.
---
 .../components/AbsoluteChangeRuleDetector.java     |  58 +++++++--
 .../detection/components/HoltWintersDetector.java  |  19 ++-
 .../components/PercentageChangeRuleDetector.java   |  62 +++++++---
 .../components/ThresholdRuleDetector.java          |  41 +++++--
 .../detection/spi/components/AnomalyDetector.java  |  11 +-
 .../detection/spi/model/DetectionResult.java       |  81 +++++++++++++
 .../thirdeye/detection/spi/model/TimeSeries.java   |  33 +++++-
 .../detection/wrapper/AnomalyDetectorWrapper.java  |  12 +-
 .../thirdeye/detection/yaml/YamlResource.java      |   2 +-
 .../components/AbsoluteChangeRuleDetectorTest.java |  17 ++-
 .../components/HoltWintersDetectorTest.java        |   4 +-
 .../PercentageChangeRuleDetectorTest.java          |  54 +++++++--
 .../components/ThresholdRuleDetectorTest.java      |  14 ++-
 .../compositePipelineTranslatorTestResult-1.json   | 130 ++++++++++-----------
 .../compositePipelineTranslatorTestResult-2.json   |  52 ++++-----
 15 files changed, 416 insertions(+), 174 deletions(-)

diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/AbsoluteChangeRuleDetector.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/AbsoluteChangeRuleDetector.java
index a5a2ab9..9b80912 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/AbsoluteChangeRuleDetector.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/AbsoluteChangeRuleDetector.java
@@ -19,10 +19,14 @@
 
 package org.apache.pinot.thirdeye.detection.components;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 import org.apache.pinot.thirdeye.common.time.TimeGranularity;
 import org.apache.pinot.thirdeye.dashboard.resources.v2.BaselineParsingUtils;
 import org.apache.pinot.thirdeye.dataframe.BooleanSeries;
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
+import org.apache.pinot.thirdeye.dataframe.DoubleSeries;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
@@ -35,16 +39,16 @@ import org.apache.pinot.thirdeye.detection.annotation.Param;
 import org.apache.pinot.thirdeye.detection.annotation.PresentationOption;
 import org.apache.pinot.thirdeye.detection.spec.AbsoluteChangeRuleDetectorSpec;
 import org.apache.pinot.thirdeye.detection.spi.components.AnomalyDetector;
+import org.apache.pinot.thirdeye.detection.spi.components.BaselineProvider;
+import org.apache.pinot.thirdeye.detection.spi.model.DetectionResult;
 import org.apache.pinot.thirdeye.detection.spi.model.InputData;
 import org.apache.pinot.thirdeye.detection.spi.model.InputDataSpec;
+import org.apache.pinot.thirdeye.detection.spi.model.TimeSeries;
 import org.apache.pinot.thirdeye.rootcause.impl.MetricEntity;
 import org.apache.pinot.thirdeye.rootcause.timeseries.Baseline;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
 import org.joda.time.Interval;
-import org.joda.time.Period;
 
+import static org.apache.pinot.thirdeye.dataframe.DoubleSeries.*;
 import static org.apache.pinot.thirdeye.dataframe.util.DataFrameUtils.*;
 
 
@@ -59,7 +63,8 @@ import static org.apache.pinot.thirdeye.dataframe.util.DataFrameUtils.*;
         @Param(name = "change", placeholder = "value"),
         @Param(name = "pattern", allowableValues = {"up", "down"})
     })
-public class AbsoluteChangeRuleDetector implements AnomalyDetector<AbsoluteChangeRuleDetectorSpec> {
+public class AbsoluteChangeRuleDetector implements AnomalyDetector<AbsoluteChangeRuleDetectorSpec>,
+                                                   BaselineProvider<AbsoluteChangeRuleDetectorSpec> {
   private double absoluteChange;
   private InputDataFetcher dataFetcher;
   private Baseline baseline;
@@ -68,14 +73,13 @@ public class AbsoluteChangeRuleDetector implements AnomalyDetector<AbsoluteChang
   private TimeGranularity timeGranularity;
 
   private static final String COL_CURR = "current";
-  private static final String COL_BASE = "baseline";
   private static final String COL_ANOMALY = "anomaly";
   private static final String COL_DIFF = "diff";
   private static final String COL_PATTERN = "pattern";
   private static final String COL_DIFF_VIOLATION = "diff_violation";
 
   @Override
-  public List<MergedAnomalyResultDTO> runDetection(Interval window, String metricUrn) {
+  public DetectionResult runDetection(Interval window, String metricUrn) {
     MetricEntity me = MetricEntity.fromURN(metricUrn);
     MetricSlice slice = MetricSlice.from(me.getId(), window.getStartMillis(), window.getEndMillis(), me.getFilters(), timeGranularity);
     List<MetricSlice> slices = new ArrayList<>(this.baseline.scatter(slice));
@@ -83,11 +87,11 @@ public class AbsoluteChangeRuleDetector implements AnomalyDetector<AbsoluteChang
     InputData data = this.dataFetcher.fetchData(new InputDataSpec().withTimeseriesSlices(slices).withMetricIdsForDataset(
         Collections.singletonList(slice.getMetricId())));
     DataFrame dfCurr = data.getTimeseries().get(slice).renameSeries(COL_VALUE, COL_CURR);
-    DataFrame dfBase = this.baseline.gather(slice, data.getTimeseries()).renameSeries(COL_VALUE, COL_BASE);
+    DataFrame dfBase = this.baseline.gather(slice, data.getTimeseries());
 
     DataFrame df = new DataFrame(dfCurr).addSeries(dfBase);
     // calculate absolute change
-    df.addSeries(COL_DIFF, df.getDoubles(COL_CURR).subtract(df.get(COL_BASE)));
+    df.addSeries(COL_DIFF, df.getDoubles(COL_CURR).subtract(df.get(COL_VALUE)));
 
     // defaults
     df.addSeries(COL_ANOMALY, BooleanSeries.fillValues(df.size(), false));
@@ -105,11 +109,45 @@ public class AbsoluteChangeRuleDetector implements AnomalyDetector<AbsoluteChang
 
     // make anomalies
     DatasetConfigDTO datasetConfig = data.getDatasetForMetricId().get(me.getId());
-    return DetectionUtils.makeAnomalies(slice, df, COL_ANOMALY, window.getEndMillis(),
+    List<MergedAnomalyResultDTO> anomalies = DetectionUtils.makeAnomalies(slice, df, COL_ANOMALY, window.getEndMillis(),
         DetectionUtils.getMonitoringGranularityPeriod(monitoringGranularity, datasetConfig), datasetConfig);
+    DataFrame baselineWithBoundaries = constructAbsoluteChangeBoundaries(df);
+    return DetectionResult.from(anomalies, TimeSeries.fromDataFrame(baselineWithBoundaries));
   }
 
   @Override
+  public TimeSeries computePredictedTimeSeries(MetricSlice slice) {
+    InputData data = this.dataFetcher.fetchData(new InputDataSpec().withTimeseriesSlices(this.baseline.scatter(slice)));
+    DataFrame dfBase = this.baseline.gather(slice, data.getTimeseries());
+    return TimeSeries.fromDataFrame(constructAbsoluteChangeBoundaries(dfBase));
+  }
+
+  private DataFrame constructAbsoluteChangeBoundaries(DataFrame dfBase) {
+    if (!Double.isNaN(this.absoluteChange)) {
+      switch (this.pattern) {
+        case UP:
+          fillAbsoluteChangeBound(dfBase, COL_UPPER_BOUND, this.absoluteChange);
+          dfBase.addSeries(COL_LOWER_BOUND, DoubleSeries.zeros(dfBase.size()));
+          break;
+        case DOWN:
+          dfBase.addSeries(COL_UPPER_BOUND, DoubleSeries.fillValues(dfBase.size(), POSITIVE_INFINITY));
+          fillAbsoluteChangeBound(dfBase, COL_LOWER_BOUND, -this.absoluteChange);
+          break;
+        case UP_OR_DOWN:
+          fillAbsoluteChangeBound(dfBase, COL_UPPER_BOUND, this.absoluteChange);
+          fillAbsoluteChangeBound(dfBase, COL_LOWER_BOUND, -this.absoluteChange);
+          break;
+        default:
+          throw new IllegalArgumentException();
+      }
+    }
+    return dfBase;
+  }
+
+  private void fillAbsoluteChangeBound(DataFrame dfBase, String colBound, double change) {
+    dfBase.addSeries(colBound, map((DoubleFunction) values -> values[0] + change, dfBase.getDoubles(COL_VALUE)));
+  }
+    @Override
   public void init(AbsoluteChangeRuleDetectorSpec spec, InputDataFetcher dataFetcher) {
     this.absoluteChange = spec.getAbsoluteChange();
     this.dataFetcher = dataFetcher;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/HoltWintersDetector.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/HoltWintersDetector.java
index 64949bc..123f44c 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/HoltWintersDetector.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/HoltWintersDetector.java
@@ -43,6 +43,7 @@ import org.apache.pinot.thirdeye.detection.annotation.Param;
 import org.apache.pinot.thirdeye.detection.spec.HoltWintersDetectorSpec;
 import org.apache.pinot.thirdeye.detection.spi.components.AnomalyDetector;
 import org.apache.pinot.thirdeye.detection.spi.components.BaselineProvider;
+import org.apache.pinot.thirdeye.detection.spi.model.DetectionResult;
 import org.apache.pinot.thirdeye.detection.spi.model.InputData;
 import org.apache.pinot.thirdeye.detection.spi.model.InputDataSpec;
 import org.apache.pinot.thirdeye.detection.spi.model.TimeSeries;
@@ -83,7 +84,6 @@ public class HoltWintersDetector implements BaselineProvider<HoltWintersDetector
   private static final Logger LOG = LoggerFactory.getLogger(HoltWintersDetector.class);
   private InputDataFetcher dataFetcher;
   private static final String COL_CURR = "current";
-  private static final String COL_BASE = "baseline";
   private static final String COL_ANOMALY = "anomaly";
   private static final String COL_PATTERN = "pattern";
   private static final String COL_DIFF = "diff";
@@ -150,7 +150,7 @@ public class HoltWintersDetector implements BaselineProvider<HoltWintersDetector
   }
 
   @Override
-  public List<MergedAnomalyResultDTO> runDetection(Interval window, String metricUrn) {
+  public DetectionResult runDetection(Interval window, String metricUrn) {
     MetricEntity metricEntity = MetricEntity.fromURN(metricUrn);
     DateTime trainStart;
     if (isMultiDayGranularity()) {
@@ -183,14 +183,9 @@ public class HoltWintersDetector implements BaselineProvider<HoltWintersDetector
     }
 
     DataFrame dfCurr = new DataFrame(dfInput).renameSeries(COL_VALUE, COL_CURR);
-    DataFrame dfBase = computePredictionInterval(dfInput, window.getStartMillis(), datasetConfig.getTimezone())
-        .renameSeries(COL_VALUE, COL_BASE);
-    // remove COL_CURR from baseline to use the smoothed value
-    if (dfBase.contains(COL_CURR)) {
-      dfBase.dropSeries(COL_CURR);
-    }
-    DataFrame df = new DataFrame(dfCurr).addSeries(dfBase);
-    df.addSeries(COL_DIFF, df.getDoubles(COL_CURR).subtract(df.get(COL_BASE)));
+    DataFrame dfBase = computePredictionInterval(dfInput, window.getStartMillis(), datasetConfig.getTimezone());
+    DataFrame df = new DataFrame(dfCurr).addSeries(dfBase, COL_VALUE, COL_ERROR);
+    df.addSeries(COL_DIFF, df.getDoubles(COL_CURR).subtract(df.get(COL_VALUE)));
     df.addSeries(COL_ANOMALY, BooleanSeries.fillValues(df.size(), false));
 
     // Filter pattern
@@ -204,12 +199,12 @@ public class HoltWintersDetector implements BaselineProvider<HoltWintersDetector
     df.mapInPlace(BooleanSeries.ALL_TRUE, COL_ANOMALY, COL_PATTERN, COL_DIFF_VIOLATION);
 
     // Anomalies
-    List<MergedAnomalyResultDTO> results = DetectionUtils.makeAnomalies(sliceData, df, COL_ANOMALY,
+    List<MergedAnomalyResultDTO> anomalyResults = DetectionUtils.makeAnomalies(sliceData, df, COL_ANOMALY,
         window.getEndMillis(),
         DetectionUtils.getMonitoringGranularityPeriod(timeGranularity.toAggregationGranularityString(),
             datasetConfig), datasetConfig);
 
-    return results;
+    return DetectionResult.from(anomalyResults, TimeSeries.fromDataFrame(dfBase));
   }
 
   /**
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/PercentageChangeRuleDetector.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/PercentageChangeRuleDetector.java
index ec828c6..bd4101a 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/PercentageChangeRuleDetector.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/PercentageChangeRuleDetector.java
@@ -19,10 +19,14 @@
 
 package org.apache.pinot.thirdeye.detection.components;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 import org.apache.pinot.thirdeye.common.time.TimeGranularity;
 import org.apache.pinot.thirdeye.dashboard.resources.v2.BaselineParsingUtils;
 import org.apache.pinot.thirdeye.dataframe.BooleanSeries;
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
+import org.apache.pinot.thirdeye.dataframe.DoubleSeries;
 import org.apache.pinot.thirdeye.dataframe.Series;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
@@ -36,16 +40,14 @@ import org.apache.pinot.thirdeye.detection.annotation.Param;
 import org.apache.pinot.thirdeye.detection.annotation.PresentationOption;
 import org.apache.pinot.thirdeye.detection.spec.PercentageChangeRuleDetectorSpec;
 import org.apache.pinot.thirdeye.detection.spi.components.AnomalyDetector;
+import org.apache.pinot.thirdeye.detection.spi.components.BaselineProvider;
+import org.apache.pinot.thirdeye.detection.spi.model.DetectionResult;
 import org.apache.pinot.thirdeye.detection.spi.model.InputData;
 import org.apache.pinot.thirdeye.detection.spi.model.InputDataSpec;
+import org.apache.pinot.thirdeye.detection.spi.model.TimeSeries;
 import org.apache.pinot.thirdeye.rootcause.impl.MetricEntity;
 import org.apache.pinot.thirdeye.rootcause.timeseries.Baseline;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
 import org.joda.time.Interval;
-import org.joda.time.Period;
-import org.joda.time.PeriodType;
 
 import static org.apache.pinot.thirdeye.dataframe.DoubleSeries.*;
 import static org.apache.pinot.thirdeye.dataframe.util.DataFrameUtils.*;
@@ -58,7 +60,7 @@ import static org.apache.pinot.thirdeye.dataframe.util.DataFrameUtils.*;
     @PresentationOption(name = "percentage change", template = "comparing ${offset} is ${pattern} more than ${percentageChange}")}, params = {
     @Param(name = "offset", defaultValue = "wo1w"), @Param(name = "percentageChange", placeholder = "value"),
     @Param(name = "pattern", allowableValues = {"up", "down"})})
-public class PercentageChangeRuleDetector implements AnomalyDetector<PercentageChangeRuleDetectorSpec> {
+public class PercentageChangeRuleDetector implements AnomalyDetector<PercentageChangeRuleDetectorSpec>, BaselineProvider<PercentageChangeRuleDetectorSpec> {
   private double percentageChange;
   private InputDataFetcher dataFetcher;
   private Baseline baseline;
@@ -67,24 +69,22 @@ public class PercentageChangeRuleDetector implements AnomalyDetector<PercentageC
   private TimeGranularity timeGranularity;
 
   private static final String COL_CURR = "current";
-  private static final String COL_BASE = "baseline";
   private static final String COL_CHANGE = "change";
   private static final String COL_ANOMALY = "anomaly";
   private static final String COL_PATTERN = "pattern";
   private static final String COL_CHANGE_VIOLATION = "change_violation";
 
   @Override
-  public List<MergedAnomalyResultDTO> runDetection(Interval window, String metricUrn) {
+  public DetectionResult runDetection(Interval window, String metricUrn) {
     MetricEntity me = MetricEntity.fromURN(metricUrn);
-    MetricSlice slice =
-        MetricSlice.from(me.getId(), window.getStartMillis(), window.getEndMillis(), me.getFilters(), timeGranularity);
+    MetricSlice slice = MetricSlice.from(me.getId(), window.getStartMillis(), window.getEndMillis(), me.getFilters(), timeGranularity);
     List<MetricSlice> slices = new ArrayList<>(this.baseline.scatter(slice));
     slices.add(slice);
 
     InputData data = this.dataFetcher.fetchData(new InputDataSpec().withTimeseriesSlices(slices)
         .withMetricIdsForDataset(Collections.singletonList(slice.getMetricId())));
     DataFrame dfCurr = data.getTimeseries().get(slice).renameSeries(COL_VALUE, COL_CURR);
-    DataFrame dfBase = this.baseline.gather(slice, data.getTimeseries()).renameSeries(COL_VALUE, COL_BASE);
+    DataFrame dfBase = this.baseline.gather(slice, data.getTimeseries());
 
     DataFrame df = new DataFrame(dfCurr).addSeries(dfBase);
 
@@ -94,7 +94,7 @@ public class PercentageChangeRuleDetector implements AnomalyDetector<PercentageC
         return Double.compare(values[0], 0.0) == 0 ? 0.0 : (values[0] > 0 ? Double.POSITIVE_INFINITY : Double.NEGATIVE_INFINITY);
       }
       return (values[0] - values[1]) / values[1];
-    }, df.getDoubles(COL_CURR), df.get(COL_BASE)));
+    }, df.getDoubles(COL_CURR), df.get(COL_VALUE)));
 
     // defaults
     df.addSeries(COL_ANOMALY, BooleanSeries.fillValues(df.size(), false));
@@ -112,10 +112,44 @@ public class PercentageChangeRuleDetector implements AnomalyDetector<PercentageC
       df.mapInPlace(BooleanSeries.ALL_TRUE, COL_ANOMALY, COL_PATTERN, COL_CHANGE_VIOLATION);
     }
 
-    // anomalies
     DatasetConfigDTO datasetConfig = data.getDatasetForMetricId().get(me.getId());
-    return DetectionUtils.makeAnomalies(slice, df, COL_ANOMALY, window.getEndMillis(),
+    List<MergedAnomalyResultDTO> anomalies = DetectionUtils.makeAnomalies(slice, df, COL_ANOMALY, window.getEndMillis(),
         DetectionUtils.getMonitoringGranularityPeriod(monitoringGranularity, datasetConfig), datasetConfig);
+    DataFrame baselineWithBoundaries = constructPercentageChangeBoundaries(df);
+    return DetectionResult.from(anomalies, TimeSeries.fromDataFrame(baselineWithBoundaries));
+  }
+
+  @Override
+  public TimeSeries computePredictedTimeSeries(MetricSlice slice) {
+    InputData data = this.dataFetcher.fetchData(new InputDataSpec().withTimeseriesSlices(this.baseline.scatter(slice)));
+    DataFrame dfBase = this.baseline.gather(slice, data.getTimeseries());
+    return TimeSeries.fromDataFrame(constructPercentageChangeBoundaries(dfBase));
+  }
+
+  private DataFrame constructPercentageChangeBoundaries(DataFrame dfBase) {
+    if (!Double.isNaN(this.percentageChange)) {
+      switch (this.pattern) {
+        case UP:
+          fillPercentageChangeBound(dfBase, COL_UPPER_BOUND, 1 + this.percentageChange);
+          dfBase.addSeries(COL_LOWER_BOUND, DoubleSeries.zeros(dfBase.size()));
+          break;
+        case DOWN:
+          dfBase.addSeries(COL_UPPER_BOUND, DoubleSeries.fillValues(dfBase.size(), POSITIVE_INFINITY));
+          fillPercentageChangeBound(dfBase, COL_LOWER_BOUND, 1 - this.percentageChange);
+          break;
+        case UP_OR_DOWN:
+          fillPercentageChangeBound(dfBase, COL_UPPER_BOUND, 1 + this.percentageChange);
+          fillPercentageChangeBound(dfBase, COL_LOWER_BOUND, 1 - this.percentageChange);
+          break;
+        default:
+          throw new IllegalArgumentException();
+      }
+    }
+    return dfBase;
+  }
+
+  private void fillPercentageChangeBound(DataFrame dfBase, String colBound, double multiplier) {
+    dfBase.addSeries(colBound, map((DoubleFunction) values -> values[0] * multiplier, dfBase.getDoubles(COL_VALUE)));
   }
 
   @Override
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/ThresholdRuleDetector.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/ThresholdRuleDetector.java
index 236e457..3d7fd83 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/ThresholdRuleDetector.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/ThresholdRuleDetector.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.apache.pinot.thirdeye.common.time.TimeGranularity;
 import org.apache.pinot.thirdeye.dataframe.BooleanSeries;
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
+import org.apache.pinot.thirdeye.dataframe.DoubleSeries;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
@@ -35,8 +36,11 @@ import org.apache.pinot.thirdeye.detection.annotation.Param;
 import org.apache.pinot.thirdeye.detection.annotation.PresentationOption;
 import org.apache.pinot.thirdeye.detection.spec.ThresholdRuleDetectorSpec;
 import org.apache.pinot.thirdeye.detection.spi.components.AnomalyDetector;
+import org.apache.pinot.thirdeye.detection.spi.components.BaselineProvider;
+import org.apache.pinot.thirdeye.detection.spi.model.DetectionResult;
 import org.apache.pinot.thirdeye.detection.spi.model.InputData;
 import org.apache.pinot.thirdeye.detection.spi.model.InputDataSpec;
+import org.apache.pinot.thirdeye.detection.spi.model.TimeSeries;
 import org.apache.pinot.thirdeye.rootcause.impl.MetricEntity;
 import org.joda.time.Interval;
 import org.joda.time.Period;
@@ -48,7 +52,7 @@ import static org.apache.pinot.thirdeye.dataframe.util.DataFrameUtils.*;
     DetectionTag.RULE_DETECTION}, description = "Simple threshold rule algorithm with (optional) upper and lower bounds on a metric value.", presentation = {
     @PresentationOption(name = "absolute value", template = "is lower than ${min} or higher than ${max}")}, params = {
     @Param(name = "min", placeholder = "value"), @Param(name = "max", placeholder = "value")})
-public class ThresholdRuleDetector implements AnomalyDetector<ThresholdRuleDetectorSpec> {
+public class ThresholdRuleDetector implements AnomalyDetector<ThresholdRuleDetectorSpec>, BaselineProvider<ThresholdRuleDetectorSpec> {
   private final String COL_TOO_HIGH = "tooHigh";
   private final String COL_TOO_LOW = "tooLow";
   private final String COL_ANOMALY = "anomaly";
@@ -60,7 +64,7 @@ public class ThresholdRuleDetector implements AnomalyDetector<ThresholdRuleDetec
   private TimeGranularity timeGranularity;
 
   @Override
-  public List<MergedAnomalyResultDTO> runDetection(Interval window, String metricUrn) {
+  public DetectionResult runDetection(Interval window, String metricUrn) {
     MetricEntity me = MetricEntity.fromURN(metricUrn);
     Long endTime = window.getEndMillis();
     MetricSlice slice = MetricSlice.from(me.getId(), window.getStartMillis(), endTime, me.getFilters(), timeGranularity);
@@ -68,7 +72,7 @@ public class ThresholdRuleDetector implements AnomalyDetector<ThresholdRuleDetec
     InputData data = this.dataFetcher.fetchData(
         new InputDataSpec().withTimeseriesSlices(Collections.singletonList(slice))
             .withMetricIdsForDataset(Collections.singletonList(me.getId())));
-    DataFrame df = data.getTimeseries().get(slice);
+    DataFrame df = data.getTimeseries().get(slice).renameSeries(COL_VALUE, COL_CURRENT);
 
     // defaults
     df.addSeries(COL_TOO_HIGH, BooleanSeries.fillValues(df.size(), false));
@@ -76,19 +80,39 @@ public class ThresholdRuleDetector implements AnomalyDetector<ThresholdRuleDetec
 
     // max
     if (!Double.isNaN(this.max)) {
-      df.addSeries(COL_TOO_HIGH, df.getDoubles(COL_VALUE).gt(this.max));
+      df.addSeries(COL_TOO_HIGH, df.getDoubles(COL_CURRENT).gt(this.max));
     }
 
     // min
     if (!Double.isNaN(this.min)) {
-      df.addSeries(COL_TOO_LOW, df.getDoubles(COL_VALUE).lt(this.min));
+      df.addSeries(COL_TOO_LOW, df.getDoubles(COL_CURRENT).lt(this.min));
     }
-
+    // predicted value is the same as the current value
+    df.addSeries(COL_VALUE, df.get(COL_CURRENT));
     df.mapInPlace(BooleanSeries.HAS_TRUE, COL_ANOMALY, COL_TOO_HIGH, COL_TOO_LOW);
-
     DatasetConfigDTO datasetConfig = data.getDatasetForMetricId().get(me.getId());
-    return DetectionUtils.makeAnomalies(slice, df, COL_ANOMALY, endTime,
+    List<MergedAnomalyResultDTO> anomalies = DetectionUtils.makeAnomalies(slice, df, COL_ANOMALY, endTime,
         DetectionUtils.getMonitoringGranularityPeriod(monitoringGranularity, datasetConfig), datasetConfig);
+    DataFrame baselineWithBoundaries = constructThresholdBoundaries(df);
+    return DetectionResult.from(anomalies, TimeSeries.fromDataFrame(baselineWithBoundaries));
+  }
+
+  @Override
+  public TimeSeries computePredictedTimeSeries(MetricSlice slice) {
+    InputData data =
+        this.dataFetcher.fetchData(new InputDataSpec().withTimeseriesSlices(Collections.singletonList(slice)));
+    DataFrame df = data.getTimeseries().get(slice);
+    return TimeSeries.fromDataFrame(constructThresholdBoundaries(df));
+  }
+
+  private DataFrame constructThresholdBoundaries(DataFrame df) {
+    if (!Double.isNaN(this.min)) {
+      df.addSeries(COL_LOWER_BOUND, DoubleSeries.fillValues(df.size(), this.min));
+    }
+    if (!Double.isNaN(this.max)) {
+      df.addSeries(COL_UPPER_BOUND, DoubleSeries.fillValues(df.size(), this.max));
+    }
+    return df;
   }
 
   @Override
@@ -102,6 +126,5 @@ public class ThresholdRuleDetector implements AnomalyDetector<ThresholdRuleDetec
     } else {
       this.timeGranularity = TimeGranularity.fromString(spec.getMonitoringGranularity());
     }
-
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/components/AnomalyDetector.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/components/AnomalyDetector.java
index 072f977..5eb4135 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/components/AnomalyDetector.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/components/AnomalyDetector.java
@@ -19,19 +19,16 @@
 
 package org.apache.pinot.thirdeye.detection.spi.components;
 
-import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import org.apache.pinot.thirdeye.detection.spec.AbstractSpec;
 import org.apache.pinot.thirdeye.detection.spi.exception.DetectorException;
-import org.apache.pinot.thirdeye.detection.spi.model.InputData;
-import org.apache.pinot.thirdeye.detection.spi.model.InputDataSpec;
-import java.util.List;
+import org.apache.pinot.thirdeye.detection.spi.model.DetectionResult;
 import org.joda.time.Interval;
 
 
 public interface AnomalyDetector<T extends AbstractSpec> extends BaseComponent<T> {
   /**
-   * Run detection in the specified time range and return a list of anomalies
-   * @return list of anomalies
+   * Run detection in the specified time range and return the detection result
+   * @return the detection result which contains anomalies and current, predicted baselines.
    */
-  List<MergedAnomalyResultDTO> runDetection(Interval window, String metricUrn) throws DetectorException;
+  DetectionResult runDetection(Interval window, String metricUrn) throws DetectorException;
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/DetectionResult.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/DetectionResult.java
new file mode 100644
index 0000000..debd4e7
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/DetectionResult.java
@@ -0,0 +1,81 @@
+/*
+ *
+ *  * 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 java.util.Collections;
+import java.util.List;
+import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+
+
+/**
+ * The detection result. Contains a list of anomalies detected and time series
+ * (which can include time stamps, predicted baseline, current value, upper/lower bounds)
+ */
+public class DetectionResult {
+  private final List<MergedAnomalyResultDTO> anomalies;
+  private final TimeSeries timeseries;
+
+  private DetectionResult(List<MergedAnomalyResultDTO> anomalies, TimeSeries timeseries) {
+    this.anomalies = anomalies;
+    this.timeseries = timeseries;
+  }
+
+  public List<MergedAnomalyResultDTO> getAnomalies() {
+    return anomalies;
+  }
+
+  public TimeSeries getTimeseries() {
+    return timeseries;
+  }
+
+  /**
+   * Create a empty detection result
+   * @return the empty detection result
+   */
+  public static DetectionResult empty() {
+    return new DetectionResult(Collections.emptyList(), TimeSeries.empty());
+  }
+
+  /**
+   * Create a detection result from a list of anomalies
+   * @param anomalies the list of anomalies generated
+   * @return the detection result contains the list of anomalies
+   */
+  public static DetectionResult from(List<MergedAnomalyResultDTO> anomalies) {
+    return new DetectionResult(anomalies, TimeSeries.empty());
+  }
+
+  /**
+   * Create a detection result from a list of anomalies and time series
+   * @param anomalies the list of anomalies generated
+   * @param timeSeries the time series which including the current, predicted baseline and optionally upper and lower bounds
+   * @return the detection result contains the list of anomalies and the time series
+   */
+  public static DetectionResult from(List<MergedAnomalyResultDTO> anomalies, TimeSeries timeSeries) {
+    return new DetectionResult(anomalies, timeSeries);
+  }
+
+  @Override
+  public String toString() {
+    return "DetectionResult{" + "anomalies=" + anomalies + ", timeseries=" + timeseries + '}';
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/TimeSeries.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/TimeSeries.java
index a513037..9cae39e 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/TimeSeries.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spi/model/TimeSeries.java
@@ -19,6 +19,7 @@
 
 package org.apache.pinot.thirdeye.detection.spi.model;
 
+import com.google.common.base.Preconditions;
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
 import org.apache.pinot.thirdeye.dataframe.DoubleSeries;
 import org.apache.pinot.thirdeye.dataframe.LongSeries;
@@ -33,7 +34,7 @@ import static org.apache.pinot.thirdeye.dataframe.util.DataFrameUtils.*;
 public class TimeSeries {
   private DataFrame df;
 
-  public TimeSeries() {
+  private TimeSeries() {
     this.df = new DataFrame();
   }
 
@@ -52,6 +53,14 @@ public class TimeSeries {
   }
 
   /**
+   * the size of the time series
+   * @return the size of the time series (number of data points)
+   */
+  public int size() {
+    return this.df.size();
+  }
+
+  /**
    * Add the series into TimeSeries if it exists in the DataFrame.
    * @param df The source DataFrame.
    * @param name The series name.
@@ -63,16 +72,33 @@ public class TimeSeries {
   }
 
   /**
+   * return a empty time series
+   * @return a empty time series
+   */
+  public static TimeSeries empty(){
+    TimeSeries ts = new TimeSeries();
+    ts.df.addSeries(COL_TIME, LongSeries.empty()).addSeries(COL_VALUE, DoubleSeries.empty()).setIndex(COL_TIME);
+    return ts;
+  }
+
+  /**
    * Add DataFrame into TimeSeries.
    * @param df The source DataFrame.
    * @return TimeSeries that contains the predicted values.
    */
   public static TimeSeries fromDataFrame(DataFrame df) {
+    Preconditions.checkArgument(df.contains(COL_TIME));
+    Preconditions.checkArgument(df.contains(COL_VALUE));
     TimeSeries ts = new TimeSeries();
+    // time stamp
     ts.df.addSeries(COL_TIME, df.get(COL_TIME)).setIndex(COL_TIME);
+    // predicted baseline values
     addSeries(ts, df, COL_VALUE);
+    // current values
     addSeries(ts, df, COL_CURRENT);
+    // upper bound
     addSeries(ts, df, COL_UPPER_BOUND);
+    // lower bound
     addSeries(ts, df, COL_LOWER_BOUND);
     return ts;
   }
@@ -100,4 +126,9 @@ public class TimeSeries {
   public DataFrame getDataFrame() {
     return df;
   }
+
+  @Override
+  public String toString() {
+    return "TimeSeries{" + "df=" + df + '}';
+  }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/AnomalyDetectorWrapper.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/AnomalyDetectorWrapper.java
index 283dd35..bba9e4e 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/AnomalyDetectorWrapper.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/AnomalyDetectorWrapper.java
@@ -38,19 +38,19 @@ import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
 import org.apache.pinot.thirdeye.detection.DataProvider;
-import org.apache.pinot.thirdeye.detection.DetectionPipelineException;
 import org.apache.pinot.thirdeye.detection.DetectionPipeline;
+import org.apache.pinot.thirdeye.detection.DetectionPipelineException;
 import org.apache.pinot.thirdeye.detection.DetectionPipelineResult;
 import org.apache.pinot.thirdeye.detection.DetectionUtils;
 import org.apache.pinot.thirdeye.detection.spi.components.AnomalyDetector;
 import org.apache.pinot.thirdeye.detection.spi.exception.DetectorDataInsufficientException;
+import org.apache.pinot.thirdeye.detection.spi.model.DetectionResult;
 import org.apache.pinot.thirdeye.rootcause.impl.MetricEntity;
 import org.apache.pinot.thirdeye.util.ThirdEyeUtils;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.Interval;
 import org.joda.time.Period;
-import org.omg.SendingContext.RunTime;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -168,15 +168,15 @@ public class AnomalyDetectorWrapper extends DetectionPipeline {
 
       // run detection
       Interval window = monitoringWindows.get(i);
-      List<MergedAnomalyResultDTO> anomaliesForOneWindow = new ArrayList<>();
+      DetectionResult detectionResult = DetectionResult.empty();
       try {
         LOG.info("[Pipeline] start detection for config {} metricUrn {} window ({}/{}) - start {} end {}",
             config.getId(), metricUrn, i + 1, monitoringWindows.size(), window.getStart(), window.getEnd());
         long ts = System.currentTimeMillis();
-        anomaliesForOneWindow = anomalyDetector.runDetection(window, this.metricUrn);
+        detectionResult = anomalyDetector.runDetection(window, this.metricUrn);
         LOG.info("[Pipeline] end detection for config {} metricUrn {} window ({}/{}) - start {} end {} used {} milliseconds, detected {} anomalies",
             config.getId(), metricUrn, i + 1, monitoringWindows.size(), window.getStart(), window.getEnd(),
-            System.currentTimeMillis() - ts, anomaliesForOneWindow.size());
+            System.currentTimeMillis() - ts, detectionResult.getAnomalies().size());
         successWindows++;
       }
       catch (DetectorDataInsufficientException e) {
@@ -187,7 +187,7 @@ public class AnomalyDetectorWrapper extends DetectionPipeline {
         LOG.warn("[DetectionConfigID{}] detecting anomalies for window {} to {} failed.", this.config.getId(), window.getStart(), window.getEnd(), e);
         lastException = e;
       }
-      anomalies.addAll(anomaliesForOneWindow);
+      anomalies.addAll(detectionResult.getAnomalies());
     }
 
     checkMovingWindowDetectionStatus(totalWindows, successWindows, lastException);
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 5a6278d..64f6577 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
@@ -719,7 +719,7 @@ public class YamlResource {
       return provider.get().computePredictedTimeSeries(slice);
     }
 
-    return new TimeSeries();
+    return TimeSeries.empty();
   }
 
   /**
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/AbsoluteChangeRuleDetectorTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/AbsoluteChangeRuleDetectorTest.java
index cf2bb17..56f19b3 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/AbsoluteChangeRuleDetectorTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/AbsoluteChangeRuleDetectorTest.java
@@ -17,6 +17,7 @@
 package org.apache.pinot.thirdeye.detection.components;
 
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
+import org.apache.pinot.thirdeye.dataframe.DoubleSeries;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
@@ -35,6 +36,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
+import org.apache.pinot.thirdeye.detection.spi.model.DetectionResult;
+import org.apache.pinot.thirdeye.detection.spi.model.TimeSeries;
 import org.joda.time.Interval;
 import org.testng.Assert;
 import org.testng.annotations.BeforeMethod;
@@ -81,14 +84,24 @@ public class AbsoluteChangeRuleDetectorTest {
   public void testWeekOverWeekDifference() {
     AbsoluteChangeRuleDetector detector = new AbsoluteChangeRuleDetector();
     AbsoluteChangeRuleDetectorSpec spec = new AbsoluteChangeRuleDetectorSpec();
-    spec.setAbsoluteChange(400);
+    double absoluteChange = 400;
+    spec.setAbsoluteChange(absoluteChange);
     spec.setPattern("up");
     detector.init(spec, new DefaultInputDataFetcher(this.provider, -1));
-    List<MergedAnomalyResultDTO> anomalies = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+    DetectionResult result = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+    List<MergedAnomalyResultDTO> anomalies = result.getAnomalies();
 
     Assert.assertEquals(anomalies.size(), 1);
     Assert.assertEquals(anomalies.get(0).getStartTime(), 2372400000L);
     Assert.assertEquals(anomalies.get(0).getEndTime(), 2376000000L);
+    TimeSeries ts = result.getTimeseries();
+    checkAbsoluteUpperBounds(ts, absoluteChange);
+    Assert.assertEquals(ts.getPredictedLowerBound(), DoubleSeries.zeros(ts.size()));
   }
 
+  private void checkAbsoluteUpperBounds(TimeSeries ts, double absoluteChange) {
+    for (int i = 0; i < ts.getDataFrame().size(); i++) {
+      Assert.assertEquals(ts.getPredictedUpperBound().get(i), ts.getPredictedBaseline().get(i) + absoluteChange);
+    }
+  }
 }
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/HoltWintersDetectorTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/HoltWintersDetectorTest.java
index ba3e299..ade1e9f 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/HoltWintersDetectorTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/HoltWintersDetectorTest.java
@@ -123,7 +123,7 @@ public class HoltWintersDetectorTest {
     detector.init(spec, new DefaultInputDataFetcher(this.provider, -1));
     Interval window = new Interval(1306627200000L, 1309219200000L);
     String metricUrn = "thirdeye:metric:1";
-    List<MergedAnomalyResultDTO> anomalies = detector.runDetection(window, metricUrn);
+    List<MergedAnomalyResultDTO> anomalies = detector.runDetection(window, metricUrn).getAnomalies();
 
     Assert.assertEquals(anomalies.size(), 6);
   }
@@ -152,7 +152,7 @@ public class HoltWintersDetectorTest {
     detector.init(spec, new DefaultInputDataFetcher(this.provider, -1));
     Interval window = new Interval(1322773200000L, 1323378000000L);
     String metricUrn = "thirdeye:metric:123";
-    List<MergedAnomalyResultDTO> anomalies = detector.runDetection(window, metricUrn);
+    List<MergedAnomalyResultDTO> anomalies = detector.runDetection(window, metricUrn).getAnomalies();
 
     Assert.assertEquals(anomalies.size(), 2);
   }
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/PercentageChangeRuleDetectorTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/PercentageChangeRuleDetectorTest.java
index 9bbe29b..75e58b2 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/PercentageChangeRuleDetectorTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/PercentageChangeRuleDetectorTest.java
@@ -17,6 +17,7 @@
 package org.apache.pinot.thirdeye.detection.components;
 
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
+import org.apache.pinot.thirdeye.dataframe.DoubleSeries;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
@@ -37,6 +38,8 @@ import java.util.concurrent.TimeUnit;
 import org.apache.pinot.thirdeye.detection.spec.ThresholdRuleDetectorSpec;
 import org.apache.pinot.thirdeye.detection.spi.components.AnomalyDetector;
 import org.apache.pinot.thirdeye.detection.spi.exception.DetectorException;
+import org.apache.pinot.thirdeye.detection.spi.model.DetectionResult;
+import org.apache.pinot.thirdeye.detection.spi.model.TimeSeries;
 import org.joda.time.Interval;
 import org.testng.Assert;
 import org.testng.annotations.BeforeMethod;
@@ -94,25 +97,32 @@ public class PercentageChangeRuleDetectorTest {
     PercentageChangeRuleDetector detector = new PercentageChangeRuleDetector();
     PercentageChangeRuleDetectorSpec spec = new PercentageChangeRuleDetectorSpec();
     spec.setPattern("up");
-    spec.setPercentageChange(0.4);
+    double percentageChange = 0.4;
+    spec.setPercentageChange(percentageChange);
     detector.init(spec, new DefaultInputDataFetcher(this.provider, -1));
-    List<MergedAnomalyResultDTO> anomalies = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+    DetectionResult detectionResult = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+    List<MergedAnomalyResultDTO> anomalies = detectionResult.getAnomalies();
     Assert.assertEquals(anomalies.size(), 2);
     Assert.assertEquals(anomalies.get(0).getStartTime(), 2372400000L);
     Assert.assertEquals(anomalies.get(0).getEndTime(), 2376000000L);
     Assert.assertEquals(anomalies.get(1).getStartTime(), 2379600000L);
     Assert.assertEquals(anomalies.get(1).getEndTime(), 2383200000L);
+    TimeSeries ts = detectionResult.getTimeseries();
+    checkPercentageUpperBounds(ts, percentageChange);
+    Assert.assertEquals(ts.getPredictedLowerBound(), DoubleSeries.zeros(ts.size()));
   }
 
   @Test
   public void testThreeWeekMedianChange() {
     PercentageChangeRuleDetector detector = new PercentageChangeRuleDetector();
     PercentageChangeRuleDetectorSpec spec = new PercentageChangeRuleDetectorSpec();
-    spec.setPercentageChange(0.3);
+    double percentageChange = 0.3;
+    spec.setPercentageChange(percentageChange);
     spec.setOffset("median3w");
     spec.setPattern("up");
     detector.init(spec, new DefaultInputDataFetcher(this.provider, -1));
-    List<MergedAnomalyResultDTO> anomalies = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+    DetectionResult detectionResult = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+    List<MergedAnomalyResultDTO> anomalies = detectionResult.getAnomalies();
     Assert.assertEquals(anomalies.size(), 4);
     Assert.assertEquals(anomalies.get(0).getStartTime(), 2005200000L);
     Assert.assertEquals(anomalies.get(0).getEndTime(), 2008800000L);
@@ -122,31 +132,41 @@ public class PercentageChangeRuleDetectorTest {
     Assert.assertEquals(anomalies.get(2).getEndTime(), 2156400000L);
     Assert.assertEquals(anomalies.get(3).getStartTime(), 2322000000L);
     Assert.assertEquals(anomalies.get(3).getEndTime(), 2325600000L);
+    TimeSeries ts = detectionResult.getTimeseries();
+    checkPercentageUpperBounds(ts, percentageChange);
+    Assert.assertEquals(ts.getPredictedLowerBound(), DoubleSeries.zeros(ts.size()));
   }
 
   @Test
   public void testThreeWeekMedianChangeDown() {
     PercentageChangeRuleDetector detector = new PercentageChangeRuleDetector();
     PercentageChangeRuleDetectorSpec spec = new PercentageChangeRuleDetectorSpec();
-    spec.setPercentageChange(0.3);
+    double percentageChange = 0.3;
+    spec.setPercentageChange(percentageChange);
     spec.setOffset("median3w");
     spec.setPattern("down");
     detector.init(spec, new DefaultInputDataFetcher(this.provider, -1));
-    List<MergedAnomalyResultDTO> anomalies = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+    DetectionResult result = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+    List<MergedAnomalyResultDTO> anomalies = result.getAnomalies();
     Assert.assertEquals(anomalies.size(), 1);
     Assert.assertEquals(anomalies.get(0).getStartTime(), 2181600000L);
     Assert.assertEquals(anomalies.get(0).getEndTime(), 2185200000L);
+    TimeSeries ts = result.getTimeseries();
+    checkPercentageLowerBounds(ts, percentageChange);
+    Assert.assertEquals(ts.getPredictedUpperBound(), DoubleSeries.fillValues(ts.size(), Double.POSITIVE_INFINITY));
   }
 
   @Test
   public void testThreeWeekMedianChangeUporDown() {
     PercentageChangeRuleDetector detector = new PercentageChangeRuleDetector();
     PercentageChangeRuleDetectorSpec spec = new PercentageChangeRuleDetectorSpec();
-    spec.setPercentageChange(0.3);
+    double percentageChange = 0.3;
+    spec.setPercentageChange(percentageChange);
     spec.setOffset("median3w");
     spec.setPattern("up_or_down");
     detector.init(spec, new DefaultInputDataFetcher(this.provider, -1));
-    List<MergedAnomalyResultDTO> anomalies = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+    DetectionResult result = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+    List<MergedAnomalyResultDTO> anomalies = result.getAnomalies();
     Assert.assertEquals(anomalies.size(), 5);
     Assert.assertEquals(anomalies.get(0).getStartTime(), 2005200000L);
     Assert.assertEquals(anomalies.get(0).getEndTime(), 2008800000L);
@@ -158,6 +178,8 @@ public class PercentageChangeRuleDetectorTest {
     Assert.assertEquals(anomalies.get(3).getEndTime(), 2185200000L);
     Assert.assertEquals(anomalies.get(4).getStartTime(), 2322000000L);
     Assert.assertEquals(anomalies.get(4).getEndTime(), 2325600000L);
+    checkPercentageUpperBounds(result.getTimeseries(), percentageChange);
+    checkPercentageLowerBounds(result.getTimeseries(), percentageChange);
   }
 
   @Test
@@ -168,7 +190,7 @@ public class PercentageChangeRuleDetectorTest {
     spec.setPercentageChange(0.4);
     spec.setMonitoringGranularity("1_MONTHS");
     percentageRule.init(spec, new DefaultInputDataFetcher(this.provider, -1));
-    List<MergedAnomalyResultDTO> anomalies = percentageRule.runDetection(new Interval(1546214400000L, 1551312000000L), "thirdeye:metric:1");
+    List<MergedAnomalyResultDTO> anomalies = percentageRule.runDetection(new Interval(1546214400000L, 1551312000000L), "thirdeye:metric:1").getAnomalies();
     Assert.assertEquals(anomalies.size(), 1);
     Assert.assertEquals(anomalies.get(0).getStartTime(), 1548892800000L);
     Assert.assertEquals(anomalies.get(0).getEndTime(), 1551312000000L);
@@ -181,9 +203,21 @@ public class PercentageChangeRuleDetectorTest {
     spec.setOffset("wo1w");
     spec.setPercentageChange(0.1);
     percentageRule.init(spec, new DefaultInputDataFetcher(this.provider, -1));
-    List<MergedAnomalyResultDTO> anomalies = percentageRule.runDetection(new Interval(1551398400000L, 1551571200000L), "thirdeye:metric:1");
+    List<MergedAnomalyResultDTO> anomalies = percentageRule.runDetection(new Interval(1551398400000L, 1551571200000L), "thirdeye:metric:1").getAnomalies();
     Assert.assertEquals(anomalies.size(), 1);
     Assert.assertEquals(anomalies.get(0).getStartTime(), 1551484800000L);
     Assert.assertEquals(anomalies.get(0).getEndTime(), 1551488400000L);
   }
+
+  private void checkPercentageUpperBounds(TimeSeries ts, double percentageChange) {
+    for (int i = 0; i < ts.getDataFrame().size(); i++) {
+      Assert.assertEquals(ts.getPredictedUpperBound().get(i), ts.getPredictedBaseline().get(i) * (1 + percentageChange));
+    }
+  }
+
+  private void checkPercentageLowerBounds(TimeSeries ts, double percentageChange) {
+    for (int i = 0; i < ts.getDataFrame().size(); i++) {
+      Assert.assertEquals(ts.getPredictedLowerBound().get(i), ts.getPredictedBaseline().get(i) * (1 - percentageChange));
+    }
+  }
 }
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/ThresholdRuleDetectorTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/ThresholdRuleDetectorTest.java
index 92d84c6..b93089b 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/ThresholdRuleDetectorTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/ThresholdRuleDetectorTest.java
@@ -22,6 +22,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
+import org.apache.pinot.thirdeye.dataframe.DoubleSeries;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
@@ -33,6 +34,8 @@ import org.apache.pinot.thirdeye.detection.MockDataProvider;
 import org.apache.pinot.thirdeye.detection.spec.ThresholdRuleDetectorSpec;
 import org.apache.pinot.thirdeye.detection.spi.components.AnomalyDetector;
 import org.apache.pinot.thirdeye.detection.spi.exception.DetectorException;
+import org.apache.pinot.thirdeye.detection.spi.model.DetectionResult;
+import org.apache.pinot.thirdeye.detection.spi.model.TimeSeries;
 import org.joda.time.Interval;
 import org.testng.Assert;
 import org.testng.annotations.BeforeMethod;
@@ -89,12 +92,17 @@ public class ThresholdRuleDetectorTest {
     spec.setMin(100);
     spec.setMax(500);
     thresholdAlgorithm.init(spec, new DefaultInputDataFetcher(testDataProvider, -1));
-    List<MergedAnomalyResultDTO> anomalies = thresholdAlgorithm.runDetection(new Interval(0, 10), "thirdeye:metric:123");
+    DetectionResult result = thresholdAlgorithm.runDetection(new Interval(0, 10), "thirdeye:metric:123");
+    List<MergedAnomalyResultDTO> anomalies = result.getAnomalies();
     Assert.assertEquals(anomalies.size(), 2);
     Assert.assertEquals(anomalies.get(0).getStartTime(), 0);
     Assert.assertEquals(anomalies.get(0).getEndTime(), 2);
     Assert.assertEquals(anomalies.get(1).getStartTime(), 8);
     Assert.assertEquals(anomalies.get(1).getEndTime(), 10);
+    TimeSeries ts = result.getTimeseries();
+    Assert.assertEquals(ts.getPredictedUpperBound(), DoubleSeries.fillValues(ts.size(), 500));
+    Assert.assertEquals(ts.getPredictedLowerBound(), DoubleSeries.fillValues(ts.size(), 100));
+    Assert.assertEquals(ts.getPredictedBaseline(), ts.getCurrent());
   }
 
   @Test
@@ -104,7 +112,7 @@ public class ThresholdRuleDetectorTest {
     spec.setMin(200);
     spec.setMonitoringGranularity("1_MONTHS");
     thresholdRule.init(spec, new DefaultInputDataFetcher(testDataProvider, -1));
-    List<MergedAnomalyResultDTO> anomalies = thresholdRule.runDetection(new Interval(1546214400000L, 1551398400000L), "thirdeye:metric:123");
+    List<MergedAnomalyResultDTO> anomalies = thresholdRule.runDetection(new Interval(1546214400000L, 1551398400000L), "thirdeye:metric:123").getAnomalies();
     Assert.assertEquals(anomalies.size(), 1);
     Assert.assertEquals(anomalies.get(0).getStartTime(), 1546214400000L);
     Assert.assertEquals(anomalies.get(0).getEndTime(), 1548892800000L);
@@ -117,7 +125,7 @@ public class ThresholdRuleDetectorTest {
     spec.setMax(200);
     spec.setMonitoringGranularity("1_MONTHS");
     thresholdRule.init(spec, new DefaultInputDataFetcher(testDataProvider, -1));
-    List<MergedAnomalyResultDTO> anomalies = thresholdRule.runDetection(new Interval(1546214400000L, 1551398400000L), "thirdeye:metric:123");
+    List<MergedAnomalyResultDTO> anomalies = thresholdRule.runDetection(new Interval(1546214400000L, 1551398400000L), "thirdeye:metric:123").getAnomalies();
     Assert.assertEquals(anomalies.size(), 1);
     Assert.assertEquals(anomalies.get(0).getStartTime(), 1551312000000L);
     Assert.assertEquals(anomalies.get(0).getEndTime(), 1551398400000L);
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/compositePipelineTranslatorTestResult-1.json b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/compositePipelineTranslatorTestResult-1.json
index 4630cd8..9da8c2f 100644
--- a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/compositePipelineTranslatorTestResult-1.json
+++ b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/compositePipelineTranslatorTestResult-1.json
@@ -1,78 +1,70 @@
 {
-  "properties" : {
-    "className" : "org.apache.pinot.thirdeye.detection.wrapper.ChildKeepingMergeWrapper",
-    "maxGap" : 0,
-    "nested" : [ {
-      "nestedMetricUrns" : [ "thirdeye:metric:1:D1%3Dv1:D1%3Dv2:D2%3Dv3" ],
-      "className" : "org.apache.pinot.thirdeye.detection.algorithm.DimensionWrapper",
-      "metricUrn" : "thirdeye:metric:1:D1%3Dv1:D1%3Dv2:D2%3Dv3",
-      "nested" : [ {
-        "filter" : "$thresholdFilter_2:THRESHOLD_RULE_FILTER",
-        "className" : "org.apache.pinot.thirdeye.detection.wrapper.AnomalyFilterWrapper",
-        "nested" : [ {
-          "filter" : "$thresholdFilter_1:THRESHOLD_RULE_FILTER",
-          "className" : "org.apache.pinot.thirdeye.detection.wrapper.AnomalyFilterWrapper",
-          "nested" : [ {
-            "baselineValueProvider" : "$maxThreshold_1:RULE_BASELINE",
-            "className" : "org.apache.pinot.thirdeye.detection.wrapper.BaselineFillingMergeWrapper",
-            "maxGap" : 0,
-            "nested" : [ {
-              "bucketPeriod":"P1D",
-              "className" : "org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper"
-            } ],
-            "detector" : "$maxThreshold_1:THRESHOLD",
-            "maxDuration" : 100
-          } ]
-        } ]
+  "properties": {
+    "className": "org.apache.pinot.thirdeye.detection.wrapper.ChildKeepingMergeWrapper",
+    "maxGap": 0,
+    "nested": [{
+      "nestedMetricUrns": ["thirdeye:metric:1:D1%3Dv1:D1%3Dv2:D2%3Dv3"],
+      "className": "org.apache.pinot.thirdeye.detection.algorithm.DimensionWrapper",
+      "metricUrn": "thirdeye:metric:1:D1%3Dv1:D1%3Dv2:D2%3Dv3",
+      "nested": [{
+        "filter": "$thresholdFilter_2:THRESHOLD_RULE_FILTER",
+        "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyFilterWrapper",
+        "nested": [{
+          "filter": "$thresholdFilter_1:THRESHOLD_RULE_FILTER",
+          "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyFilterWrapper",
+          "nested": [{
+            "baselineValueProvider": "$maxThreshold_1:THRESHOLD",
+            "className": "org.apache.pinot.thirdeye.detection.wrapper.BaselineFillingMergeWrapper",
+            "maxGap": 0,
+            "nested": [{
+              "bucketPeriod": "P1D",
+              "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper"
+            }],
+            "detector": "$maxThreshold_1:THRESHOLD",
+            "maxDuration": 100
+          }]
+        }]
       }, {
-        "filter" : "$thresholdFilter_3:THRESHOLD_RULE_FILTER",
-        "className" : "org.apache.pinot.thirdeye.detection.wrapper.AnomalyFilterWrapper",
-        "nested" : [ {
-          "baselineValueProvider" : "$maxThreshold_2:RULE_BASELINE",
-          "className" : "org.apache.pinot.thirdeye.detection.wrapper.BaselineFillingMergeWrapper",
-          "maxGap" : 0,
-          "nested" : [ {
-            "bucketPeriod":"P1D",
-            "className" : "org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper"
-          } ],
-          "detector" : "$maxThreshold_2:THRESHOLD",
-          "maxDuration" : 100
-        } ]
-      } ],
-      "minContribution" : 0.05,
-      "dimensions" : [ "D1", "D2" ]
-    } ],
-    "maxDuration" : 100
+        "filter": "$thresholdFilter_3:THRESHOLD_RULE_FILTER",
+        "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyFilterWrapper",
+        "nested": [{
+          "baselineValueProvider": "$maxThreshold_2:THRESHOLD",
+          "className": "org.apache.pinot.thirdeye.detection.wrapper.BaselineFillingMergeWrapper",
+          "maxGap": 0,
+          "nested": [{
+            "bucketPeriod": "P1D",
+            "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper"
+          }],
+          "detector": "$maxThreshold_2:THRESHOLD",
+          "maxDuration": 100
+        }]
+      }],
+      "minContribution": 0.05,
+      "dimensions": ["D1", "D2"]
+    }],
+    "maxDuration": 100
   },
-  "components" : {
-    "maxThreshold_2:THRESHOLD" : {
-      "max" : 100,
-      "className" : "org.apache.pinot.thirdeye.detection.components.ThresholdRuleDetector"
+  "components": {
+    "maxThreshold_2:THRESHOLD": {
+      "max": 100,
+      "className": "org.apache.pinot.thirdeye.detection.components.ThresholdRuleDetector"
     },
-    "maxThreshold_1:THRESHOLD" : {
-      "max" : 100,
-      "className" : "org.apache.pinot.thirdeye.detection.components.ThresholdRuleDetector"
+    "thresholdFilter_2:THRESHOLD_RULE_FILTER": {
+      "min": 50,
+      "className": "org.apache.pinot.thirdeye.detection.components.ThresholdRuleAnomalyFilter"
     },
-    "maxThreshold_2:RULE_BASELINE" : {
-      "max" : 100,
-      "className" : "org.apache.pinot.thirdeye.detection.components.RuleBaselineProvider"
+    "thresholdFilter_1:THRESHOLD_RULE_FILTER": {
+      "min": 50,
+      "className": "org.apache.pinot.thirdeye.detection.components.ThresholdRuleAnomalyFilter"
     },
-    "thresholdFilter_1:THRESHOLD_RULE_FILTER" : {
-      "min" : 50,
-      "className" : "org.apache.pinot.thirdeye.detection.components.ThresholdRuleAnomalyFilter"
+    "maxThreshold_1:THRESHOLD": {
+      "max": 100,
+      "className": "org.apache.pinot.thirdeye.detection.components.ThresholdRuleDetector"
     },
-    "thresholdFilter_2:THRESHOLD_RULE_FILTER" : {
-      "min" : 50,
-      "className" : "org.apache.pinot.thirdeye.detection.components.ThresholdRuleAnomalyFilter"
-    },
-    "thresholdFilter_3:THRESHOLD_RULE_FILTER" : {
-      "min" : 50,
-      "className" : "org.apache.pinot.thirdeye.detection.components.ThresholdRuleAnomalyFilter"
-    },
-    "maxThreshold_1:RULE_BASELINE" : {
-      "max" : 100,
-      "className" : "org.apache.pinot.thirdeye.detection.components.RuleBaselineProvider"
+    "thresholdFilter_3:THRESHOLD_RULE_FILTER": {
+      "min": 50,
+      "className": "org.apache.pinot.thirdeye.detection.components.ThresholdRuleAnomalyFilter"
     }
   },
-  "cron" : "0 0 14 * * ? *"
-}
+  "cron": "0 0 14 * * ? *"
+}
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/compositePipelineTranslatorTestResult-2.json b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/compositePipelineTranslatorTestResult-2.json
index 67292a5..06a93fd 100644
--- a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/compositePipelineTranslatorTestResult-2.json
+++ b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/compositePipelineTranslatorTestResult-2.json
@@ -1,32 +1,28 @@
 {
-  "properties" : {
-    "className" : "org.apache.pinot.thirdeye.detection.wrapper.ChildKeepingMergeWrapper",
-    "nested" : [ {
-      "nestedMetricUrns" : [ "thirdeye:metric:1:D1%3Dv1:D1%3Dv2:D2%3Dv3" ],
-      "className" : "org.apache.pinot.thirdeye.detection.algorithm.DimensionWrapper",
-      "metricUrn" : "thirdeye:metric:1:D1%3Dv1:D1%3Dv2:D2%3Dv3",
-      "nested" : [ {
-        "baselineValueProvider" : "$rule1:RULE_BASELINE",
-        "className" : "org.apache.pinot.thirdeye.detection.wrapper.BaselineFillingMergeWrapper",
-        "nested" : [ {
-          "bucketPeriod":"P1D",
-          "className" : "org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper"
-        } ],
-        "detector" : "$rule1:THRESHOLD"
-      } ],
-      "minContribution" : 0.05,
-      "dimensions" : [ "D1", "D2" ]
-    } ]
+  "properties": {
+    "className": "org.apache.pinot.thirdeye.detection.wrapper.ChildKeepingMergeWrapper",
+    "nested": [{
+      "nestedMetricUrns": ["thirdeye:metric:1:D1%3Dv1:D1%3Dv2:D2%3Dv3"],
+      "className": "org.apache.pinot.thirdeye.detection.algorithm.DimensionWrapper",
+      "metricUrn": "thirdeye:metric:1:D1%3Dv1:D1%3Dv2:D2%3Dv3",
+      "nested": [{
+        "baselineValueProvider": "$rule1:THRESHOLD",
+        "className": "org.apache.pinot.thirdeye.detection.wrapper.BaselineFillingMergeWrapper",
+        "nested": [{
+          "bucketPeriod": "P1D",
+          "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper"
+        }],
+        "detector": "$rule1:THRESHOLD"
+      }],
+      "minContribution": 0.05,
+      "dimensions": ["D1", "D2"]
+    }]
   },
-  "components" : {
-    "rule1:THRESHOLD" : {
-      "max" : 100,
-      "className" : "org.apache.pinot.thirdeye.detection.components.ThresholdRuleDetector"
-    },
-    "rule1:RULE_BASELINE" : {
-      "max" : 100,
-      "className" : "org.apache.pinot.thirdeye.detection.components.RuleBaselineProvider"
+  "components": {
+    "rule1:THRESHOLD": {
+      "max": 100,
+      "className": "org.apache.pinot.thirdeye.detection.components.ThresholdRuleDetector"
     }
   },
-  "cron" : "0 0 14 * * ? *"
-}
+  "cron": "0 0 14 * * ? *"
+}
\ 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