You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ak...@apache.org on 2019/05/30 18:09:56 UTC

[incubator-pinot] branch master updated: SoC - Separate out Tuning from Translator (#4250)

This is an automated email from the ASF dual-hosted git repository.

akshayrai09 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 7abfb12  SoC - Separate out Tuning from Translator (#4250)
7abfb12 is described below

commit 7abfb12e1ad96614c1811f1276eedd3af16af051
Author: Akshay Rai <ak...@gmail.com>
AuthorDate: Thu May 30 11:09:51 2019 -0700

    SoC - Separate out Tuning from Translator (#4250)
    
    Currently tuning is tied together with the translator. Each time we need to tune the detection, we have to call the translator which tries to re-convert the yaml config to detection object. This PR separates the tuner from the translator so that it can be triggered directly on the detection object.
    
    Changes:
    * Introduced ConfigTranslator interface for detection and subscription
    * Enforce validation of raw and parsed config through interface (template-pattern)
    * Moved out tuning related code under DetectionConfigTuner
    * Move translator specific code under translator package
    * Remove YamlTranslationResult; Cleaned up buildMetricUrn; Consistently use componentKey throughout instead of componentName
    * Added and updated unit tests
---
 .../api/application/ApplicationResource.java       |   2 +-
 .../thirdeye/detection/DetectionPipeline.java      |  17 +-
 .../pinot/thirdeye/detection/DetectionUtils.java   |  18 ++-
 .../annotation/registry/DetectionRegistry.java     |   4 +-
 .../onboard/YamlOnboardingTaskRunner.java          |  18 +--
 .../detection/validators/ConfigValidator.java      |   2 +-
 .../validators/SubscriptionConfigValidator.java    |   2 +-
 .../detection/wrapper/AnomalyDetectorWrapper.java  |   2 +-
 .../detection/wrapper/AnomalyFilterWrapper.java    |   2 +-
 .../wrapper/BaselineFillingMergeWrapper.java       |  12 +-
 .../thirdeye/detection/wrapper/GrouperWrapper.java |   2 +-
 .../detection/yaml/DetectionConfigTuner.java       | 178 +++++++++++++++++++++
 .../thirdeye/detection/yaml/YamlResource.java      |  36 +++--
 .../detection/yaml/YamlTranslationResult.java      |  87 ----------
 .../CompositePipelineConfigTranslator.java         | 113 ++++---------
 .../yaml/translator/ConfigTranslator.java          |  55 +++++++
 .../YamlDetectionAlertConfigTranslator.java        |  48 +++---
 .../YamlDetectionConfigTranslator.java             |  57 ++-----
 .../YamlDetectionTranslatorLoader.java             |   2 +-
 .../thirdeye/rootcause/impl/MetricEntity.java      |  13 ++
 .../thirdeye/detection/DetectionUtilsTest.java     |  51 ++++++
 .../yaml/MockYamlDetectionConfigTranslator.java    |  21 ---
 .../yaml/YamlDetectionConfigTranslatorTest.java    |  60 -------
 .../CompositePipelineConfigTranslatorTest.java     |  38 +++--
 .../YamlDetectionAlertConfigTranslatorTest.java    |  13 +-
 .../yaml/translator/YamlTranslationResult.java     |  54 +++++++
 .../compositePipelineTranslatorTestResult-1.json   |   6 +-
 .../compositePipelineTranslatorTestResult-2.json   |   0
 .../yaml/{ => translator}/pipeline-config-1.yaml   |   2 +-
 .../yaml/{ => translator}/pipeline-config-2.yaml   |   0
 .../yaml/{ => translator}/pipeline-config-3.yaml   |   0
 31 files changed, 524 insertions(+), 391 deletions(-)

diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/api/application/ApplicationResource.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/api/application/ApplicationResource.java
index 639e79a..7a2edf4 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/api/application/ApplicationResource.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/api/application/ApplicationResource.java
@@ -45,7 +45,7 @@ import org.apache.pinot.thirdeye.detection.ConfigUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.pinot.thirdeye.detection.yaml.YamlDetectionAlertConfigTranslator.*;
+import static org.apache.pinot.thirdeye.detection.yaml.translator.YamlDetectionAlertConfigTranslator.*;
 
 
 @Path(value = "/application")
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionPipeline.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionPipeline.java
index d8b2d66..dafc844 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionPipeline.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionPipeline.java
@@ -90,22 +90,21 @@ public abstract class DetectionPipeline {
     Map<String, BaseComponent> instancesMap = config.getComponents();
     Map<String, Object> componentSpecs = config.getComponentSpecs();
     if (componentSpecs != null) {
-      for (String componentName : componentSpecs.keySet()) {
-        Map<String, Object> componentSpec = MapUtils.getMap(componentSpecs, componentName);
-        if (!instancesMap.containsKey(componentName)){
-          instancesMap.put(componentName, createComponent(componentSpec));
+      for (String componentKey : componentSpecs.keySet()) {
+        Map<String, Object> componentSpec = MapUtils.getMap(componentSpecs, componentKey);
+        if (!instancesMap.containsKey(componentKey)){
+          instancesMap.put(componentKey, createComponent(componentSpec));
         }
       }
 
-      for (String componentName : componentSpecs.keySet()) {
-        Map<String, Object> componentSpec = MapUtils.getMap(componentSpecs, componentName);
+      for (String componentKey : componentSpecs.keySet()) {
+        Map<String, Object> componentSpec = MapUtils.getMap(componentSpecs, componentKey);
         for (Map.Entry<String, Object> entry : componentSpec.entrySet()){
           if (DetectionUtils.isReferenceName(entry.getValue().toString())) {
-            String refComponentName = DetectionUtils.getComponentName(entry.getValue().toString());
-            componentSpec.put(entry.getKey(), instancesMap.get(refComponentName));
+            componentSpec.put(entry.getKey(), instancesMap.get(DetectionUtils.getComponentKey(entry.getValue().toString())));
           }
         }
-        instancesMap.get(componentName).init(getComponentSpec(componentSpec), dataFetcher);
+        instancesMap.get(componentKey).init(getComponentSpec(componentSpec), dataFetcher);
       }
     }
     config.setComponents(instancesMap);
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionUtils.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionUtils.java
index 8da6ed3..8d58fe9 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionUtils.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionUtils.java
@@ -43,7 +43,6 @@ import org.apache.pinot.thirdeye.detection.spi.components.BaselineProvider;
 import org.apache.pinot.thirdeye.detection.spi.model.TimeSeries;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
-import org.joda.time.Duration;
 import org.joda.time.Period;
 import org.joda.time.PeriodType;
 
@@ -67,13 +66,22 @@ public class DetectionUtils {
     return key.startsWith("$");
   }
 
-  // get the component name from the reference key
-  // example "$myRule:ALGORITHM:0" -> "myRule:ALGORITHM:0"
-  public static String getComponentName(String key) {
-    if (isReferenceName(key)) return key.substring(1);
+  // Extracts the component key from the reference key
+  // e.g., "$myRule:ALGORITHM" -> "myRule:ALGORITHM"
+  public static String getComponentKey(String componentRefKey) {
+    if (isReferenceName(componentRefKey)) return componentRefKey.substring(1);
     else throw new IllegalArgumentException("not a component reference key. should starts with $");
   }
 
+  // Extracts the component type from the component key
+  // e.g., "myRule:ALGORITHM" -> "ALGORITHM"
+  public static String getComponentType(String componentKey) {
+    if (componentKey != null && componentKey.contains(":")) {
+      return componentKey.substring(componentKey.lastIndexOf(":") + 1);
+    }
+    throw new IllegalArgumentException("componentKey is invalid; must be of type componentName:type");
+  }
+
   // get the spec class name for a component class
   public static String getSpecClassName(Class<BaseComponent> componentClass) {
     ParameterizedType genericSuperclass = (ParameterizedType) componentClass.getGenericInterfaces()[0];
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/annotation/registry/DetectionRegistry.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/annotation/registry/DetectionRegistry.java
index 9df220e..4fe121c 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/annotation/registry/DetectionRegistry.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/annotation/registry/DetectionRegistry.java
@@ -24,10 +24,9 @@ import com.google.common.collect.ImmutableMap;
 import org.apache.pinot.thirdeye.detection.annotation.Components;
 import org.apache.pinot.thirdeye.detection.annotation.Tune;
 import org.apache.pinot.thirdeye.detection.annotation.Yaml;
-import org.apache.pinot.thirdeye.detection.spec.AbstractSpec;
 import org.apache.pinot.thirdeye.detection.spi.components.BaseComponent;
 import org.apache.pinot.thirdeye.detection.spi.components.BaselineProvider;
-import org.apache.pinot.thirdeye.detection.yaml.YamlDetectionConfigTranslator;
+import org.apache.pinot.thirdeye.detection.yaml.translator.YamlDetectionConfigTranslator;
 import java.lang.annotation.Annotation;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -35,7 +34,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.commons.collections.MapUtils;
-import org.apache.pinot.thirdeye.rootcause.timeseries.Baseline;
 import org.reflections.Reflections;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskRunner.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskRunner.java
index 4bd012e..95456fb 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskRunner.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskRunner.java
@@ -22,7 +22,6 @@ package org.apache.pinot.thirdeye.detection.onboard;
 
 import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 import org.apache.pinot.thirdeye.anomaly.task.TaskContext;
 import org.apache.pinot.thirdeye.anomaly.task.TaskInfo;
 import org.apache.pinot.thirdeye.anomaly.task.TaskResult;
@@ -47,8 +46,8 @@ import org.apache.pinot.thirdeye.detection.DefaultDataProvider;
 import org.apache.pinot.thirdeye.detection.DetectionPipeline;
 import org.apache.pinot.thirdeye.detection.DetectionPipelineLoader;
 import org.apache.pinot.thirdeye.detection.DetectionPipelineResult;
-import org.apache.pinot.thirdeye.detection.yaml.YamlDetectionConfigTranslator;
-import org.apache.pinot.thirdeye.detection.yaml.YamlDetectionTranslatorLoader;
+import org.apache.pinot.thirdeye.detection.yaml.DetectionConfigTuner;
+import org.apache.pinot.thirdeye.detection.yaml.translator.YamlDetectionTranslatorLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
@@ -125,16 +124,9 @@ public class YamlOnboardingTaskRunner implements TaskRunner {
     }
 
     // re-tune the detection pipeline because tuning is depend on replay result. e.g. algorithm-based alert filter
-    YamlDetectionConfigTranslator translator =
-        this.translatorLoader.from((Map<String, Object>) this.yaml.load(config.getYaml()), this.provider);
-
-    DetectionConfigDTO newDetectionConfig =
-        translator.withTuningWindow(info.getTuningWindowStart(), info.getTuningWindowEnd())
-        .withExistingDetectionConfig(config)
-        .generateDetectionConfig();
-    newDetectionConfig.setYaml(config.getYaml());
-
-    this.detectionDAO.save(newDetectionConfig);
+    DetectionConfigTuner detectionConfigTuner = new DetectionConfigTuner(config, provider);
+    DetectionConfigDTO tunedConfig = detectionConfigTuner.tune(info.getTuningWindowStart(), info.getTuningWindowEnd());
+    this.detectionDAO.save(tunedConfig);
 
     LOG.info("Yaml detection onboarding task for id {} completed", info.getConfigId());
     return Collections.emptyList();
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/ConfigValidator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/ConfigValidator.java
index 8f22720..cf7e778 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/ConfigValidator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/ConfigValidator.java
@@ -27,7 +27,7 @@ import org.apache.pinot.thirdeye.datalayer.dto.AbstractDTO;
  * Validate a config
  * @param <T> the type of the config
  */
-interface ConfigValidator<T extends AbstractDTO> {
+public interface ConfigValidator<T extends AbstractDTO> {
   /**
    * Validate the configuration
    * @param config the config
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/SubscriptionConfigValidator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/SubscriptionConfigValidator.java
index 85f4f39..d0a35cc 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/SubscriptionConfigValidator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/SubscriptionConfigValidator.java
@@ -28,7 +28,7 @@ import org.apache.pinot.thirdeye.datasource.DAORegistry;
 import org.apache.pinot.thirdeye.detection.ConfigUtils;
 import org.quartz.CronExpression;
 
-import static org.apache.pinot.thirdeye.detection.yaml.YamlDetectionAlertConfigTranslator.*;
+import static org.apache.pinot.thirdeye.detection.yaml.translator.YamlDetectionAlertConfigTranslator.*;
 
 
 public class SubscriptionConfigValidator implements ConfigValidator<DetectionAlertConfigDTO> {
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 807a4eb..f7fa40d 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
@@ -119,7 +119,7 @@ public class AnomalyDetectorWrapper extends DetectionPipeline {
     this.metric = provider.fetchMetrics(Collections.singleton(this.metricEntity.getId())).get(this.metricEntity.getId());
 
     Preconditions.checkArgument(this.config.getProperties().containsKey(PROP_DETECTOR));
-    this.detectorName = DetectionUtils.getComponentName(MapUtils.getString(config.getProperties(), PROP_DETECTOR));
+    this.detectorName = DetectionUtils.getComponentKey(MapUtils.getString(config.getProperties(), PROP_DETECTOR));
     Preconditions.checkArgument(this.config.getComponents().containsKey(this.detectorName));
     this.anomalyDetector = (AnomalyDetector) this.config.getComponents().get(this.detectorName);
 
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/AnomalyFilterWrapper.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/AnomalyFilterWrapper.java
index c4d52fb..b624772 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/AnomalyFilterWrapper.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/AnomalyFilterWrapper.java
@@ -59,7 +59,7 @@ public class AnomalyFilterWrapper extends DetectionPipeline {
     this.nestedProperties = ConfigUtils.getList(properties.get(PROP_NESTED));
 
     Preconditions.checkArgument(this.config.getProperties().containsKey(PROP_FILTER));
-    String detectorReferenceKey = DetectionUtils.getComponentName(MapUtils.getString(config.getProperties(), PROP_FILTER));
+    String detectorReferenceKey = DetectionUtils.getComponentKey(MapUtils.getString(config.getProperties(), PROP_FILTER));
     Preconditions.checkArgument(this.config.getComponents().containsKey(detectorReferenceKey));
     this.anomalyFilter = (AnomalyFilter) this.config.getComponents().get(detectorReferenceKey);
 
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/BaselineFillingMergeWrapper.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/BaselineFillingMergeWrapper.java
index 8355b5c..a0a0644 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/BaselineFillingMergeWrapper.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/BaselineFillingMergeWrapper.java
@@ -77,7 +77,7 @@ public class BaselineFillingMergeWrapper extends MergeWrapper {
     super(provider, config, startTime, endTime);
 
     if (config.getProperties().containsKey(PROP_BASELINE_PROVIDER)) {
-      this.baselineProviderComponentName = DetectionUtils.getComponentName(MapUtils.getString(config.getProperties(), PROP_BASELINE_PROVIDER));
+      this.baselineProviderComponentName = DetectionUtils.getComponentKey(MapUtils.getString(config.getProperties(), PROP_BASELINE_PROVIDER));
       Preconditions.checkArgument(this.config.getComponents().containsKey(this.baselineProviderComponentName));
       this.baselineValueProvider = (BaselineProvider) this.config.getComponents().get(this.baselineProviderComponentName);
     } else {
@@ -91,7 +91,7 @@ public class BaselineFillingMergeWrapper extends MergeWrapper {
     }
 
     if (config.getProperties().containsKey(PROP_CURRENT_PROVIDER)) {
-      String detectorReferenceKey = DetectionUtils.getComponentName(MapUtils.getString(config.getProperties(), PROP_CURRENT_PROVIDER));
+      String detectorReferenceKey = DetectionUtils.getComponentKey(MapUtils.getString(config.getProperties(), PROP_CURRENT_PROVIDER));
       Preconditions.checkArgument(this.config.getComponents().containsKey(detectorReferenceKey));
       this.currentValueProvider = (BaselineProvider) this.config.getComponents().get(detectorReferenceKey);
     } else {
@@ -104,11 +104,11 @@ public class BaselineFillingMergeWrapper extends MergeWrapper {
     }
 
     // inject detector to nested property if possible
-    String detectorComponentKey = MapUtils.getString(config.getProperties(), PROP_DETECTOR);
-    if (detectorComponentKey != null){
-      this.detectorComponentName = DetectionUtils.getComponentName(detectorComponentKey);
+    String detectorComponentRefKey = MapUtils.getString(config.getProperties(), PROP_DETECTOR);
+    if (detectorComponentRefKey != null){
+      this.detectorComponentName = DetectionUtils.getComponentKey(detectorComponentRefKey);
       for (Map<String, Object> properties : this.nestedProperties){
-        properties.put(PROP_DETECTOR, detectorComponentKey);
+        properties.put(PROP_DETECTOR, detectorComponentRefKey);
       }
     }
 
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/GrouperWrapper.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/GrouperWrapper.java
index 64b1c87..f0f9dba 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/GrouperWrapper.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/GrouperWrapper.java
@@ -59,7 +59,7 @@ public class GrouperWrapper extends DetectionPipeline {
 
     // Get the configured grouper from the components
     Preconditions.checkArgument(this.config.getProperties().containsKey(PROP_GROUPER));
-    this.grouperName = DetectionUtils.getComponentName(MapUtils.getString(config.getProperties(), PROP_GROUPER));
+    this.grouperName = DetectionUtils.getComponentKey(MapUtils.getString(config.getProperties(), PROP_GROUPER));
     Preconditions.checkArgument(this.config.getComponents().containsKey(this.grouperName));
     this.grouper = (Grouper) this.config.getComponents().get(this.grouperName);
   }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/DetectionConfigTuner.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/DetectionConfigTuner.java
new file mode 100644
index 0000000..ded20cc
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/DetectionConfigTuner.java
@@ -0,0 +1,178 @@
+/*
+ * 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.yaml;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.collections.MapUtils;
+import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
+import org.apache.pinot.thirdeye.detection.ConfigUtils;
+import org.apache.pinot.thirdeye.detection.DataProvider;
+import org.apache.pinot.thirdeye.detection.DefaultInputDataFetcher;
+import org.apache.pinot.thirdeye.detection.DetectionUtils;
+import org.apache.pinot.thirdeye.detection.InputDataFetcher;
+import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionRegistry;
+import org.apache.pinot.thirdeye.detection.spec.AbstractSpec;
+import org.apache.pinot.thirdeye.detection.spi.components.Tunable;
+import org.apache.pinot.thirdeye.rootcause.impl.MetricEntity;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.pinot.thirdeye.detection.DetectionUtils.*;
+
+
+/**
+ * This class focuses on tuning the detection config for the provided tuning window
+ *
+ * This wraps and calls the appropriate tunable implementation
+ */
+public class DetectionConfigTuner {
+  protected static final Logger LOG = LoggerFactory.getLogger(DetectionConfigTuner.class);
+
+  private static final String PROP_FILTERS = "filters";
+  private static final String PROP_CLASS_NAME = "className";
+  private static final String DEFAULT_TIMEZONE = "America/Los_Angeles";
+  private static final String PROP_METRIC = "metric";
+  private static final String PROP_DATASET = "dataset";
+
+  private static final DetectionRegistry DETECTION_REGISTRY = DetectionRegistry.getInstance();
+  static {
+    // do not tune for alerts migrated from legacy anomaly function.
+    DetectionRegistry.registerComponent("com.linkedin.thirdeye.detection.components.AdLibAlertFilter",
+        "MIGRATED_ALGORITHM_FILTER");
+    DetectionRegistry.registerComponent("com.linkedin.thirdeye.detection.components.AdLibAnomalyDetector",
+        "MIGRATED_ALGORITHM");
+  }
+
+  public static final String PROP_YAML_PARAMS = "yamlParams";
+  public static final Set<String> TURNOFF_TUNING_COMPONENTS =
+      ImmutableSet.of("MIGRATED_ALGORITHM_FILTER", "MIGRATED_ALGORITHM", "MIGRATED_ALGORITHM_BASELINE");
+
+  private final DetectionConfigDTO detectionConfig;
+  private final DataProvider dataProvider;
+  private final DatasetConfigDTO datasetConfig;
+  private final String metricUrn;
+
+  public DetectionConfigTuner(DetectionConfigDTO config, DataProvider dataProvider) {
+    this.detectionConfig = config;
+    this.dataProvider = dataProvider;
+
+    Map<String, Object> yamlConfig = ConfigUtils.getMap(new org.yaml.snakeyaml.Yaml().load(config.getYaml()));
+
+    MetricConfigDTO metricConfig = dataProvider.fetchMetric(
+        MapUtils.getString(yamlConfig, PROP_METRIC),
+        MapUtils.getString(yamlConfig, PROP_DATASET));
+    Preconditions.checkNotNull(metricConfig, "metric not found");
+    this.datasetConfig = dataProvider.fetchDatasets(Collections.singletonList(metricConfig.getDataset()))
+        .get(metricConfig.getDataset());
+    Preconditions.checkNotNull(this.datasetConfig, "dataset not found");
+
+    this.metricUrn = MetricEntity.fromMetric(MapUtils.getMap(yamlConfig, PROP_FILTERS), metricConfig.getId()).getUrn();
+  }
+
+  /**
+   * Returns the default(for new alert) or tuned configuration for the specified component
+   *
+   * @param componentProps previously tuned configuration along with user supplied yaml config
+   * @param startTime start time of the tuning window
+   * @param endTime end time of the tuning window
+   */
+  private Map<String, Object> getTunedSpecs(Map<String, Object> componentProps, long startTime, long endTime)
+      throws Exception {
+    Map<String, Object> tunedSpec = new HashMap<>();
+
+    // Instantiate tunable component
+    long configId = detectionConfig == null ? 0 : detectionConfig.getId();
+    String componentClassName = componentProps.get(PROP_CLASS_NAME).toString();
+    Map<String, Object> yamlParams = ConfigUtils.getMap(componentProps.get(PROP_YAML_PARAMS));
+    InputDataFetcher dataFetcher = new DefaultInputDataFetcher(dataProvider, configId);
+    Tunable tunable = instantiateTunable(componentClassName, yamlParams, dataFetcher);
+
+    // round to daily boundary
+    DateTimeZone timezone = DateTimeZone.forID(this.datasetConfig.getTimezone() == null ? DEFAULT_TIMEZONE : this.datasetConfig.getTimezone());
+    DateTime start = new DateTime(startTime, timezone).withTimeAtStartOfDay();
+    DateTime end =  new DateTime(endTime, timezone).withTimeAtStartOfDay();
+    Interval window = new Interval(start, end);
+
+    // TODO: if dimension drill down applied, pass in the metric urn of top dimension
+    tunedSpec.putAll(tunable.tune(componentProps, window, this.metricUrn));
+
+    // Hack to retain the raw yaml parameters.
+    // The tunable requires raw yaml params and previously tuned params to generate fresh params
+    tunedSpec.put(PROP_YAML_PARAMS, yamlParams);
+
+    return tunedSpec;
+  }
+
+  private Tunable instantiateTunable(String componentClassName, Map<String, Object> yamlParams, InputDataFetcher dataFetcher)
+      throws Exception {
+    String tunableClassName = DETECTION_REGISTRY.lookupTunable(componentClassName);
+    Class clazz = Class.forName(tunableClassName);
+    Class<AbstractSpec> specClazz = (Class<AbstractSpec>) Class.forName(getSpecClassName(clazz));
+    AbstractSpec spec = AbstractSpec.fromProperties(yamlParams, specClazz);
+    Tunable tunable = (Tunable) clazz.newInstance();
+    tunable.init(spec, dataFetcher);
+    return tunable;
+  }
+
+  /**
+   * Scans through all the tunable components and injects tuned model params into the detection config
+   */
+  public DetectionConfigDTO tune(long tuningWindowStart, long tuningWindowEnd) {
+    Map<String, Object> tunedComponentSpecs = new HashMap<>();
+
+    // Tune each tunable component in the detection componentSpecs
+    Map<String, Object> allComponentSpecs = this.detectionConfig.getComponentSpecs();
+    for (Map.Entry<String, Object> componentSpec : allComponentSpecs.entrySet()) {
+      Map<String, Object> tunedComponentProps = new HashMap<>();
+
+      String componentKey = componentSpec.getKey();
+      Map<String, Object> existingComponentProps = ConfigUtils.getMap(componentSpec.getValue());
+
+      // For tunable components, the model params are computed from user supplied yaml params and previous model params.
+      String componentClassName = existingComponentProps.get(PROP_CLASS_NAME).toString();
+      String type = DetectionUtils.getComponentType(componentKey);
+      if (!TURNOFF_TUNING_COMPONENTS.contains(type) && DETECTION_REGISTRY.isTunable(componentClassName)) {
+        try {
+          tunedComponentProps.put(PROP_CLASS_NAME, componentClassName);
+          tunedComponentProps.putAll(getTunedSpecs(existingComponentProps, tuningWindowStart, tuningWindowEnd));
+        } catch (Exception e) {
+          LOG.error("Tuning failed for component " + type, e);
+        }
+      } else {
+        tunedComponentProps.putAll(existingComponentProps);
+      }
+
+      tunedComponentSpecs.put(componentKey, tunedComponentProps);
+    }
+
+    detectionConfig.setComponentSpecs(tunedComponentSpecs);
+    return detectionConfig;
+  }
+}
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 430bf2a..a247b71 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
@@ -79,13 +79,16 @@ import org.apache.pinot.thirdeye.detection.spi.components.BaselineProvider;
 import org.apache.pinot.thirdeye.detection.spi.model.TimeSeries;
 import org.apache.pinot.thirdeye.detection.validators.DetectionConfigValidator;
 import org.apache.pinot.thirdeye.detection.validators.SubscriptionConfigValidator;
+import org.apache.pinot.thirdeye.detection.yaml.translator.YamlDetectionAlertConfigTranslator;
+import org.apache.pinot.thirdeye.detection.yaml.translator.YamlDetectionConfigTranslator;
+import org.apache.pinot.thirdeye.detection.yaml.translator.YamlDetectionTranslatorLoader;
 import org.apache.pinot.thirdeye.rootcause.impl.MetricEntity;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
 
 import static org.apache.pinot.thirdeye.dataframe.util.DataFrameUtils.*;
-import static org.apache.pinot.thirdeye.detection.yaml.YamlDetectionAlertConfigTranslator.*;
+import static org.apache.pinot.thirdeye.detection.yaml.translator.YamlDetectionAlertConfigTranslator.*;
 
 
 @Path("/yaml")
@@ -102,7 +105,6 @@ public class YamlResource {
   private final DetectionConfigManager detectionConfigDAO;
   private final DetectionAlertConfigManager detectionAlertConfigDAO;
   private final YamlDetectionTranslatorLoader translatorLoader;
-  private final YamlDetectionAlertConfigTranslator alertConfigTranslator;
   private final DetectionConfigValidator detectionValidator;
   private final SubscriptionConfigValidator subscriptionValidator;
   private final DataProvider provider;
@@ -119,7 +121,6 @@ public class YamlResource {
     this.detectionConfigDAO = DAORegistry.getInstance().getDetectionConfigManager();
     this.detectionAlertConfigDAO = DAORegistry.getInstance().getDetectionAlertConfigManager();
     this.translatorLoader = new YamlDetectionTranslatorLoader();
-    this.alertConfigTranslator = new YamlDetectionAlertConfigTranslator(this.detectionConfigDAO);
     this.metricDAO = DAORegistry.getInstance().getMetricConfigDAO();
     this.datasetDAO = DAORegistry.getInstance().getDatasetConfigDAO();
     this.eventDAO = DAORegistry.getInstance().getEventDAO();
@@ -147,7 +148,7 @@ public class YamlResource {
    * Build the detection config from a yaml.
    */
   private DetectionConfigDTO buildDetectionConfigFromYaml(long tuningStartTime, long tuningEndTime, Map<String, Object> yamlConfig,
-      DetectionConfigDTO existingDetectionConfig) {
+      DetectionConfigDTO existingConfig) {
 
     // Configure the tuning window
     if (tuningStartTime == 0L && tuningEndTime == 0L) {
@@ -156,15 +157,26 @@ public class YamlResource {
       tuningStartTime = tuningEndTime - TimeUnit.DAYS.toMillis(28);
     }
 
-    YamlDetectionConfigTranslator translator;
+    YamlDetectionConfigTranslator detectionConfigTranslator;
     try {
-      translator = this.translatorLoader.from(yamlConfig, this.provider);
+      detectionConfigTranslator = this.translatorLoader.from(yamlConfig, this.provider);
     } catch (Exception e) {
       throw new IllegalArgumentException("Unable to instantiate the detection pipeline.", e);
     }
-    return translator.withTuningWindow(tuningStartTime, tuningEndTime)
-        .withExistingDetectionConfig(existingDetectionConfig)
-        .generateDetectionConfig();
+
+    // Translate the raw yaml config to detection config object
+    DetectionConfigDTO config = (DetectionConfigDTO) detectionConfigTranslator.translate();
+
+    // Tune the detection config - Passes the raw yaml params & injects tuned params
+    DetectionConfigTuner detectionTuner = new DetectionConfigTuner(config, provider);
+    config = detectionTuner.tune(tuningStartTime, tuningEndTime);
+
+    if (existingConfig != null) {
+      config.setId(existingConfig.getId());
+      config.setLastTimestamp(existingConfig.getLastTimestamp());
+      config.setCreatedBy(existingConfig.getCreatedBy());
+    }
+    return config;
   }
 
   /*
@@ -408,7 +420,8 @@ public class YamlResource {
     // Translate config from YAML to detection alert config (JSON)
     TreeMap<String, Object> newAlertConfigMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
     newAlertConfigMap.putAll(ConfigUtils.getMap(this.yaml.load(yamlAlertConfig)));
-    DetectionAlertConfigDTO alertConfig = this.alertConfigTranslator.translate(newAlertConfigMap);
+    DetectionAlertConfigDTO alertConfig = (DetectionAlertConfigDTO)
+        new YamlDetectionAlertConfigTranslator(detectionConfigDAO, newAlertConfigMap).translate();
     alertConfig.setYaml(yamlAlertConfig);
 
     // Check for duplicates
@@ -488,7 +501,8 @@ public class YamlResource {
     // Translate payload to detection alert config
     TreeMap<String, Object> newAlertConfigMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
     newAlertConfigMap.putAll(ConfigUtils.getMap(this.yaml.load(yamlAlertConfig)));
-    DetectionAlertConfigDTO newAlertConfig = this.alertConfigTranslator.translate(newAlertConfigMap);
+    DetectionAlertConfigDTO newAlertConfig = (DetectionAlertConfigDTO)
+        new YamlDetectionAlertConfigTranslator(detectionConfigDAO, newAlertConfigMap).translate();
 
     // Update existing alert config with the newly supplied config.
     DetectionAlertConfigDTO updatedAlertConfig = updateDetectionAlertConfig(oldAlertConfig, newAlertConfig);
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlTranslationResult.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlTranslationResult.java
deleted file mode 100644
index d73fe7a..0000000
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlTranslationResult.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * 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.yaml;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Objects;
-
-
-public class YamlTranslationResult {
-  private static final String CRON_SCHEDULE_DEFAULT = "0 0 14 * * ? *";
-
-  private final Map<String, Object> properties;
-  private final Map<String, Object> components;
-  private final String cron;
-
-  public YamlTranslationResult() {
-    this.properties = new HashMap<>();
-    this.components = new HashMap<>();
-    this.cron = CRON_SCHEDULE_DEFAULT;
-  }
-
-  public YamlTranslationResult(Map<String, Object> properties, Map<String, Object> components, String cron) {
-    this.properties = properties;
-    this.components = components;
-    this.cron = cron;
-  }
-
-  public Map<String, Object> getProperties() {
-    return properties;
-  }
-
-  public Map<String, Object> getComponents() {
-    return components;
-  }
-
-  public String getCron() {
-    return cron;
-  }
-
-  YamlTranslationResult withProperties(Map<String, Object> properties) {
-    return new YamlTranslationResult(properties, this.components, this.cron);
-  }
-
-  YamlTranslationResult withComponents(Map<String, Object> components) {
-    return new YamlTranslationResult(this.properties, components, this.cron);
-  }
-
-  YamlTranslationResult withCron(String cron) {
-    return new YamlTranslationResult(this.properties, components, cron);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (!(o instanceof YamlTranslationResult)) {
-      return false;
-    }
-    YamlTranslationResult that = (YamlTranslationResult) o;
-    return Objects.equals(properties, that.properties) && Objects.equals(components, that.components) && Objects.equals(
-        cron, that.cron);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(properties, components, cron);
-  }
-}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/CompositePipelineConfigTranslator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/CompositePipelineConfigTranslator.java
similarity index 79%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/CompositePipelineConfigTranslator.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/CompositePipelineConfigTranslator.java
index dde36f4..f925e46 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/CompositePipelineConfigTranslator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/CompositePipelineConfigTranslator.java
@@ -17,12 +17,10 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.detection.yaml;
+package org.apache.pinot.thirdeye.detection.yaml.translator;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Multimap;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -34,29 +32,24 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.collections.MapUtils;
 import org.apache.pinot.thirdeye.common.time.TimeGranularity;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
 import org.apache.pinot.thirdeye.detection.ConfigUtils;
 import org.apache.pinot.thirdeye.detection.DataProvider;
-import org.apache.pinot.thirdeye.detection.DefaultInputDataFetcher;
 import org.apache.pinot.thirdeye.detection.DetectionUtils;
-import org.apache.pinot.thirdeye.detection.InputDataFetcher;
 import org.apache.pinot.thirdeye.detection.algorithm.DimensionWrapper;
 import org.apache.pinot.thirdeye.detection.annotation.Yaml;
 import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionRegistry;
-import org.apache.pinot.thirdeye.detection.spec.AbstractSpec;
-import org.apache.pinot.thirdeye.detection.spi.components.Tunable;
+import org.apache.pinot.thirdeye.detection.validators.DetectionConfigValidator;
 import org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper;
 import org.apache.pinot.thirdeye.detection.wrapper.AnomalyFilterWrapper;
 import org.apache.pinot.thirdeye.detection.wrapper.BaselineFillingMergeWrapper;
 import org.apache.pinot.thirdeye.detection.wrapper.ChildKeepingMergeWrapper;
 import org.apache.pinot.thirdeye.detection.wrapper.GrouperWrapper;
 import org.apache.pinot.thirdeye.rootcause.impl.MetricEntity;
-import org.joda.time.DateTime;
-import org.joda.time.DateTimeZone;
-import org.joda.time.Interval;
 
 import static org.apache.pinot.thirdeye.detection.ConfigUtils.*;
-import static org.apache.pinot.thirdeye.detection.DetectionUtils.*;
+import static org.apache.pinot.thirdeye.detection.yaml.DetectionConfigTuner.*;
 
 
 /**
@@ -160,10 +153,8 @@ public class CompositePipelineConfigTranslator extends YamlDetectionConfigTransl
   private static final String PROP_MERGER = "merger";
   private static final String PROP_TIMEZONE = "timezone";
   private static final String PROP_NAME = "name";
-  private static final String DEFAULT_TIMEZONE = "America/Los_Angeles";
   private static final String DEFAULT_BASELINE_PROVIDER_YAML_TYPE = "RULE_BASELINE";
   private static final String PROP_BUCKET_PERIOD = "bucketPeriod";
-  private static final String PROP_MAX_DURATION = "maxDuration";
   private static final String PROP_CACHE_PERIOD_LOOKBACK = "cachingPeriodLookback";
 
   private static final DetectionRegistry DETECTION_REGISTRY = DetectionRegistry.getInstance();
@@ -174,8 +165,6 @@ public class CompositePipelineConfigTranslator extends YamlDetectionConfigTransl
     DetectionRegistry.registerComponent("com.linkedin.thirdeye.detection.components.AdLibAnomalyDetector",
         "MIGRATED_ALGORITHM");
   }
-  private static final Set<String> TUNING_OFF_COMPONENTS =
-      ImmutableSet.of("MIGRATED_ALGORITHM_FILTER", "MIGRATED_ALGORITHM", "MIGRATED_ALGORITHM_BASELINE");
   private static final Set<String> MOVING_WINDOW_DETECTOR_TYPES = ImmutableSet.of("ALGORITHM", "MIGRATED_ALGORITHM");
 
   private final Map<String, Object> components = new HashMap<>();
@@ -188,7 +177,11 @@ public class CompositePipelineConfigTranslator extends YamlDetectionConfigTransl
   protected final org.yaml.snakeyaml.Yaml yaml;
 
   public CompositePipelineConfigTranslator(Map<String, Object> yamlConfig, DataProvider provider) {
-    super(yamlConfig, provider);
+    this(yamlConfig, provider, new DetectionConfigValidator(provider));
+  }
+
+  public CompositePipelineConfigTranslator(Map<String, Object> yamlConfig, DataProvider provider, DetectionConfigValidator validator) {
+    super(yamlConfig, provider, validator);
     this.yaml = new org.yaml.snakeyaml.Yaml();
     this.metricConfig = this.dataProvider.fetchMetric(MapUtils.getString(yamlConfig, PROP_METRIC),
         MapUtils.getString(yamlConfig, PROP_DATASET));
@@ -199,11 +192,11 @@ public class CompositePipelineConfigTranslator extends YamlDetectionConfigTransl
     Preconditions.checkNotNull(this.datasetConfig, "dataset not found");
     this.mergerProperties = MapUtils.getMap(yamlConfig, PROP_MERGER, new HashMap<String, Object>());
     this.filterMaps = MapUtils.getMap(yamlConfig, PROP_FILTERS);
-    this.metricUrn = buildMetricUrn(filterMaps, this.metricConfig.getId());
+    this.metricUrn = MetricEntity.fromMetric(filterMaps, this.metricConfig.getId()).getUrn();
   }
 
   @Override
-  YamlTranslationResult translateYaml() {
+  DetectionConfigDTO translateConfig() throws IllegalArgumentException {
     String detectionCronInYaml = MapUtils.getString(yamlConfig, PROP_CRON);
     String cron = (detectionCronInYaml == null) ? buildCron() : detectionCronInYaml;
 
@@ -235,7 +228,7 @@ public class CompositePipelineConfigTranslator extends YamlDetectionConfigTransl
       properties = buildGroupWrapperProperties(grouperYamls.get(0), properties);
     }
 
-    return new YamlTranslationResult().withProperties(properties).withComponents(this.components).withCron(cron);
+    return super.generateDetectionConfig(properties, this.components, cron);
   }
 
   private Map<String, Object> buildDimensionWrapperProperties() {
@@ -246,7 +239,7 @@ public class CompositePipelineConfigTranslator extends YamlDetectionConfigTransl
       dimensionWrapperProperties.putAll(dimensionExploreYaml);
       if (dimensionExploreYaml.containsKey(PROP_DIMENSION_FILTER_METRIC)){
         MetricConfigDTO dimensionExploreMetric = this.dataProvider.fetchMetric(MapUtils.getString(dimensionExploreYaml, PROP_DIMENSION_FILTER_METRIC), this.datasetConfig.getDataset());
-        dimensionWrapperProperties.put(PROP_METRIC_URN, buildMetricUrn(filterMaps, dimensionExploreMetric.getId()));
+        dimensionWrapperProperties.put(PROP_METRIC_URN, MetricEntity.fromMetric(filterMaps, dimensionExploreMetric.getId()).getUrn());
       } else {
         dimensionWrapperProperties.put(PROP_METRIC_URN, this.metricUrn);
       }
@@ -268,24 +261,24 @@ public class CompositePipelineConfigTranslator extends YamlDetectionConfigTransl
     String name = MapUtils.getString(yamlConfig, PROP_NAME);
     Map<String, Object> nestedProperties = new HashMap<>();
     nestedProperties.put(PROP_CLASS_NAME, AnomalyDetectorWrapper.class.getName());
-    String detectorKey = makeComponentKey(detectorType, name);
+    String detectorRefKey = makeComponentRefKey(detectorType, name);
 
     fillInDetectorWrapperProperties(nestedProperties, yamlConfig, detectorType);
 
-    buildComponentSpec(yamlConfig, detectorType, detectorKey);
+    buildComponentSpec(yamlConfig, detectorType, detectorRefKey);
 
     Map<String, Object> properties = new HashMap<>();
     properties.put(PROP_CLASS_NAME, BaselineFillingMergeWrapper.class.getName());
     properties.put(PROP_NESTED, Collections.singletonList(nestedProperties));
-    properties.put(PROP_DETECTOR, detectorKey);
+    properties.put(PROP_DETECTOR, detectorRefKey);
 
     // fill in baseline provider properties
     if (DETECTION_REGISTRY.isBaselineProvider(detectorType)) {
       // if the detector implements the baseline provider interface, use it to generate baseline
-      properties.put(PROP_BASELINE_PROVIDER, detectorKey);
+      properties.put(PROP_BASELINE_PROVIDER, detectorRefKey);
     } else {
       String baselineProviderType = DEFAULT_BASELINE_PROVIDER_YAML_TYPE;
-      String baselineProviderKey = makeComponentKey(baselineProviderType, name);
+      String baselineProviderKey = makeComponentRefKey(baselineProviderType, name);
       buildComponentSpec(yamlConfig, baselineProviderType, baselineProviderKey);
       properties.put(PROP_BASELINE_PROVIDER, baselineProviderKey);
     }
@@ -300,10 +293,10 @@ public class CompositePipelineConfigTranslator extends YamlDetectionConfigTransl
 
     String grouperType = MapUtils.getString(grouperYaml, PROP_TYPE);
     String grouperName = MapUtils.getString(grouperYaml, PROP_NAME);
-    String grouperKey = makeComponentKey(grouperType, grouperName);
-    properties.put(PROP_GROUPER, grouperKey);
+    String grouperRefKey = makeComponentRefKey(grouperType, grouperName);
+    properties.put(PROP_GROUPER, grouperRefKey);
 
-    buildComponentSpec(grouperYaml, grouperType, grouperKey);
+    buildComponentSpec(grouperYaml, grouperType, grouperRefKey);
 
     return properties;
   }
@@ -371,9 +364,9 @@ public class CompositePipelineConfigTranslator extends YamlDetectionConfigTransl
     }
     String name = MapUtils.getString(yamlConfig, PROP_NAME);
     String filterType = MapUtils.getString(yamlConfig, PROP_TYPE);
-    String filterKey = makeComponentKey(filterType, name);
-    wrapperProperties.put(PROP_FILTER, filterKey);
-    buildComponentSpec(yamlConfig, filterType, filterKey);
+    String filterRefKey = makeComponentRefKey(filterType, name);
+    wrapperProperties.put(PROP_FILTER, filterRefKey);
+    buildComponentSpec(yamlConfig, filterType, filterRefKey);
 
     return Collections.singletonList(wrapperProperties);
   }
@@ -419,19 +412,7 @@ public class CompositePipelineConfigTranslator extends YamlDetectionConfigTransl
     }
   }
 
-  private String buildMetricUrn(Map<String, Collection<String>> filterMaps, long metricId) {
-    Multimap<String, String> filters = ArrayListMultimap.create();
-    if (filterMaps != null) {
-      for (Map.Entry<String, Collection<String>> entry : filterMaps.entrySet()) {
-        filters.putAll(entry.getKey(), entry.getValue());
-      }
-    }
-
-    MetricEntity me = MetricEntity.fromMetric(1.0, metricId, filters);
-    return me.getUrn();
-  }
-
-  private void buildComponentSpec(Map<String, Object> yamlConfig, String type, String componentKey) {
+  private void buildComponentSpec(Map<String, Object> yamlConfig, String type, String componentRefKey) {
     Map<String, Object> componentSpecs = new HashMap<>();
 
     String componentClassName = DETECTION_REGISTRY.lookup(type);
@@ -442,50 +423,18 @@ public class CompositePipelineConfigTranslator extends YamlDetectionConfigTransl
       params = MapUtils.getMap(yamlConfig, PROP_PARAMS);
     }
 
-    String componentName = DetectionUtils.getComponentName(componentKey);
-    if (!TUNING_OFF_COMPONENTS.contains(type) && DETECTION_REGISTRY.isTunable(componentClassName)) {
-      try {
-        componentSpecs.putAll(getTunedSpecs(componentName, componentClassName, params));
-      } catch (Exception e) {
-        LOG.error("Tuning failed for component " + type, e);
-      }
+    // For tunable components, the model params are computed from user supplied yaml params and previous model params.
+    // We store the yaml params under a separate key, PROP_YAML_PARAMS, to distinguish from model params.
+    if (!TURNOFF_TUNING_COMPONENTS.contains(type) && DETECTION_REGISTRY.isTunable(componentClassName)) {
+      componentSpecs.put(PROP_YAML_PARAMS, params);
     } else {
       componentSpecs.putAll(params);
     }
-    this.components.put(componentName, componentSpecs);
-  }
-
-  private Map<String, Object> getTunedSpecs(String componentName, String componentClassName, Map<String, Object> params)
-      throws Exception {
-    long configId = this.existingConfig == null ? 0 : this.existingConfig.getId();
-    InputDataFetcher dataFetcher = new DefaultInputDataFetcher(this.dataProvider, configId);
-    Tunable tunable = getTunable(componentClassName, params, dataFetcher);
-
-    // round to daily boundary
-    DateTimeZone timezone = DateTimeZone.forID(this.datasetConfig.getTimezone() == null ? DEFAULT_TIMEZONE : this.datasetConfig.getTimezone());
-    DateTime start = new DateTime(this.startTime, timezone).withTimeAtStartOfDay();
-    DateTime end =  new DateTime(this.endTime, timezone).withTimeAtStartOfDay();
-    Interval window = new Interval(start, end);
-    Map<String, Object> existingComponentSpec =
-        this.existingComponentSpecs.containsKey(componentName) ? MapUtils.getMap(this.existingComponentSpecs,
-            componentName) : Collections.emptyMap();
-
-    // TODO: if dimension drill down applied, pass in the metric urn of top dimension
-    return tunable.tune(existingComponentSpec, window, this.metricUrn);
-  }
 
-  private Tunable getTunable(String componentClassName, Map<String, Object> params, InputDataFetcher dataFetcher)
-      throws Exception {
-    String tunableClassName = DETECTION_REGISTRY.lookupTunable(componentClassName);
-    Class clazz = Class.forName(tunableClassName);
-    Class<AbstractSpec> specClazz = (Class<AbstractSpec>) Class.forName(getSpecClassName(clazz));
-    AbstractSpec spec = AbstractSpec.fromProperties(params, specClazz);
-    Tunable tunable = (Tunable) clazz.newInstance();
-    tunable.init(spec, dataFetcher);
-    return tunable;
+    this.components.put(DetectionUtils.getComponentKey(componentRefKey), componentSpecs);
   }
 
-  private String makeComponentKey(String type, String name) {
+  private String makeComponentRefKey(String type, String name) {
     return "$" + name + ":" + type;
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/ConfigTranslator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/ConfigTranslator.java
new file mode 100644
index 0000000..b6e93e9
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/ConfigTranslator.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.thirdeye.detection.yaml.translator;
+
+import java.util.Map;
+import org.apache.pinot.thirdeye.datalayer.dto.AbstractDTO;
+import org.apache.pinot.thirdeye.detection.DataProvider;
+import org.apache.pinot.thirdeye.detection.validators.ConfigValidator;
+
+
+/**
+ * Translate a raw yaml config and map it to a config object
+ */
+public abstract class ConfigTranslator<T extends AbstractDTO, V extends ConfigValidator> {
+
+  Map<String, Object> yamlConfig;
+
+  protected ConfigValidator validator;
+  protected DataProvider dataProvider;
+
+  ConfigTranslator(Map<String, Object> yamlConfig, V validator) {
+    this.yamlConfig = yamlConfig;
+    this.validator = validator;
+  }
+
+  abstract T translateConfig() throws IllegalArgumentException;
+
+  /**
+   * Convert raw yaml configuration into config object with pre and post validation
+   */
+  public T translate() throws IllegalArgumentException {
+    validator.validateYaml(this.yamlConfig);
+    T configDTO = this.translateConfig();
+    validator.validateConfig(configDTO);
+
+    return configDTO;
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlDetectionAlertConfigTranslator.java
similarity index 84%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslator.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlDetectionAlertConfigTranslator.java
index 6b90e1b..e424d77 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlDetectionAlertConfigTranslator.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.detection.yaml;
+package org.apache.pinot.thirdeye.detection.yaml.translator;
 
 import com.google.common.base.CaseFormat;
 import java.util.stream.Collectors;
@@ -26,6 +26,7 @@ import org.apache.pinot.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.pojo.AlertConfigBean;
 import org.apache.pinot.thirdeye.datalayer.util.Predicate;
 import org.apache.pinot.thirdeye.detection.ConfigUtils;
+import org.apache.pinot.thirdeye.detection.DataProvider;
 import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionAlertRegistry;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -44,7 +45,7 @@ import org.yaml.snakeyaml.Yaml;
 /**
  * The translator converts the alert yaml config into a detection alert config
  */
-public class YamlDetectionAlertConfigTranslator {
+public class YamlDetectionAlertConfigTranslator extends ConfigTranslator {
   public static final String PROP_DETECTION_CONFIG_IDS = "detectionConfigIds";
   public static final String PROP_RECIPIENTS = "recipients";
 
@@ -69,16 +70,18 @@ public class YamlDetectionAlertConfigTranslator {
   private static final String PROP_DIMENSION = "dimension";
   private static final String PROP_DIMENSION_RECIPIENTS = "dimensionRecipients";
 
-  SubscriptionConfigValidator subscriptionValidator;
-
   private static final DetectionAlertRegistry DETECTION_ALERT_REGISTRY = DetectionAlertRegistry.getInstance();
   private static final Set<String> PROPERTY_KEYS = new HashSet<>(
       Arrays.asList(PROP_RECIPIENTS, PROP_DIMENSION, PROP_DIMENSION_RECIPIENTS));
   private final DetectionConfigManager detectionConfigDAO;
 
-  public YamlDetectionAlertConfigTranslator(DetectionConfigManager detectionConfigDAO) {
+  public YamlDetectionAlertConfigTranslator(DetectionConfigManager detectionConfigDAO, Map<String,Object> yamlAlertConfig) {
+    this(detectionConfigDAO, yamlAlertConfig, new SubscriptionConfigValidator());
+  }
+
+  public YamlDetectionAlertConfigTranslator(DetectionConfigManager detectionConfigDAO, Map<String,Object> yamlAlertConfig, SubscriptionConfigValidator validator) {
+    super(yamlAlertConfig, validator);
     this.detectionConfigDAO = detectionConfigDAO;
-    this.subscriptionValidator = new SubscriptionConfigValidator();
   }
 
   private Map<String, Object> buildAlerterProperties(Map<String, Object> alertYamlConfigs, Collection<Long> detectionConfigIds) {
@@ -159,39 +162,38 @@ public class YamlDetectionAlertConfigTranslator {
   /**
    * Generates the {@link DetectionAlertConfigDTO} from the YAML Alert Map
    */
-  public DetectionAlertConfigDTO translate(Map<String,Object> yamlAlertConfig) throws IllegalArgumentException {
-    subscriptionValidator.validateYaml(yamlAlertConfig);
-
+  @Override
+  DetectionAlertConfigDTO translateConfig() throws IllegalArgumentException {
     DetectionAlertConfigDTO alertConfigDTO = new DetectionAlertConfigDTO();
 
-    alertConfigDTO.setName(MapUtils.getString(yamlAlertConfig, PROP_SUBS_GROUP_NAME));
-    alertConfigDTO.setApplication(MapUtils.getString(yamlAlertConfig, PROP_APPLICATION));
-    alertConfigDTO.setFrom(MapUtils.getString(yamlAlertConfig, PROP_FROM));
+    alertConfigDTO.setName(MapUtils.getString(this.yamlConfig, PROP_SUBS_GROUP_NAME));
+    alertConfigDTO.setApplication(MapUtils.getString(this.yamlConfig, PROP_APPLICATION));
+    alertConfigDTO.setFrom(MapUtils.getString(this.yamlConfig, PROP_FROM));
 
-    alertConfigDTO.setCronExpression(MapUtils.getString(yamlAlertConfig, PROP_CRON, CRON_SCHEDULE_DEFAULT));
-    alertConfigDTO.setActive(MapUtils.getBooleanValue(yamlAlertConfig, PROP_ACTIVE, true));
+    alertConfigDTO.setCronExpression(MapUtils.getString(this.yamlConfig, PROP_CRON, CRON_SCHEDULE_DEFAULT));
+    alertConfigDTO.setActive(MapUtils.getBooleanValue(this.yamlConfig, PROP_ACTIVE, true));
 
     alertConfigDTO.setSubjectType(AlertConfigBean.SubjectType.valueOf(
-        (String) MapUtils.getObject(yamlAlertConfig, PROP_EMAIL_SUBJECT_TYPE, AlertConfigBean.SubjectType.METRICS.name())));
+        (String) MapUtils.getObject(this.yamlConfig, PROP_EMAIL_SUBJECT_TYPE, AlertConfigBean.SubjectType.METRICS.name())));
 
-    Map<String, String> refLinks = MapUtils.getMap(yamlAlertConfig, PROP_REFERENCE_LINKS);
+    Map<String, String> refLinks = MapUtils.getMap(this.yamlConfig, PROP_REFERENCE_LINKS);
     if (refLinks == null) {
       refLinks = new HashMap<>();
-      yamlAlertConfig.put(PROP_REFERENCE_LINKS, refLinks);
+      this.yamlConfig.put(PROP_REFERENCE_LINKS, refLinks);
     }
     if (refLinks.isEmpty()) {
       refLinks.put("ThirdEye User Guide", "https://go/thirdeyeuserguide");
       refLinks.put("Add Reference Links", "https://go/thirdeyealertreflink");
     }
-    alertConfigDTO.setReferenceLinks(MapUtils.getMap(yamlAlertConfig, PROP_REFERENCE_LINKS));
+    alertConfigDTO.setReferenceLinks(MapUtils.getMap(this.yamlConfig, PROP_REFERENCE_LINKS));
 
-    alertConfigDTO.setAlertSchemes(buildAlertSchemes(yamlAlertConfig));
-    alertConfigDTO.setAlertSuppressors(buildAlertSuppressors(yamlAlertConfig));
+    alertConfigDTO.setAlertSchemes(buildAlertSchemes(this.yamlConfig));
+    alertConfigDTO.setAlertSuppressors(buildAlertSuppressors(this.yamlConfig));
     alertConfigDTO.setHighWaterMark(0L);
 
     // NOTE: The below fields will/should be hidden from the YAML/UI. They will only be updated by the backend pipeline.
     List<Long> detectionConfigIds = new ArrayList<>();
-    List<String> detectionNames = ConfigUtils.getList(yamlAlertConfig.get(PROP_DETECTION_NAMES));
+    List<String> detectionNames = ConfigUtils.getList(this.yamlConfig.get(PROP_DETECTION_NAMES));
 
     try {
       detectionConfigIds.addAll(detectionNames.stream().map(detectionName ->  this.detectionConfigDAO.findByPredicate(
@@ -200,7 +202,7 @@ public class YamlDetectionAlertConfigTranslator {
       throw new IllegalArgumentException("Cannot find detection pipeline, please check the subscribed detections.");
     }
 
-    alertConfigDTO.setProperties(buildAlerterProperties(yamlAlertConfig, detectionConfigIds));
+    alertConfigDTO.setProperties(buildAlerterProperties(this.yamlConfig, detectionConfigIds));
     Map<Long, Long> vectorClocks = new HashMap<>();
     long currentTimestamp = System.currentTimeMillis();
     for (long detectionConfigId : detectionConfigIds) {
@@ -211,7 +213,7 @@ public class YamlDetectionAlertConfigTranslator {
     DumperOptions options = new DumperOptions();
     options.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK);
     options.setPrettyFlow(true);
-    alertConfigDTO.setYaml(new Yaml(options).dump(yamlAlertConfig));
+    alertConfigDTO.setYaml(new Yaml(options).dump(this.yamlConfig));
 
     return alertConfigDTO;
   }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionConfigTranslator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlDetectionConfigTranslator.java
similarity index 54%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionConfigTranslator.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlDetectionConfigTranslator.java
index 892a50d..dfcc4b1 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionConfigTranslator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlDetectionConfigTranslator.java
@@ -17,10 +17,8 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.detection.yaml;
+package org.apache.pinot.thirdeye.detection.yaml.translator;
 
-import com.google.common.base.Preconditions;
-import java.util.HashMap;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
 import org.apache.pinot.thirdeye.detection.DataProvider;
 import java.util.Map;
@@ -36,62 +34,31 @@ import org.yaml.snakeyaml.Yaml;
  * The YAML config translator converts the yaml config into a detection config.
  * Calls training module for each stage.
  */
-public abstract class YamlDetectionConfigTranslator {
+public abstract class YamlDetectionConfigTranslator extends ConfigTranslator {
   protected static final Logger LOG = LoggerFactory.getLogger(YamlDetectionConfigTranslator.class);
   private static final String PROP_NAME = "detectionName";
   private static final String PROP_DESC_NAME = "description";
   private static final String PROP_ACTIVE = "active";
 
-  DetectionConfigValidator detectionValidator;
-  Map<String, Object> yamlConfig;
-  DetectionConfigDTO existingConfig;
-  Map<String, Object> existingComponentSpecs;
-
-  protected long startTime;
-  protected long endTime;
   protected DataProvider dataProvider;
 
-
-  public YamlDetectionConfigTranslator(Map<String, Object> yamlConfig, DataProvider provider) {
-    this.yamlConfig = yamlConfig;
+  YamlDetectionConfigTranslator(Map<String, Object> yamlConfig, DataProvider provider, DetectionConfigValidator validator) {
+    super(yamlConfig, validator);
     this.dataProvider = provider;
-    this.existingComponentSpecs = new HashMap<>();
-    this.detectionValidator = new DetectionConfigValidator(dataProvider);
-  }
-
-  public YamlDetectionConfigTranslator withTuningWindow(long startTime, long endTime) {
-    this.startTime = startTime;
-    this.endTime = endTime;
-    return this;
-  }
-
-  public YamlDetectionConfigTranslator withExistingDetectionConfig(DetectionConfigDTO existingDTO) {
-    this.existingConfig = existingDTO;
-    if(existingDTO != null) this.existingComponentSpecs = existingDTO.getComponentSpecs();
-    return this;
   }
 
   /**
-   * Convert Yaml configurations into detection properties. Can be customized and override by different detection flow.
-   * @return properties of the detection pipeline
-   */
-  abstract YamlTranslationResult translateYaml();
-
-  /**
    * Fill in common fields of detection config. Properties of the pipeline is filled by the subclass.
    */
-  public DetectionConfigDTO generateDetectionConfig() {
-    detectionValidator.validateYaml(yamlConfig);
-
+  DetectionConfigDTO generateDetectionConfig(Map<String, Object> properties, Map<String, Object> components, String cron) {
     DetectionConfigDTO config = new DetectionConfigDTO();
     config.setName(MapUtils.getString(yamlConfig, PROP_NAME));
     config.setDescription(MapUtils.getString(yamlConfig, PROP_DESC_NAME));
     config.setLastTimestamp(System.currentTimeMillis());
-    YamlTranslationResult translationResult = translateYaml();
-    Preconditions.checkArgument(!translationResult.getProperties().isEmpty(), "Empty detection property");
-    config.setProperties(translationResult.getProperties());
-    config.setComponentSpecs(translationResult.getComponents());
-    config.setCron(translationResult.getCron());
+
+    config.setProperties(properties);
+    config.setComponentSpecs(components);
+    config.setCron(cron);
     config.setActive(MapUtils.getBooleanValue(yamlConfig, PROP_ACTIVE, true));
 
     DumperOptions options = new DumperOptions();
@@ -99,12 +66,6 @@ public abstract class YamlDetectionConfigTranslator {
     options.setPrettyFlow(true);
     config.setYaml(new Yaml(options).dump(yamlConfig));
 
-    if (existingConfig != null) {
-      config.setId(existingConfig.getId());
-      config.setLastTimestamp(existingConfig.getLastTimestamp());
-      config.setCreatedBy(existingConfig.getCreatedBy());
-    }
-
     return config;
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionTranslatorLoader.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlDetectionTranslatorLoader.java
similarity index 96%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionTranslatorLoader.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlDetectionTranslatorLoader.java
index 66a3ee2..1cb0890 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionTranslatorLoader.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlDetectionTranslatorLoader.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.detection.yaml;
+package org.apache.pinot.thirdeye.detection.yaml.translator;
 
 import org.apache.pinot.thirdeye.detection.DataProvider;
 import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionRegistry;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/rootcause/impl/MetricEntity.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/rootcause/impl/MetricEntity.java
index 59e5364..4e32aa9 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/rootcause/impl/MetricEntity.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/rootcause/impl/MetricEntity.java
@@ -19,8 +19,10 @@
 
 package org.apache.pinot.thirdeye.rootcause.impl;
 
+import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.TreeMultimap;
+import java.util.Map;
 import org.apache.pinot.thirdeye.dataframe.util.MetricSlice;
 import org.apache.pinot.thirdeye.rootcause.Entity;
 import org.apache.pinot.thirdeye.rootcause.util.EntityUtils;
@@ -89,6 +91,17 @@ public class MetricEntity extends Entity {
     return fromMetric(score, new ArrayList<Entity>(), id, TreeMultimap.<String, String>create());
   }
 
+  public static MetricEntity fromMetric(Map<String, Collection<String>> filterMaps, long id) {
+    Multimap<String, String> filters = ArrayListMultimap.create();
+    if (filterMaps != null) {
+      for (Map.Entry<String, Collection<String>> entry : filterMaps.entrySet()) {
+        filters.putAll(entry.getKey(), entry.getValue());
+      }
+    }
+
+    return fromMetric(1.0, id, filters);
+  }
+
   public static MetricEntity fromURN(String urn, double score) {
     ParsedUrn parsedUrn = EntityUtils.parseUrnString(urn, TYPE, 3);
     long id = Long.parseLong(parsedUrn.getPrefixes().get(2));
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DetectionUtilsTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DetectionUtilsTest.java
new file mode 100644
index 0000000..a96ba95
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DetectionUtilsTest.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.thirdeye.detection;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class DetectionUtilsTest {
+
+  @Test
+  public void testGetComponentType() {
+    Assert.assertEquals(DetectionUtils.getComponentType("myRule:ALGORITHM"), "ALGORITHM");
+
+    try {
+      Assert.assertEquals(DetectionUtils.getComponentType(null), "ALGORITHM");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(true);
+    }
+
+    try {
+      Assert.assertEquals(DetectionUtils.getComponentType("ALGORITHM"), "ALGORITHM");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(true);
+    }
+
+    try {
+      Assert.assertEquals(DetectionUtils.getComponentType(""), "ALGORITHM");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(true);
+    }
+  }
+
+}
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/MockYamlDetectionConfigTranslator.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/MockYamlDetectionConfigTranslator.java
deleted file mode 100644
index 7f0587f..0000000
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/MockYamlDetectionConfigTranslator.java
+++ /dev/null
@@ -1,21 +0,0 @@
-package org.apache.pinot.thirdeye.detection.yaml;
-
-import org.apache.pinot.thirdeye.detection.DataProvider;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.pinot.thirdeye.detection.validators.DetectionConfigValidator;
-
-
-public class MockYamlDetectionConfigTranslator extends YamlDetectionConfigTranslator {
-  public MockYamlDetectionConfigTranslator(Map<String, Object> yamlConfig, DataProvider provider, DetectionConfigValidator validator) {
-    super(yamlConfig, provider);
-    super.detectionValidator = validator;
-  }
-
-  @Override
-  YamlTranslationResult translateYaml() {
-    Map<String, Object> result = new HashMap<>();
-    result.put("yamlConfigs", yamlConfig);
-    return new YamlTranslationResult().withProperties(result);
-  }
-}
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionConfigTranslatorTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionConfigTranslatorTest.java
deleted file mode 100644
index 6b8c7cb..0000000
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionConfigTranslatorTest.java
+++ /dev/null
@@ -1,60 +0,0 @@
-package org.apache.pinot.thirdeye.detection.yaml;
-
-import org.apache.pinot.thirdeye.datalayer.bao.DAOTestBase;
-import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
-import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
-import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
-import org.apache.pinot.thirdeye.datasource.DAORegistry;
-import org.apache.pinot.thirdeye.detection.MockDataProvider;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.pinot.thirdeye.detection.validators.DetectionConfigValidator;
-import org.testng.Assert;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-import static org.mockito.Mockito.*;
-
-
-public class YamlDetectionConfigTranslatorTest {
-  private MetricConfigManager metricDAO;
-  private DAOTestBase testDAOProvider;
-
-  @Test
-  public void testGenerateDetectionConfig() {
-    Map<String, Object> properties = new HashMap<>();
-    properties.put("className", "test.linkedin.thirdeye");
-
-    Map<String, Object> yamlConfigs = new HashMap<>();
-    yamlConfigs.put("detectionName", "testPipeline");
-    yamlConfigs.put("description", "myTestPipeline");
-    MetricConfigDTO metricConfigDTO = new MetricConfigDTO();
-    metricConfigDTO.setName("a_daily_metric");
-    metricConfigDTO.setDataset("a_test_dataset");
-    metricConfigDTO.setAlias("alias");
-    this.metricDAO.save(metricConfigDTO);
-
-    DetectionConfigValidator validateMocker = mock(DetectionConfigValidator.class);
-    doNothing().when(validateMocker).validateYaml(yamlConfigs);
-
-    YamlDetectionConfigTranslator translator = new MockYamlDetectionConfigTranslator(yamlConfigs, new MockDataProvider(), validateMocker);
-    DetectionConfigDTO detectionConfigDTO = translator.generateDetectionConfig();
-    Assert.assertEquals(detectionConfigDTO.getName(), "testPipeline");
-    Assert.assertEquals(detectionConfigDTO.getDescription(), "myTestPipeline");
-    Assert.assertEquals(detectionConfigDTO.getCron(), "0 0 14 * * ? *");
-    Assert.assertEquals(detectionConfigDTO.getProperties().get("yamlConfigs"), yamlConfigs);
-  }
-
-  @BeforeMethod
-  public void setUp() {
-    this.testDAOProvider = DAOTestBase.getInstance();
-    DAORegistry daoRegistry = DAORegistry.getInstance();
-    this.metricDAO = daoRegistry.getMetricConfigDAO();
-  }
-
-  @AfterMethod
-  public void tearDown() {
-    this.testDAOProvider.cleanup();
-  }
-}
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/CompositePipelineConfigTranslatorTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/CompositePipelineConfigTranslatorTest.java
similarity index 78%
rename from thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/CompositePipelineConfigTranslatorTest.java
rename to thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/CompositePipelineConfigTranslatorTest.java
index eca3b33..85b1aec 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/CompositePipelineConfigTranslatorTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/CompositePipelineConfigTranslatorTest.java
@@ -1,10 +1,12 @@
-package org.apache.pinot.thirdeye.detection.yaml;
+package org.apache.pinot.thirdeye.detection.yaml.translator;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
-import org.apache.commons.io.IOUtils;
+import org.apache.pinot.thirdeye.datalayer.bao.DAOTestBase;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
 import org.apache.pinot.thirdeye.detection.DataProvider;
 import org.apache.pinot.thirdeye.detection.MockDataProvider;
 import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionRegistry;
@@ -16,6 +18,8 @@ import java.util.Collections;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 import org.yaml.snakeyaml.Yaml;
@@ -28,6 +32,19 @@ public class CompositePipelineConfigTranslatorTest {
   private Map<String, Object> yamlConfig;
   private DataProvider provider;
   private static ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  private DAOTestBase testDAOProvider;
+  private DAORegistry daoRegistry;
+
+  @BeforeClass
+  void beforeClass() {
+    testDAOProvider = DAOTestBase.getInstance();
+    daoRegistry = DAORegistry.getInstance();
+  }
+
+  @AfterClass(alwaysRun = true)
+  void afterClass() {
+    testDAOProvider.cleanup();
+  }
 
   @BeforeMethod
   public void setUp() {
@@ -37,10 +54,14 @@ public class CompositePipelineConfigTranslatorTest {
     metricConfig.setDataset("test_dataset");
     this.metricId = 1L;
     metricConfig.setId(metricId);
+    daoRegistry.getMetricConfigDAO().save(metricConfig);
+
     DatasetConfigDTO datasetConfigDTO = new DatasetConfigDTO();
     datasetConfigDTO.setDataset("test_dataset");
     datasetConfigDTO.setTimeUnit(TimeUnit.DAYS);
     datasetConfigDTO.setTimeDuration(1);
+    daoRegistry.getDatasetConfigDAO().save(datasetConfigDTO);
+
     this.yaml = new Yaml();
     DetectionRegistry.registerComponent(ThresholdRuleDetector.class.getName(), "THRESHOLD");
     DetectionRegistry.registerComponent(ThresholdRuleAnomalyFilter.class.getName(), "THRESHOLD_RULE_FILTER");
@@ -49,23 +70,22 @@ public class CompositePipelineConfigTranslatorTest {
     this.provider = new MockDataProvider().setMetrics(Collections.singletonList(metricConfig)).setDatasets(Collections.singletonList(datasetConfigDTO));
   }
 
-
   @Test
   public void testBuildPropertiesFull() throws Exception {
     this.yamlConfig = (Map<String, Object>) this.yaml.load(this.getClass().getResourceAsStream("pipeline-config-1.yaml"));
     CompositePipelineConfigTranslator translator = new CompositePipelineConfigTranslator(this.yamlConfig, this.provider);
-    YamlTranslationResult result = translator.translateYaml();
+    DetectionConfigDTO result = (DetectionConfigDTO) translator.translate();
     YamlTranslationResult expected = OBJECT_MAPPER.readValue(this.getClass().getResourceAsStream("compositePipelineTranslatorTestResult-1.json"), YamlTranslationResult.class);
-    Assert.assertEquals(expected, result);
+    Assert.assertEquals(result.getProperties(), expected.getProperties());
   }
 
   @Test
   public void testBuildDetectionPropertiesNoFilter() throws Exception {
     this.yamlConfig = (Map<String, Object>) this.yaml.load(this.getClass().getResourceAsStream("pipeline-config-2.yaml"));
     CompositePipelineConfigTranslator translator = new CompositePipelineConfigTranslator(this.yamlConfig, this.provider);
-    YamlTranslationResult result = translator.translateYaml();
+    DetectionConfigDTO result = (DetectionConfigDTO) translator.translate();
     YamlTranslationResult expected = OBJECT_MAPPER.readValue(this.getClass().getResourceAsStream("compositePipelineTranslatorTestResult-2.json"), YamlTranslationResult.class);
-    Assert.assertEquals(expected, result);
+    Assert.assertEquals(result.getProperties(), expected.getProperties());
   }
 
   @Test(expectedExceptions = IllegalArgumentException.class)
@@ -74,13 +94,13 @@ public class CompositePipelineConfigTranslatorTest {
     this.yamlConfig.put("rules", Collections.singletonList(
         ImmutableMap.of("name", "rule2","detection", Collections.singletonList(ImmutableMap.of("change", 0.3)))));
     CompositePipelineConfigTranslator translator = new CompositePipelineConfigTranslator(this.yamlConfig, this.provider);
-    translator.generateDetectionConfig();
+    translator.translate();
   }
 
   @Test(expectedExceptions = IllegalArgumentException.class)
   public void testMultipleGrouperLogic() {
     this.yamlConfig = (Map<String, Object>) this.yaml.load(this.getClass().getResourceAsStream("pipeline-config-3.yaml"));
     CompositePipelineConfigTranslator translator = new CompositePipelineConfigTranslator(this.yamlConfig, this.provider);
-    translator.generateDetectionConfig();
+    translator.translate();
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslatorTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlDetectionAlertConfigTranslatorTest.java
similarity index 88%
rename from thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslatorTest.java
rename to thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlDetectionAlertConfigTranslatorTest.java
index cb57d1c..8cf9ede 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslatorTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlDetectionAlertConfigTranslatorTest.java
@@ -1,4 +1,4 @@
-package org.apache.pinot.thirdeye.detection.yaml;
+package org.apache.pinot.thirdeye.detection.yaml.translator;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -13,12 +13,14 @@ import org.apache.pinot.thirdeye.datalayer.pojo.AlertConfigBean;
 import org.apache.pinot.thirdeye.datasource.DAORegistry;
 import org.apache.pinot.thirdeye.detection.ConfigUtils;
 import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionAlertRegistry;
+import org.apache.pinot.thirdeye.detection.validators.SubscriptionConfigValidator;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
-import static org.apache.pinot.thirdeye.detection.yaml.YamlDetectionAlertConfigTranslator.*;
+import static org.apache.pinot.thirdeye.detection.yaml.translator.YamlDetectionAlertConfigTranslator.*;
+import static org.mockito.Mockito.*;
 
 
 public class YamlDetectionAlertConfigTranslatorTest {
@@ -35,6 +37,7 @@ public class YamlDetectionAlertConfigTranslatorTest {
     alertYamlConfigs.put(PROP_SUBS_GROUP_NAME, "test_group_name");
     alertYamlConfigs.put(PROP_APPLICATION, "test_application");
     alertYamlConfigs.put(PROP_FROM, "thirdeye@thirdeye");
+    alertYamlConfigs.put(PROP_TYPE, "DEFAULT_ALERTER_PIPELINE");
     alertYamlConfigs.put(PROP_CRON, CRON_SCHEDULE_DEFAULT);
     alertYamlConfigs.put(PROP_ACTIVE, true);
     alertYamlConfigs.put(PROP_DETECTION_NAMES, Collections.singletonList("test_pipeline_1"));
@@ -65,7 +68,11 @@ public class YamlDetectionAlertConfigTranslatorTest {
     recipients.put("cc", new ArrayList<>(Collections.singleton("userCc@thirdeye.com")));
     alertYamlConfigs.put(PROP_RECIPIENTS, recipients);
 
-    DetectionAlertConfigDTO alertConfig = new YamlDetectionAlertConfigTranslator(this.detectionConfigManager).translate(alertYamlConfigs);
+    SubscriptionConfigValidator validateMocker = mock(SubscriptionConfigValidator.class);
+    doNothing().when(validateMocker).validateYaml(alertYamlConfigs);
+
+    DetectionAlertConfigDTO alertConfig = (DetectionAlertConfigDTO) new YamlDetectionAlertConfigTranslator(
+        this.detectionConfigManager, alertYamlConfigs, validateMocker).translate();
 
     Assert.assertTrue(alertConfig.isActive());
     Assert.assertEquals(alertConfig.getName(), "test_group_name");
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlTranslationResult.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlTranslationResult.java
new file mode 100644
index 0000000..10055fa
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/YamlTranslationResult.java
@@ -0,0 +1,54 @@
+/*
+ * 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.yaml.translator;
+
+import java.util.Map;
+
+
+public class YamlTranslationResult {
+  private Map<String, Object> properties;
+  private Map<String, Object> components;
+  private String cron;
+
+  public void setProperties(Map<String, Object> properties) {
+    this.properties = properties;
+  }
+
+  public Map<String, Object> getProperties() {
+    return properties;
+  }
+
+  public void setComponents(Map<String, Object> components) {
+    this.components = components;
+  }
+
+  public Map<String, Object> getComponents() {
+    return components;
+  }
+
+  public void setCron(String cron) {
+    this.cron = cron;
+  }
+
+  public String getCron() {
+    return cron;
+  }
+
+}
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/translator/compositePipelineTranslatorTestResult-1.json
similarity index 96%
rename from thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/compositePipelineTranslatorTestResult-1.json
rename to thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/compositePipelineTranslatorTestResult-1.json
index aea2eb3..31d11a9 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/translator/compositePipelineTranslatorTestResult-1.json
@@ -33,7 +33,7 @@
                           }
                         ],
                         "detector": "$maxThreshold_1:THRESHOLD",
-                        "maxDuration": 100
+                        "maxDuration": 100000000
                       }
                     ]
                   }
@@ -54,7 +54,7 @@
                       }
                     ],
                     "detector": "$maxThreshold_2:THRESHOLD",
-                    "maxDuration": 100
+                    "maxDuration": 100000000
                   }
                 ]
               }
@@ -66,7 +66,7 @@
             ]
           }
         ],
-        "maxDuration": 100
+        "maxDuration": 100000000
       }]
   },
   "components": {
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/translator/compositePipelineTranslatorTestResult-2.json
similarity index 100%
rename from thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/compositePipelineTranslatorTestResult-2.json
rename to thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/compositePipelineTranslatorTestResult-2.json
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/pipeline-config-1.yaml b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-1.yaml
similarity index 97%
rename from thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/pipeline-config-1.yaml
rename to thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-1.yaml
index 60e6436..26e43c9 100644
--- a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/pipeline-config-1.yaml
+++ b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-1.yaml
@@ -43,7 +43,7 @@ rules:
       min: 50
 merger:
   maxGap: 0
-  maxDuration: 100
+  maxDuration: 100000000
 grouper:
   - type: MOCK_GROUPER
     name: test_grouper
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/pipeline-config-2.yaml b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-2.yaml
similarity index 100%
rename from thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/pipeline-config-2.yaml
rename to thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-2.yaml
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/pipeline-config-3.yaml b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-3.yaml
similarity index 100%
rename from thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/pipeline-config-3.yaml
rename to thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-3.yaml


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