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

[GitHub] jihaozh closed pull request #3467: [TE] Percentage change and absolute change rule detector

jihaozh closed pull request #3467: [TE] Percentage change and absolute change rule detector
URL: https://github.com/apache/incubator-pinot/pull/3467
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/components/AbsoluteChangeRuleDetector.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/components/AbsoluteChangeRuleDetector.java
new file mode 100644
index 0000000000..a8008b8f27
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/components/AbsoluteChangeRuleDetector.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
+ *
+ * Licensed 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 com.linkedin.thirdeye.detection.components;
+
+import com.linkedin.thirdeye.dashboard.resources.v2.BaselineParsingUtils;
+import com.linkedin.thirdeye.dataframe.BooleanSeries;
+import com.linkedin.thirdeye.dataframe.DataFrame;
+import com.linkedin.thirdeye.dataframe.util.MetricSlice;
+import com.linkedin.thirdeye.datalayer.dto.DatasetConfigDTO;
+import com.linkedin.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import com.linkedin.thirdeye.detection.DetectionUtils;
+import com.linkedin.thirdeye.detection.InputDataFetcher;
+import com.linkedin.thirdeye.detection.annotation.Components;
+import com.linkedin.thirdeye.detection.annotation.DetectionTag;
+import com.linkedin.thirdeye.detection.annotation.Param;
+import com.linkedin.thirdeye.detection.annotation.PresentationOption;
+import com.linkedin.thirdeye.detection.spec.AbsoluteChangeRuleDetectorSpec;
+import com.linkedin.thirdeye.detection.spi.components.AnomalyDetector;
+import com.linkedin.thirdeye.detection.spi.model.InputData;
+import com.linkedin.thirdeye.detection.spi.model.InputDataSpec;
+import com.linkedin.thirdeye.rootcause.impl.MetricEntity;
+import com.linkedin.thirdeye.rootcause.timeseries.Baseline;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.joda.time.Interval;
+
+import static com.linkedin.thirdeye.dataframe.util.DataFrameUtils.*;
+
+
+@Components(title = "Absolute change rule detection",
+    type = "ABSOLUTE_CHANGE_RULE",
+    tags = {DetectionTag.RULE_DETECTION},
+    presentation = {
+        @PresentationOption(name = "absolute value", template = "comparing ${offset} is more than ${difference}"),
+    },
+    params = {
+        @Param(name = "offset", defaultValue = "wo1w"),
+        @Param(name = "change", placeholder = "value")
+    })
+public class AbsoluteChangeRuleDetector implements AnomalyDetector<AbsoluteChangeRuleDetectorSpec> {
+  private double absoluteChange;
+  private InputDataFetcher dataFetcher;
+  private Baseline baseline;
+  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_DIFF = "diff";
+
+  @Override
+  public List<MergedAnomalyResultDTO> runDetection(Interval window, String metricUrn) {
+    MetricEntity me = MetricEntity.fromURN(metricUrn);
+    MetricSlice slice = MetricSlice.from(me.getId(), window.getStartMillis(), window.getEndMillis(), me.getFilters());
+    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 df = new DataFrame(dfCurr).addSeries(dfBase);
+    // calculate absolute change
+    df.addSeries(COL_DIFF, df.getDoubles(COL_CURR).subtract(df.get(COL_BASE)));
+
+    // defaults
+    df.addSeries(COL_ANOMALY, BooleanSeries.fillValues(df.size(), false));
+
+    // absolute change
+    if (!Double.isNaN(this.absoluteChange)) {
+      df.addSeries(COL_ANOMALY, df.getDoubles(COL_DIFF).abs().gte(this.absoluteChange));
+    }
+
+    // make anomalies
+    DatasetConfigDTO datasetConfig = data.getDatasetForMetricId().get(me.getId());
+    return DetectionUtils.makeAnomalies(slice, df, COL_ANOMALY, window.getEndMillis(), datasetConfig);
+  }
+
+  @Override
+  public void init(AbsoluteChangeRuleDetectorSpec spec, InputDataFetcher dataFetcher) {
+    this.absoluteChange = spec.getAbsoluteChangeChange();
+    this.dataFetcher = dataFetcher;
+    String timezone = spec.getTimezone();
+    String offset = spec.getOffset();
+    this.baseline = BaselineParsingUtils.parseOffset(offset, timezone);
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/components/PercentageChangeRuleDetector.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/components/PercentageChangeRuleDetector.java
new file mode 100644
index 0000000000..1814e6c6be
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/components/PercentageChangeRuleDetector.java
@@ -0,0 +1,108 @@
+/*
+ * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
+ *
+ * Licensed 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 com.linkedin.thirdeye.detection.components;
+
+import com.linkedin.thirdeye.dashboard.resources.v2.BaselineParsingUtils;
+import com.linkedin.thirdeye.dataframe.BooleanSeries;
+import com.linkedin.thirdeye.dataframe.DataFrame;
+import com.linkedin.thirdeye.dataframe.Series;
+import com.linkedin.thirdeye.dataframe.util.MetricSlice;
+import com.linkedin.thirdeye.datalayer.dto.DatasetConfigDTO;
+import com.linkedin.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import com.linkedin.thirdeye.detection.DetectionUtils;
+import com.linkedin.thirdeye.detection.InputDataFetcher;
+import com.linkedin.thirdeye.detection.annotation.Components;
+import com.linkedin.thirdeye.detection.annotation.DetectionTag;
+import com.linkedin.thirdeye.detection.annotation.Param;
+import com.linkedin.thirdeye.detection.annotation.PresentationOption;
+import com.linkedin.thirdeye.detection.spec.PercentageChangeRuleDetectorSpec;
+import com.linkedin.thirdeye.detection.spi.components.AnomalyDetector;
+import com.linkedin.thirdeye.detection.spi.model.InputData;
+import com.linkedin.thirdeye.detection.spi.model.InputDataSpec;
+import com.linkedin.thirdeye.rootcause.impl.MetricEntity;
+import com.linkedin.thirdeye.rootcause.timeseries.Baseline;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.joda.time.Interval;
+
+import static com.linkedin.thirdeye.dataframe.DoubleSeries.*;
+import static com.linkedin.thirdeye.dataframe.util.DataFrameUtils.*;
+
+@Components(title = "Percentage change rule detection",
+    type = "PERCENTAGE_RULE",
+    tags = {DetectionTag.RULE_DETECTION},
+    description = "Computes a multi-week aggregate baseline and compares the current value "
+        + "based on relative change.",
+    presentation = {
+        @PresentationOption(name = "percentage change", template = "comparing ${offset} is more than ${change}")
+    },
+    params = {
+        @Param(name = "offset", defaultValue = "wo1w"),
+        @Param(name = "change", placeholder = "value")
+    })
+public class PercentageChangeRuleDetector implements AnomalyDetector<PercentageChangeRuleDetectorSpec> {
+  private double percentageChange;
+  private InputDataFetcher dataFetcher;
+  private Baseline baseline;
+  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";
+
+  @Override
+  public List<MergedAnomalyResultDTO> runDetection(Interval window, String metricUrn) {
+    MetricEntity me = MetricEntity.fromURN(metricUrn);
+    MetricSlice slice = MetricSlice.from(me.getId(), window.getStartMillis(), window.getEndMillis(), me.getFilters());
+    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 df = new DataFrame(dfCurr).addSeries(dfBase);
+
+    // calculate percentage change
+    df.addSeries(COL_CHANGE, map((Series.DoubleFunction) values -> {
+      if (values[1] == 0) {
+        return values[0] > 0 ? Double.POSITIVE_INFINITY : Double.NEGATIVE_INFINITY;
+      }
+      return values[0] / values[1];
+    }, df.getDoubles(COL_CURR), df.get(COL_BASE)).subtract(1));
+
+    // defaults
+    df.addSeries(COL_ANOMALY, BooleanSeries.fillValues(df.size(), false));
+
+    // relative change
+    if (!Double.isNaN(this.percentageChange)) {
+      df.addSeries(COL_ANOMALY, df.getDoubles(COL_CHANGE).abs().gte(this.percentageChange));
+    }
+
+    // anomalies
+    DatasetConfigDTO datasetConfig = data.getDatasetForMetricId().get(me.getId());
+    return DetectionUtils.makeAnomalies(slice, df, COL_ANOMALY, window.getEndMillis(), datasetConfig);
+  }
+
+  @Override
+  public void init(PercentageChangeRuleDetectorSpec spec, InputDataFetcher dataFetcher) {
+    this.percentageChange = spec.getPercentageChange();
+    this.dataFetcher = dataFetcher;
+    String timezone = spec.getTimezone();
+    String offset = spec.getOffset();
+    this.baseline = BaselineParsingUtils.parseOffset(offset, timezone);
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/spec/AbsoluteChangeRuleDetectorSpec.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/spec/AbsoluteChangeRuleDetectorSpec.java
new file mode 100644
index 0000000000..c69659ea85
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/spec/AbsoluteChangeRuleDetectorSpec.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
+ *
+ * Licensed 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 com.linkedin.thirdeye.detection.spec;
+
+public class AbsoluteChangeRuleDetectorSpec extends AbstractSpec {
+  private double absoluteChange = Double.NaN;
+  private String offset = "wo1w";
+  private String timezone = "UTC";
+
+  public String getTimezone() {
+    return timezone;
+  }
+
+  public void setTimezone(String timezone) {
+    this.timezone = timezone;
+  }
+
+  public String getOffset() {
+    return offset;
+  }
+
+  public void setOffset(String offset) {
+    this.offset = offset;
+  }
+
+  public double getAbsoluteChangeChange() {
+    return absoluteChange;
+  }
+
+  public void setAbsoluteChangeChange(double absoluteChange) {
+    this.absoluteChange = absoluteChange;
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/spec/PercentageChangeRuleDetectorSpec.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/spec/PercentageChangeRuleDetectorSpec.java
new file mode 100644
index 0000000000..c1e78fb9d6
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/spec/PercentageChangeRuleDetectorSpec.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
+ *
+ * Licensed 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 com.linkedin.thirdeye.detection.spec;
+
+public class PercentageChangeRuleDetectorSpec extends AbstractSpec {
+  private double percentageChange = Double.NaN;
+  private String offset = "wo1w";
+  private String timezone = "UTC";
+
+  public String getTimezone() {
+    return timezone;
+  }
+
+  public void setTimezone(String timezone) {
+    this.timezone = timezone;
+  }
+
+  public String getOffset() {
+    return offset;
+  }
+
+  public void setOffset(String offset) {
+    this.offset = offset;
+  }
+
+  public double getPercentageChange() {
+    return percentageChange;
+  }
+
+  public void setPercentageChange(double percentageChange) {
+    this.percentageChange = percentageChange;
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/AbsoluteChangeRuleDetectorTest.java b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/AbsoluteChangeRuleDetectorTest.java
new file mode 100644
index 0000000000..fc73e8d1d8
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/AbsoluteChangeRuleDetectorTest.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
+ *
+ * Licensed 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 com.linkedin.thirdeye.detection.components;
+
+import com.linkedin.thirdeye.dataframe.DataFrame;
+import com.linkedin.thirdeye.dataframe.util.MetricSlice;
+import com.linkedin.thirdeye.datalayer.dto.DatasetConfigDTO;
+import com.linkedin.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import com.linkedin.thirdeye.datalayer.dto.MetricConfigDTO;
+import com.linkedin.thirdeye.detection.DataProvider;
+import com.linkedin.thirdeye.detection.DefaultInputDataFetcher;
+import com.linkedin.thirdeye.detection.InputDataFetcher;
+import com.linkedin.thirdeye.detection.MockDataProvider;
+import com.linkedin.thirdeye.detection.algorithm.AlgorithmUtils;
+import com.linkedin.thirdeye.detection.spec.AbsoluteChangeRuleDetectorSpec;
+import com.linkedin.thirdeye.detection.spec.PercentageChangeRuleDetectorSpec;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.joda.time.Interval;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import static com.linkedin.thirdeye.dataframe.util.DataFrameUtils.*;
+
+
+public class AbsoluteChangeRuleDetectorTest {
+  private DataProvider provider;
+  private DataFrame data;
+
+  @BeforeMethod
+  public void beforeMethod() throws Exception {
+    try (Reader dataReader = new InputStreamReader(AlgorithmUtils.class.getResourceAsStream("timeseries-4w.csv"))) {
+      this.data = DataFrame.fromCsv(dataReader);
+      this.data.setIndex(COL_TIME);
+      this.data.addSeries(COL_TIME, this.data.getLongs(COL_TIME).multiply(1000));
+    }
+
+    MetricConfigDTO metricConfigDTO = new MetricConfigDTO();
+    metricConfigDTO.setId(1L);
+    metricConfigDTO.setName("thirdeye-test");
+    metricConfigDTO.setDataset("thirdeye-test-dataset");
+
+    DatasetConfigDTO datasetConfigDTO = new DatasetConfigDTO();
+    datasetConfigDTO.setTimeUnit(TimeUnit.HOURS);
+    datasetConfigDTO.setDataset("thirdeye-test-dataset");
+    datasetConfigDTO.setTimeDuration(1);
+
+    Map<MetricSlice, DataFrame> timeseries = new HashMap<>();
+    timeseries.put(MetricSlice.from(1L, 0L, 604800000L), this.data);
+    timeseries.put(MetricSlice.from(1L, 604800000L, 1209600000L), this.data);
+    timeseries.put(MetricSlice.from(1L, 1209600000L, 1814400000L), this.data);
+    timeseries.put(MetricSlice.from(1L, 1814400000L, 2419200000L), this.data);
+
+    this.provider = new MockDataProvider()
+        .setTimeseries(timeseries)
+        .setMetrics(Collections.singletonList(metricConfigDTO))
+        .setDatasets(Collections.singletonList(datasetConfigDTO));;
+  }
+
+  @Test
+  public void testWeekOverWeekDifference() throws Exception {
+    AbsoluteChangeRuleDetector detector = new AbsoluteChangeRuleDetector();
+    AbsoluteChangeRuleDetectorSpec spec = new AbsoluteChangeRuleDetectorSpec();
+    spec.setAbsoluteChangeChange(400);
+    detector.init(spec, new DefaultInputDataFetcher(this.provider, -1));
+    List<MergedAnomalyResultDTO> anomalies = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+
+    Assert.assertEquals(anomalies.size(), 1);
+    Assert.assertEquals(anomalies.get(0).getStartTime(), 2372400000L);
+    Assert.assertEquals(anomalies.get(0).getEndTime(), 2376000000L);
+  }
+
+}
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/PercentageChangeRuleDetectorTest.java b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/PercentageChangeRuleDetectorTest.java
new file mode 100644
index 0000000000..b1e878c7da
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/PercentageChangeRuleDetectorTest.java
@@ -0,0 +1,114 @@
+/*
+ * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
+ *
+ * Licensed 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 com.linkedin.thirdeye.detection.components;
+
+import com.linkedin.thirdeye.dataframe.DataFrame;
+import com.linkedin.thirdeye.dataframe.util.MetricSlice;
+import com.linkedin.thirdeye.datalayer.dto.DatasetConfigDTO;
+import com.linkedin.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import com.linkedin.thirdeye.datalayer.dto.MetricConfigDTO;
+import com.linkedin.thirdeye.detection.DataProvider;
+import com.linkedin.thirdeye.detection.DefaultInputDataFetcher;
+import com.linkedin.thirdeye.detection.InputDataFetcher;
+import com.linkedin.thirdeye.detection.MockDataProvider;
+import com.linkedin.thirdeye.detection.algorithm.AlgorithmUtils;
+import com.linkedin.thirdeye.detection.spec.PercentageChangeRuleDetectorSpec;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.joda.time.Interval;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import static com.linkedin.thirdeye.dataframe.util.DataFrameUtils.*;
+
+
+public class PercentageChangeRuleDetectorTest {
+
+  private DataProvider provider;
+  private DataFrame data;
+
+  @BeforeMethod
+  public void beforeMethod() throws Exception {
+    try (Reader dataReader = new InputStreamReader(AlgorithmUtils.class.getResourceAsStream("timeseries-4w.csv"))) {
+      this.data = DataFrame.fromCsv(dataReader);
+      this.data.setIndex(COL_TIME);
+      this.data.addSeries(COL_TIME, this.data.getLongs(COL_TIME).multiply(1000));
+    }
+
+    MetricConfigDTO metricConfigDTO = new MetricConfigDTO();
+    metricConfigDTO.setId(1L);
+    metricConfigDTO.setName("thirdeye-test");
+    metricConfigDTO.setDataset("thirdeye-test-dataset");
+
+    DatasetConfigDTO datasetConfigDTO = new DatasetConfigDTO();
+    datasetConfigDTO.setTimeUnit(TimeUnit.HOURS);
+    datasetConfigDTO.setDataset("thirdeye-test-dataset");
+    datasetConfigDTO.setTimeDuration(1);
+
+    Map<MetricSlice, DataFrame> timeseries = new HashMap<>();
+    timeseries.put(MetricSlice.from(1L, 0L, 604800000L), this.data);
+    timeseries.put(MetricSlice.from(1L, 604800000L, 1209600000L), this.data);
+    timeseries.put(MetricSlice.from(1L, 1209600000L, 1814400000L), this.data);
+    timeseries.put(MetricSlice.from(1L, 1814400000L, 2419200000L), this.data);
+
+    this.provider = new MockDataProvider()
+        .setTimeseries(timeseries)
+        .setMetrics(Collections.singletonList(metricConfigDTO))
+        .setDatasets(Collections.singletonList(datasetConfigDTO));
+  }
+
+  @Test
+  public void testWeekOverWeekChange() throws Exception {
+    PercentageChangeRuleDetector detector = new PercentageChangeRuleDetector();
+    PercentageChangeRuleDetectorSpec spec = new PercentageChangeRuleDetectorSpec();
+    spec.setPercentageChange(0.4);
+    detector.init(spec, new DefaultInputDataFetcher(this.provider, -1));
+    List<MergedAnomalyResultDTO> anomalies = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+    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);
+  }
+
+  @Test
+  public void testThreeWeekMedianChange() throws Exception {
+    PercentageChangeRuleDetector detector = new PercentageChangeRuleDetector();
+    PercentageChangeRuleDetectorSpec spec = new PercentageChangeRuleDetectorSpec();
+    spec.setPercentageChange(0.3);
+    spec.setOffset("median3w");
+    detector.init(spec, new DefaultInputDataFetcher(this.provider, -1));
+    List<MergedAnomalyResultDTO> anomalies = detector.runDetection(new Interval(1814400000L, 2419200000L), "thirdeye:metric:1");
+    Assert.assertEquals(anomalies.size(), 5);
+    Assert.assertEquals(anomalies.get(0).getStartTime(), 2005200000L);
+    Assert.assertEquals(anomalies.get(0).getEndTime(), 2008800000L);
+    Assert.assertEquals(anomalies.get(1).getStartTime(), 2134800000L);
+    Assert.assertEquals(anomalies.get(1).getEndTime(), 2138400000L);
+    Assert.assertEquals(anomalies.get(2).getStartTime(), 2152800000L);
+    Assert.assertEquals(anomalies.get(2).getEndTime(), 2156400000L);
+    Assert.assertEquals(anomalies.get(3).getStartTime(), 2181600000L);
+    Assert.assertEquals(anomalies.get(3).getEndTime(), 2185200000L);
+    Assert.assertEquals(anomalies.get(4).getStartTime(), 2322000000L);
+    Assert.assertEquals(anomalies.get(4).getEndTime(), 2325600000L);
+  }
+}
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/spec/AbstractSpecTest.java b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/spec/AbstractSpecTest.java
index 70d3aa9272..ad871f4e7c 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/spec/AbstractSpecTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/spec/AbstractSpecTest.java
@@ -24,7 +24,7 @@
 
 public class AbstractSpecTest {
   @Test
-  public void testAbstractSpecMapping() {
+  public void testAbstractSpecMappingDefaultValue() {
     TestSpec spec = AbstractSpec.fromProperties(ImmutableMap.of(), TestSpec.class);
     Assert.assertEquals(spec.getA(), 123);
     Assert.assertEquals(spec.getB(), 456.7);
@@ -32,7 +32,7 @@ public void testAbstractSpecMapping() {
   }
 
   @Test
-  public void testAbstractSpecMapping1() {
+  public void testAbstractSpecMappingIncompleteProperty() {
     TestSpec spec = AbstractSpec.fromProperties(ImmutableMap.of("a", 321), TestSpec.class);
     Assert.assertEquals(spec.getA(), 321);
     Assert.assertEquals(spec.getB(), 456.7);
@@ -40,7 +40,7 @@ public void testAbstractSpecMapping1() {
   }
 
   @Test
-  public void testAbstractSpecMapping2() {
+  public void testAbstractSpecMappingNestedObject() {
     RuleBaselineProvider provider = new RuleBaselineProvider();
     TestSpec spec = AbstractSpec.fromProperties(ImmutableMap.of("baselineProvider", provider), TestSpec.class);
     Assert.assertEquals(spec.getA(), 123);
@@ -50,12 +50,20 @@ public void testAbstractSpecMapping2() {
   }
 
   @Test
-  public void testAbstractSpecMapping3() {
+  public void testAbstractSpecMappingExtraField() {
     TestSpec spec = AbstractSpec.fromProperties(ImmutableMap.of("a", 321, "className", "org.test.Test"), TestSpec.class);
     Assert.assertEquals(spec.getA(), 321);
     Assert.assertEquals(spec.getB(), 456.7);
     Assert.assertEquals(spec.getC(), "default");
   }
 
+  @Test
+  public void testAbstractSpecMappingNestedMap() {
+    TestSpec spec = AbstractSpec.fromProperties(ImmutableMap.of("a", 321, "className", "org.test.Test", "configuration", ImmutableMap.of("k1", "v1", "k2", "v2")), TestSpec.class);
+    Assert.assertEquals(spec.getA(), 321);
+    Assert.assertEquals(spec.getB(), 456.7);
+    Assert.assertEquals(spec.getC(), "default");
+    Assert.assertEquals(spec.getConfiguration(), ImmutableMap.of("k1", "v1", "k2", "v2"));
+  }
 }
 
diff --git a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/spec/TestSpec.java b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/spec/TestSpec.java
index f6ed4b7c55..29990b0be2 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/spec/TestSpec.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/spec/TestSpec.java
@@ -17,6 +17,7 @@
 package com.linkedin.thirdeye.detection.spec;
 
 import com.linkedin.thirdeye.detection.components.RuleBaselineProvider;
+import java.util.Map;
 
 
 public class TestSpec extends AbstractSpec{
@@ -24,6 +25,15 @@
   private double b = 456.7;
   private String c = "default";
   private RuleBaselineProvider baselineProvider;
+  private Map<String, String> configuration;
+
+  public Map<String, String> getConfiguration() {
+    return configuration;
+  }
+
+  public void setConfiguration(Map<String, String> configuration) {
+    this.configuration = configuration;
+  }
 
   public RuleBaselineProvider getBaselineProvider() {
     return baselineProvider;


 

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


With regards,
Apache Git Services

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