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 18:35:53 UTC

[GitHub] jihaozh closed pull request #3468: [TE] detection - Input data fetcher interface

jihaozh closed pull request #3468: [TE] detection - Input data fetcher interface
URL: https://github.com/apache/incubator-pinot/pull/3468
 
 
   

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/DefaultInputDataFetcher.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/DefaultInputDataFetcher.java
new file mode 100644
index 0000000000..5e3ce05ae1
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/DefaultInputDataFetcher.java
@@ -0,0 +1,79 @@
+/*
+ * 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;
+
+import com.google.common.collect.Multimap;
+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.EventDTO;
+import com.linkedin.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import com.linkedin.thirdeye.datalayer.dto.MetricConfigDTO;
+import com.linkedin.thirdeye.detection.spi.model.AnomalySlice;
+import com.linkedin.thirdeye.detection.spi.model.EventSlice;
+import com.linkedin.thirdeye.detection.spi.model.InputData;
+import com.linkedin.thirdeye.detection.spi.model.InputDataSpec;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * Input data fetcher.
+ * For components to fetch the input data it need.
+ */
+public class DefaultInputDataFetcher implements InputDataFetcher {
+  private DataProvider provider;
+  private long configId;
+
+  public DefaultInputDataFetcher(DataProvider provider, long configId) {
+    this.provider = provider;
+    this.configId = configId;
+  }
+
+  /**
+   * Fetch data for input data spec
+   */
+  public InputData fetchData(InputDataSpec inputDataSpec) {
+    Map<MetricSlice, DataFrame> timeseries = provider.fetchTimeseries(inputDataSpec.getTimeseriesSlices());
+    Map<MetricSlice, DataFrame> aggregates =
+        provider.fetchAggregates(inputDataSpec.getAggregateSlices(), Collections.<String>emptyList());
+    Multimap<AnomalySlice, MergedAnomalyResultDTO> existingAnomalies =
+        provider.fetchAnomalies(inputDataSpec.getAnomalySlices(), configId);
+    Multimap<EventSlice, EventDTO> events = provider.fetchEvents(inputDataSpec.getEventSlices());
+    Map<Long, MetricConfigDTO> metrics = provider.fetchMetrics(inputDataSpec.getMetricIds());
+    Map<String, DatasetConfigDTO> datasets = provider.fetchDatasets(inputDataSpec.getDatasetNames());
+
+    Map<Long, DatasetConfigDTO> datasetForMetricId = fetchDatasetForMetricId(provider, inputDataSpec);
+    return new InputData(inputDataSpec, timeseries, aggregates, existingAnomalies, events, metrics, datasets, datasetForMetricId);
+  }
+
+  private Map<Long, DatasetConfigDTO> fetchDatasetForMetricId(DataProvider provider, InputDataSpec inputDataSpec) {
+    Map<Long, MetricConfigDTO> metrics = provider.fetchMetrics(inputDataSpec.getMetricIdsForDatasets());
+    Map<Long, String> metricIdToDataSet = new HashMap<>();
+    for (Map.Entry<Long, MetricConfigDTO> entry : metrics.entrySet()){
+      metricIdToDataSet.put(entry.getKey(), entry.getValue().getDataset());
+    }
+    Map<String, DatasetConfigDTO> datasets = provider.fetchDatasets(metricIdToDataSet.values());
+    Map<Long, DatasetConfigDTO> result = new HashMap<>();
+    for (Map.Entry<Long, MetricConfigDTO> entry : metrics.entrySet()){
+      result.put(entry.getKey(), datasets.get(entry.getValue().getDataset()));
+    }
+    return result;
+  }
+
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/DetectionPipeline.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/DetectionPipeline.java
index 36c40605f2..776b427a2c 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/DetectionPipeline.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/DetectionPipeline.java
@@ -84,7 +84,7 @@ protected DetectionPipeline(DataProvider provider, DetectionConfigDTO config, lo
    * @throws Exception
    */
   private void initComponents() throws Exception {
-    InputDataFetcher dataFetcher = new InputDataFetcher(this.provider, this.config.getId());
+    InputDataFetcher dataFetcher = new DefaultInputDataFetcher(this.provider, this.config.getId());
     Map<String, BaseComponent> instancesMap = config.getComponents();
     Map<String, Object> componentSpecs = config.getComponentSpecs();
     if (componentSpecs != null) {
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/InputDataFetcher.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/InputDataFetcher.java
index 10c2e1460a..b5ac8cdb91 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/InputDataFetcher.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/InputDataFetcher.java
@@ -16,64 +16,17 @@
 
 package com.linkedin.thirdeye.detection;
 
-import com.google.common.collect.Multimap;
-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.EventDTO;
-import com.linkedin.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
-import com.linkedin.thirdeye.datalayer.dto.MetricConfigDTO;
-import com.linkedin.thirdeye.detection.spi.model.AnomalySlice;
-import com.linkedin.thirdeye.detection.spi.model.EventSlice;
 import com.linkedin.thirdeye.detection.spi.model.InputData;
 import com.linkedin.thirdeye.detection.spi.model.InputDataSpec;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
 
 /**
- * Input data fetcher.
+ * Input data fetcher interface.
  * For components to fetch the input data it need.
  */
-public class InputDataFetcher {
-  private DataProvider provider;
-  private long configId;
-
-  public InputDataFetcher(DataProvider provider, long configId) {
-    this.provider = provider;
-    this.configId = configId;
-  }
 
+public interface InputDataFetcher {
   /**
-   * Fetch data for input data spec
+   * fetch data for input data spec
    */
-  public InputData fetchData(InputDataSpec inputDataSpec) {
-    Map<MetricSlice, DataFrame> timeseries = provider.fetchTimeseries(inputDataSpec.getTimeseriesSlices());
-    Map<MetricSlice, DataFrame> aggregates =
-        provider.fetchAggregates(inputDataSpec.getAggregateSlices(), Collections.<String>emptyList());
-    Multimap<AnomalySlice, MergedAnomalyResultDTO> existingAnomalies =
-        provider.fetchAnomalies(inputDataSpec.getAnomalySlices(), configId);
-    Multimap<EventSlice, EventDTO> events = provider.fetchEvents(inputDataSpec.getEventSlices());
-    Map<Long, MetricConfigDTO> metrics = provider.fetchMetrics(inputDataSpec.getMetricIds());
-    Map<String, DatasetConfigDTO> datasets = provider.fetchDatasets(inputDataSpec.getDatasetNames());
-
-    Map<Long, DatasetConfigDTO> datasetForMetricId = fetchDatasetForMetricId(provider, inputDataSpec);
-    return new InputData(inputDataSpec, timeseries, aggregates, existingAnomalies, events, metrics, datasets, datasetForMetricId);
-  }
-
-  private Map<Long, DatasetConfigDTO> fetchDatasetForMetricId(DataProvider provider, InputDataSpec inputDataSpec) {
-    Map<Long, MetricConfigDTO> metrics = provider.fetchMetrics(inputDataSpec.getMetricIdsForDatasets());
-    Map<Long, String> metricIdToDataSet = new HashMap<>();
-    for (Map.Entry<Long, MetricConfigDTO> entry : metrics.entrySet()){
-      metricIdToDataSet.put(entry.getKey(), entry.getValue().getDataset());
-    }
-    Map<String, DatasetConfigDTO> datasets = provider.fetchDatasets(metricIdToDataSet.values());
-    Map<Long, DatasetConfigDTO> result = new HashMap<>();
-    for (Map.Entry<Long, MetricConfigDTO> entry : metrics.entrySet()){
-      result.put(entry.getKey(), datasets.get(entry.getValue().getDataset()));
-    }
-    return result;
-  }
-
+  InputData fetchData(InputDataSpec inputDataSpec);
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/algorithm/stage/StaticAnomalyDetectionStage.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/algorithm/stage/StaticAnomalyDetectionStage.java
index 952115cec3..e69f42019d 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/algorithm/stage/StaticAnomalyDetectionStage.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/algorithm/stage/StaticAnomalyDetectionStage.java
@@ -24,6 +24,7 @@
 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.spi.model.InputData;
 import com.linkedin.thirdeye.detection.spi.model.InputDataSpec;
@@ -67,7 +68,7 @@ public void init(Map<String, Object> specs, Long configId, long startTime, long
   @Override
   public final List<MergedAnomalyResultDTO> runDetection(DataProvider provider) {
     this.provider = provider;
-    InputDataFetcher dataFetcher = new InputDataFetcher(this.provider, this.configId);
+    InputDataFetcher dataFetcher = new DefaultInputDataFetcher(this.provider, this.configId);
     return this.runDetection(dataFetcher.fetchData(this.getInputDataSpec()));
   }
 
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/algorithm/stage/StaticAnomalyFilterStage.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/algorithm/stage/StaticAnomalyFilterStage.java
index d350246f46..e72df2f577 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/algorithm/stage/StaticAnomalyFilterStage.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/algorithm/stage/StaticAnomalyFilterStage.java
@@ -18,6 +18,7 @@
 
 import com.linkedin.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import com.linkedin.thirdeye.detection.DataProvider;
+import com.linkedin.thirdeye.detection.DefaultInputDataFetcher;
 import com.linkedin.thirdeye.detection.InputDataFetcher;
 import com.linkedin.thirdeye.detection.spi.model.InputData;
 import com.linkedin.thirdeye.detection.spi.model.InputDataSpec;
@@ -52,7 +53,7 @@ public void init(Map<String, Object> specs, Long configId, long startTime, long
 
   @Override
   public final boolean isQualified(MergedAnomalyResultDTO anomaly, DataProvider provider) {
-    InputDataFetcher dataFetcher = new InputDataFetcher(provider, -1);
+    InputDataFetcher dataFetcher = new DefaultInputDataFetcher(provider, -1);
     return isQualified(anomaly, dataFetcher.fetchData(this.getInputDataSpec()));
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/components/ThresholdRuleDetector.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/components/ThresholdRuleDetector.java
index 6ca9f0cee1..0fc223051e 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/components/ThresholdRuleDetector.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/components/ThresholdRuleDetector.java
@@ -21,6 +21,7 @@
 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;
@@ -30,7 +31,6 @@
 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.detection.DetectionUtils;
 import com.linkedin.thirdeye.rootcause.impl.MetricEntity;
 import java.util.Collections;
 import java.util.List;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/wrapper/BaselineFillingMergeWrapper.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/wrapper/BaselineFillingMergeWrapper.java
index 7dff42e124..2547a99a85 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/wrapper/BaselineFillingMergeWrapper.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/wrapper/BaselineFillingMergeWrapper.java
@@ -23,6 +23,7 @@
 import com.linkedin.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import com.linkedin.thirdeye.detection.DataProvider;
 import com.linkedin.thirdeye.detection.DetectionUtils;
+import com.linkedin.thirdeye.detection.DefaultInputDataFetcher;
 import com.linkedin.thirdeye.detection.InputDataFetcher;
 import com.linkedin.thirdeye.detection.algorithm.MergeWrapper;
 import com.linkedin.thirdeye.detection.components.RuleBaselineProvider;
@@ -71,7 +72,7 @@ public BaselineFillingMergeWrapper(DataProvider provider, DetectionConfigDTO con
       this.currentValueProvider = new RuleBaselineProvider();
       RuleBaselineProviderSpec spec = new RuleBaselineProviderSpec();
       spec.setOffset("current");
-      InputDataFetcher dataFetcher = new InputDataFetcher(this.provider, this.config.getId());
+      InputDataFetcher dataFetcher = new DefaultInputDataFetcher(this.provider, this.config.getId());
       this.currentValueProvider.init(spec, dataFetcher);
     }
     String nestedUrn = MapUtils.getString(config.getProperties(), PROP_METRIC_URN);
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/yaml/CompositePipelineConfigTranslator.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/yaml/CompositePipelineConfigTranslator.java
index e913c60a02..8834199d34 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/yaml/CompositePipelineConfigTranslator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/yaml/CompositePipelineConfigTranslator.java
@@ -8,6 +8,7 @@
 import com.linkedin.thirdeye.datalayer.dto.MetricConfigDTO;
 import com.linkedin.thirdeye.detection.ConfigUtils;
 import com.linkedin.thirdeye.detection.DataProvider;
+import com.linkedin.thirdeye.detection.DefaultInputDataFetcher;
 import com.linkedin.thirdeye.detection.InputDataFetcher;
 import com.linkedin.thirdeye.detection.algorithm.DimensionWrapper;
 import com.linkedin.thirdeye.detection.annotation.DetectionRegistry;
@@ -299,7 +300,7 @@ private void buildComponentSpec(Map<String, Object> yamlConfig, String type, Str
   private Map<String, Object> getTunedSpecs(String componentName, String componentClassName, Map<String, Object> params)
       throws Exception {
     long configId = this.existingConfig == null ? -1 : this.existingConfig.getId();
-    InputDataFetcher dataFetcher = new InputDataFetcher(this.dataProvider, configId);
+    InputDataFetcher dataFetcher = new DefaultInputDataFetcher(this.dataProvider, configId);
     Tunable tunable = getTunable(componentClassName, params, dataFetcher);
     Interval window = new Interval(this.startTime, this.endTime, DateTimeZone.forID(this.datasetConfig.getTimezone()));
     Map<String, Object> existingComponentSpec =
diff --git a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/DefaultInputDataFetcherTest.java b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/DefaultInputDataFetcherTest.java
new file mode 100644
index 0000000000..2296a9137f
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/DefaultInputDataFetcherTest.java
@@ -0,0 +1,54 @@
+package com.linkedin.thirdeye.detection;
+
+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.MetricConfigDTO;
+import com.linkedin.thirdeye.detection.spi.model.InputData;
+import com.linkedin.thirdeye.detection.spi.model.InputDataSpec;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import static com.linkedin.thirdeye.dataframe.util.DataFrameUtils.*;
+
+
+public class DefaultInputDataFetcherTest {
+  @Test
+  public void testFetchData() {
+    MockDataProvider mockDataProvider = new MockDataProvider();
+    Map<MetricSlice, DataFrame> timeSeries = new HashMap<>();
+    MetricSlice slice = MetricSlice.from(123L, 0, 10);
+    timeSeries.put(slice,
+        new DataFrame().addSeries(COL_VALUE, 0, 100, 200, 500, 1000).addSeries(COL_TIME, 0, 2, 4, 6, 8));
+
+    MetricConfigDTO metricConfigDTO = new MetricConfigDTO();
+    metricConfigDTO.setId(123L);
+    metricConfigDTO.setName("thirdeye-test");
+    metricConfigDTO.setDataset("thirdeye-test-dataset");
+
+    DatasetConfigDTO datasetConfigDTO = new DatasetConfigDTO();
+    datasetConfigDTO.setId(124L);
+    datasetConfigDTO.setDataset("thirdeye-test-dataset");
+    datasetConfigDTO.setTimeDuration(2);
+    datasetConfigDTO.setTimeUnit(TimeUnit.MILLISECONDS);
+    datasetConfigDTO.setTimezone("UTC");
+
+    mockDataProvider.setTimeseries(timeSeries)
+        .setMetrics(Collections.singletonList(metricConfigDTO))
+        .setDatasets(Collections.singletonList(datasetConfigDTO));
+    InputDataFetcher dataFetcher = new DefaultInputDataFetcher(mockDataProvider, -1);
+
+    InputData data = dataFetcher.fetchData(new InputDataSpec().withTimeseriesSlices(Collections.singletonList(slice))
+        .withMetricIds(Collections.singletonList(123L))
+        .withMetricIdsForDataset(Collections.singletonList(123L))
+        .withDatasetNames(Collections.singletonList("thirdeye-test-dataset")));
+    Assert.assertEquals(data.getTimeseries().get(slice), timeSeries.get(slice));
+    Assert.assertEquals(data.getMetrics().get(123L), metricConfigDTO);
+    Assert.assertEquals(data.getDatasetForMetricId().get(123L), datasetConfigDTO);
+    Assert.assertEquals(data.getDatasets().get("thirdeye-test-dataset"), datasetConfigDTO);
+  }
+}
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/MockBaselineProvider.java b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/MockBaselineProvider.java
index 82ef50eda4..88a670e081 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/MockBaselineProvider.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/MockBaselineProvider.java
@@ -18,6 +18,7 @@
 
 import com.linkedin.thirdeye.dataframe.Series;
 import com.linkedin.thirdeye.dataframe.util.MetricSlice;
+import com.linkedin.thirdeye.detection.DefaultInputDataFetcher;
 import com.linkedin.thirdeye.detection.InputDataFetcher;
 import com.linkedin.thirdeye.detection.spec.MockBaselineProviderSpec;
 import com.linkedin.thirdeye.detection.spi.components.BaselineProvider;
diff --git a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/RuleBaselineProviderTest.java b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/RuleBaselineProviderTest.java
index 66dec05b75..ac191ac455 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/RuleBaselineProviderTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/components/RuleBaselineProviderTest.java
@@ -19,6 +19,7 @@
 import com.linkedin.thirdeye.dataframe.DataFrame;
 import com.linkedin.thirdeye.dataframe.DoubleSeries;
 import com.linkedin.thirdeye.dataframe.util.MetricSlice;
+import com.linkedin.thirdeye.detection.DefaultInputDataFetcher;
 import com.linkedin.thirdeye.detection.InputDataFetcher;
 import com.linkedin.thirdeye.detection.MockDataProvider;
 import com.linkedin.thirdeye.detection.spec.RuleBaselineProviderSpec;
@@ -54,7 +55,7 @@ public void setUp() {
     aggregates.put(slice2Wow, DataFrame.builder(COL_TIME + ":LONG", COL_VALUE + ":DOUBLE")
         .build()
         .setIndex(COL_TIME));
-    InputDataFetcher dataFetcher = new InputDataFetcher(dataProvider, -1);
+    InputDataFetcher dataFetcher = new DefaultInputDataFetcher(dataProvider, -1);
 
     baselineProvider.init(new RuleBaselineProviderSpec("UTC", "wo1w"), dataFetcher);
 
diff --git a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/wrapper/BaselineFillingMergeWrapperTest.java b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/wrapper/BaselineFillingMergeWrapperTest.java
index 8dd3c24682..998c8eb2d4 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/wrapper/BaselineFillingMergeWrapperTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/wrapper/BaselineFillingMergeWrapperTest.java
@@ -22,6 +22,7 @@
 import com.linkedin.thirdeye.datalayer.dto.DetectionConfigDTO;
 import com.linkedin.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import com.linkedin.thirdeye.detection.DataProvider;
+import com.linkedin.thirdeye.detection.DefaultInputDataFetcher;
 import com.linkedin.thirdeye.detection.DetectionPipelineResult;
 import com.linkedin.thirdeye.detection.InputDataFetcher;
 import com.linkedin.thirdeye.detection.MockDataProvider;
@@ -30,9 +31,7 @@
 import com.linkedin.thirdeye.detection.MockPipelineOutput;
 import com.linkedin.thirdeye.detection.algorithm.MergeWrapper;
 import com.linkedin.thirdeye.detection.components.MockBaselineProvider;
-import com.linkedin.thirdeye.detection.components.RuleBaselineProvider;
 import com.linkedin.thirdeye.detection.spec.MockBaselineProviderSpec;
-import com.linkedin.thirdeye.detection.spec.RuleBaselineProviderSpec;
 import com.linkedin.thirdeye.detection.spi.components.BaselineProvider;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -109,7 +108,7 @@ public void testMergerCurrentAndBaselineLoading() throws Exception {
     BaselineProvider baselineProvider = new MockBaselineProvider();
     MockBaselineProviderSpec spec = new MockBaselineProviderSpec();
     spec.setAggregates(ImmutableMap.of(MetricSlice.from(1, 3000, 3600), 100.0));
-    InputDataFetcher dataFetcher = new InputDataFetcher(provider, this.config.getId());
+    InputDataFetcher dataFetcher = new DefaultInputDataFetcher(provider, this.config.getId());
     baselineProvider.init(spec, dataFetcher);
     this.config.setComponents(ImmutableMap.of("baseline", baselineProvider));
     this.wrapper = new BaselineFillingMergeWrapper(provider, this.config, 2900, 3600);


 

----------------------------------------------------------------
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