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/21 18:14:59 UTC

[incubator-pinot] branch master updated: [TE] Code cleanup - Pull out all validation checks from translator into validator modules (#4221)

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 3a37067  [TE] Code cleanup - Pull out all validation checks from translator into validator modules (#4221)
3a37067 is described below

commit 3a37067491baa48a9bf8ed129e158f3fbeaf8dfc
Author: Akshay Rai <ak...@gmail.com>
AuthorDate: Tue May 21 11:14:54 2019 -0700

    [TE] Code cleanup - Pull out all validation checks from translator into validator modules (#4221)
    
    Changes:
    * We will support two kinds of validation - raw yaml config validation & translated detection/subscription object validation. Updated the interface accordingly.
    * Moved raw yaml validator tasks under Composite translator into Detection Validator.
    * Consistently use Preconditons package and IllegalArgumentException
---
 .../onboard/YamlOnboardingTaskRunner.java          |   1 +
 .../detection/validators/ConfigValidator.java      |  21 ++-
 .../validators/DetectionConfigValidator.java       | 159 ++++++++++++++++-----
 .../validators/SubscriptionConfigValidator.java    |  98 +++++++------
 .../yaml/CompositePipelineConfigTranslator.java    |  60 --------
 .../yaml/YamlDetectionAlertConfigTranslator.java   |  15 +-
 .../yaml/YamlDetectionConfigTranslator.java        |  42 ++----
 .../yaml/YamlDetectionTranslatorLoader.java        |   1 -
 .../thirdeye/detection/yaml/YamlResource.java      | 109 ++++++--------
 .../yaml/MockYamlDetectionConfigTranslator.java    |   4 +-
 .../yaml/YamlDetectionConfigTranslatorTest.java    |   8 +-
 11 files changed, 261 insertions(+), 257 deletions(-)

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 5835238..4bd012e 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
@@ -132,6 +132,7 @@ public class YamlOnboardingTaskRunner implements TaskRunner {
         translator.withTuningWindow(info.getTuningWindowStart(), info.getTuningWindowEnd())
         .withExistingDetectionConfig(config)
         .generateDetectionConfig();
+    newDetectionConfig.setYaml(config.getYaml());
 
     this.detectionDAO.save(newDetectionConfig);
 
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 007353a..8f22720 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
@@ -19,7 +19,7 @@
 
 package org.apache.pinot.thirdeye.detection.validators;
 
-import javax.xml.bind.ValidationException;
+import java.util.Map;
 import org.apache.pinot.thirdeye.datalayer.dto.AbstractDTO;
 
 
@@ -29,17 +29,24 @@ import org.apache.pinot.thirdeye.datalayer.dto.AbstractDTO;
  */
 interface ConfigValidator<T extends AbstractDTO> {
   /**
-   * Validate the configuration. Thrown a validation exception if validation failed.
+   * Validate the configuration
    * @param config the config
-   * @throws ValidationException the validation exception with error message
+   * @throws IllegalArgumentException exception with error message
    */
-  void validateConfig(T config) throws ValidationException;
+  void validateConfig(T config) throws IllegalArgumentException;
 
   /**
-   * Validate the configuration. Thrown a validation exception if validation failed.
+   * Validate the yaml configuration
+   * @param config the config
+   * @throws IllegalArgumentException exception with error message
+   */
+  void validateYaml(Map<String, Object> config) throws IllegalArgumentException;
+
+  /**
+   * Validate if the updates made to the config are acceptable
    * @param updatedConfig the new config
    * @param oldConfig the old config
-   * @throws ValidationException the validation exception with error message
+   * @throws IllegalArgumentException exception with error message
    */
-  void validateUpdatedConfig(T updatedConfig, T oldConfig) throws ValidationException;
+  void validateUpdatedConfig(T updatedConfig, T oldConfig) throws IllegalArgumentException;
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidator.java
index 94ba5ff..d34da6f 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidator.java
@@ -20,51 +20,50 @@
 package org.apache.pinot.thirdeye.detection.validators;
 
 import com.google.common.base.Preconditions;
-import javax.xml.bind.ValidationException;
-import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
-import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.collections.MapUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
-import org.apache.pinot.thirdeye.datasource.DAORegistry;
-import org.apache.pinot.thirdeye.datasource.ThirdEyeCacheRegistry;
-import org.apache.pinot.thirdeye.datasource.loader.AggregationLoader;
-import org.apache.pinot.thirdeye.datasource.loader.DefaultAggregationLoader;
-import org.apache.pinot.thirdeye.datasource.loader.DefaultTimeSeriesLoader;
-import org.apache.pinot.thirdeye.datasource.loader.TimeSeriesLoader;
+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.DefaultDataProvider;
 import org.apache.pinot.thirdeye.detection.DetectionPipelineLoader;
 import org.quartz.CronExpression;
 
 
 public class DetectionConfigValidator implements ConfigValidator<DetectionConfigDTO> {
 
-  private final DataProvider provider;
   private final DetectionPipelineLoader loader;
+  private final DataProvider provider;
 
-  public DetectionConfigValidator() {
-    MetricConfigManager metricDAO = DAORegistry.getInstance().getMetricConfigDAO();
-    DatasetConfigManager datasetDAO = DAORegistry.getInstance().getDatasetConfigDAO();
-
-    TimeSeriesLoader timeseriesLoader =
-        new DefaultTimeSeriesLoader(metricDAO, datasetDAO, ThirdEyeCacheRegistry.getInstance().getQueryCache());
-
-    AggregationLoader aggregationLoader =
-        new DefaultAggregationLoader(metricDAO, datasetDAO, ThirdEyeCacheRegistry.getInstance().getQueryCache(),
-            ThirdEyeCacheRegistry.getInstance().getDatasetMaxDataTimeCache());
-
+  private static final String PROP_DETECTION = "detection";
+  private static final String PROP_FILTER = "filter";
+  private static final String PROP_METRIC = "metric";
+  private static final String PROP_GROUPER = "grouper";
+  private static final String PROP_DATASET = "dataset";
+  private static final String PROP_TYPE = "type";
+  private static final String PROP_RULES = "rules";
+  private static final String PROP_MERGER = "merger";
+  private static final String PROP_NAME = "name";
+  private static final String PROP_DETECTION_NAME = "detectionName";
+  private static final String PROP_MAX_DURATION = "maxDuration";
+  private static final String PROP_CLASS_NAME = "className";
+
+  public DetectionConfigValidator(DataProvider provider) {
+    this.provider = provider;
     this.loader = new DetectionPipelineLoader();
-
-    this.provider = new DefaultDataProvider(metricDAO, datasetDAO,
-        DAORegistry.getInstance().getEventDAO(),
-        DAORegistry.getInstance().getMergedAnomalyResultDAO(),
-        DAORegistry.getInstance().getEvaluationManager(),
-        timeseriesLoader, aggregationLoader, loader);
   }
 
   /**
    * Validate the pipeline by loading and initializing components
    */
-  private void semanticValidation(DetectionConfigDTO detectionConfig) throws ValidationException {
+  private void semanticValidation(DetectionConfigDTO detectionConfig) {
     try {
       // backup and swap out id
       Long id = detectionConfig.getId();
@@ -77,7 +76,7 @@ public class DetectionConfigValidator implements ConfigValidator<DetectionConfig
       detectionConfig.setId(id);
     } catch (Exception e){
       // exception thrown in validate pipeline via reflection
-      throw new ValidationException("Semantic error: " + e.getCause().getMessage());
+      throw new IllegalArgumentException("Semantic error: " + e.getCause().getMessage());
     }
   }
 
@@ -85,25 +84,111 @@ public class DetectionConfigValidator implements ConfigValidator<DetectionConfig
    * Perform validation on the detection config like verifying if all the required fields are set
    */
   @Override
-  public void validateConfig(DetectionConfigDTO detectionConfig) throws ValidationException {
+  public void validateConfig(DetectionConfigDTO detectionConfig) throws IllegalArgumentException {
+    Preconditions.checkNotNull(detectionConfig);
+
     // Cron Validator
-    if (!CronExpression.isValidExpression(detectionConfig.getCron())) {
-      throw new ValidationException("The detection cron specified is incorrect. Please verify your cron expression"
-          + " using online cron makers.");
-    }
+    Preconditions.checkArgument(CronExpression.isValidExpression(detectionConfig.getCron()),
+        "The detection cron specified is incorrect. Please verify your cron expression using online cron"
+            + " makers.");
+
+    // Empty detection properties
+    Preconditions.checkArgument((detectionConfig.getProperties() != null
+            && detectionConfig.getProperties().get(PROP_CLASS_NAME) != null
+            && StringUtils.isNotEmpty((detectionConfig.getProperties().get(PROP_CLASS_NAME).toString()))),
+        "No detection properties found");
 
     semanticValidation(detectionConfig);
   }
 
   /**
+   * Validates the detection or filter rule accordingly based on {@param ruleType}
+   */
+  private void validateRule(Map<String, Object> ruleYaml, int ruleIndex, String ruleType, Set<String> ruleNamesTaken) {
+    Preconditions.checkArgument(ruleYaml.containsKey(PROP_TYPE),
+        "In rule no." + (ruleIndex) + ", " + ruleType + " rule type is missing.");
+    String type = MapUtils.getString(ruleYaml, PROP_TYPE);
+    String name = MapUtils.getString(ruleYaml, PROP_NAME);
+    Preconditions.checkNotNull(name,
+        "In rule no." + (ruleIndex) + ", " + ruleType + " rule name for type " +  type + " is missing.");
+    Preconditions.checkArgument(!ruleNamesTaken.contains(name),
+        "In rule No." + (ruleIndex) + ", found duplicate rule name, rule name must be unique within config." );
+    Preconditions.checkArgument(!name.contains(":"), "Sorry, rule name cannot contain \':\'");
+  }
+
+  /**
+   * Validate the the detection yaml configuration.
+   *
+   * @param detectionYaml the detection yaml configuration to be validated
+   */
+  @Override
+  public void validateYaml(Map<String, Object> detectionYaml) {
+    // Validate all compulsory fields
+    Preconditions.checkArgument(detectionYaml.containsKey(PROP_DETECTION_NAME), "Property missing " + PROP_DETECTION_NAME);
+    Preconditions.checkArgument(detectionYaml.containsKey(PROP_METRIC), "Property missing " + PROP_METRIC);
+    Preconditions.checkArgument(detectionYaml.containsKey(PROP_DATASET), "Property missing " + PROP_DATASET);
+    Preconditions.checkArgument(detectionYaml.containsKey(PROP_RULES), "Property missing " + PROP_RULES);
+
+    // Validate fields which shouldn't be defined at root level
+    Preconditions.checkArgument(!detectionYaml.containsKey(PROP_FILTER), "Please double check the filter"
+        + " config. Adding dimensions filters should be in the yaml root level using 'filters' as the key. Anomaly "
+        + "filter should be added in to the indentation level of detection yaml it applies to.");
+
+    // Check if the metric defined in the config exists
+    MetricConfigDTO metricConfig = provider
+        .fetchMetric(MapUtils.getString(detectionYaml, PROP_METRIC), MapUtils.getString(detectionYaml, PROP_DATASET));
+    Preconditions.checkNotNull(metricConfig, "Metric defined in the config cannot be found");
+    DatasetConfigDTO datasetConfig = provider
+        .fetchDatasets(Collections.singletonList(metricConfig.getDataset()))
+        .get(metricConfig.getDataset());
+
+    // We support only one grouper per metric
+    Preconditions.checkArgument(ConfigUtils.getList(detectionYaml.get(PROP_GROUPER)).size() <= 1,
+        "Multiple groupers detected for metric. We support only one grouper per metric.");
+
+    // Validate all the rules
+    Set<String> names = new HashSet<>();
+    List<Map<String, Object>> ruleYamls = ConfigUtils.getList(detectionYaml.get(PROP_RULES));
+    for (int i = 1; i <= ruleYamls.size(); i++) {
+      Map<String, Object> ruleYaml = ruleYamls.get(i - 1);
+
+      // Validate detection rules
+      Preconditions.checkArgument(ruleYaml.containsKey(PROP_DETECTION), "In rule no." + (i) + ", detection rule is missing.");
+      List<Map<String, Object>> detectionRuleYamls = ConfigUtils.getList(ruleYaml.get(PROP_DETECTION));
+      for (Map<String, Object> detectionRuleYaml : detectionRuleYamls) {
+        validateRule(detectionRuleYaml, i, "detection", names);
+        names.add(MapUtils.getString(ruleYaml, PROP_NAME));
+      }
+
+      // Validate filter rules
+      if (ruleYaml.containsKey(PROP_FILTER)) {
+        List<Map<String, Object>> filterRuleYamls = ConfigUtils.getList(ruleYaml.get(PROP_FILTER));
+        for (Map<String, Object> filterRuleYaml : filterRuleYamls) {
+          validateRule(filterRuleYaml, i, "filter", names);
+          names.add(MapUtils.getString(ruleYaml, PROP_NAME));
+        }
+      }
+    }
+
+    // Safety condition: Validate if maxDuration is greater than 15 minutes
+    Map<String, Object> mergerProperties = MapUtils.getMap(detectionYaml, PROP_MERGER, new HashMap());
+    if (mergerProperties.get(PROP_MAX_DURATION) != null) {
+      Preconditions.checkArgument(MapUtils.getLong(mergerProperties, PROP_MAX_DURATION) >= datasetConfig.bucketTimeGranularity().toMillis(),
+          "The maxDuration field set is not acceptable. Please check the the document and set it correctly.");
+    }
+  }
+
+  /**
    * Perform validation on the updated detection config. Check for fields which shouldn't be
    * updated by the user.
    */
   @Override
   public void validateUpdatedConfig(DetectionConfigDTO updatedConfig, DetectionConfigDTO oldConfig)
-      throws ValidationException {
+      throws IllegalArgumentException {
     validateConfig(updatedConfig);
+    Preconditions.checkNotNull(oldConfig);
+
     Preconditions.checkArgument(updatedConfig.getId().equals(oldConfig.getId()));
     Preconditions.checkArgument(updatedConfig.getLastTimestamp() == oldConfig.getLastTimestamp());
   }
-}
+}
\ No newline at end of file
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 1b9eabb..85f4f39 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
@@ -19,9 +19,9 @@
 
 package org.apache.pinot.thirdeye.detection.validators;
 
+import com.google.common.base.Preconditions;
 import java.util.List;
 import java.util.Map;
-import javax.xml.bind.ValidationException;
 import org.apache.commons.lang.StringUtils;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
 import org.apache.pinot.thirdeye.datasource.DAORegistry;
@@ -39,76 +39,82 @@ public class SubscriptionConfigValidator implements ConfigValidator<DetectionAle
    * Perform validation on the alert config like verifying if all the required fields are set
    */
   @Override
-  public void validateConfig(DetectionAlertConfigDTO alertConfig) throws ValidationException {
+  public void validateConfig(DetectionAlertConfigDTO alertConfig) throws IllegalArgumentException {
+    Preconditions.checkNotNull(alertConfig);
+
     // Check for all the required fields in the alert
-    if (StringUtils.isEmpty(alertConfig.getName())) {
-      throw new ValidationException("Subscription group name field cannot be left empty.");
-    }
-    if (StringUtils.isEmpty(alertConfig.getApplication())) {
-      throw new ValidationException("Application field cannot be left empty");
-    }
-    if (StringUtils.isEmpty(alertConfig.getFrom())) {
-      throw new ValidationException("From address field cannot be left empty");
-    }
-    if (alertConfig.getProperties() == null || alertConfig.getProperties().get(PROP_CLASS_NAME) == null
-        || StringUtils.isEmpty(alertConfig.getProperties().get(PROP_CLASS_NAME).toString())) {
-      throw new ValidationException("'Type' field cannot be left empty.");
-    }
+    Preconditions.checkArgument(!StringUtils.isEmpty(alertConfig.getName()), "Subscription group name field cannot be left empty.");
+    Preconditions.checkArgument(!StringUtils.isEmpty(alertConfig.getApplication()), "Application field cannot be left empty");
+    Preconditions.checkArgument(!StringUtils.isEmpty(alertConfig.getFrom()), "From address field cannot be left empty");
+
+    // Empty subscription properties
+    Preconditions.checkArgument((alertConfig.getProperties() != null
+            && alertConfig.getProperties().get(PROP_CLASS_NAME) != null
+            && StringUtils.isNotEmpty((alertConfig.getProperties().get(PROP_CLASS_NAME).toString()))),
+        "'Type' field cannot be left empty.");
+
+    // Application name should be valid
+    Preconditions.checkArgument(!DAORegistry.getInstance().getApplicationDAO().findByName(alertConfig.getApplication()).isEmpty(),
+        "Application name doesn't exist in our registry. Please use an existing application name. You may"
+            + " search for registered applications from the ThirdEye dashboard or reach out to ask_thirdeye if you wish"
+            + " to setup a new application.");
+
+    // Cron Validator
+    Preconditions.checkArgument(CronExpression.isValidExpression(alertConfig.getCronExpression()),
+        "The subscription cron specified is incorrect. Please verify your cron expression using online cron"
+            + " makers.");
+
     // At least one alertScheme is required
-    if (alertConfig.getAlertSchemes() == null || alertConfig.getAlertSchemes().size() == 0) {
-      throw new ValidationException("Alert scheme cannot be left empty");
-    }
+    Preconditions.checkArgument((alertConfig.getAlertSchemes() != null && !alertConfig.getAlertSchemes().isEmpty()),
+        "Alert scheme cannot be left empty");
+
+    // At least one recipient must be specified
+    Map<String, Object> recipients = ConfigUtils.getMap(alertConfig.getProperties().get(PROP_RECIPIENTS));
+    Preconditions.checkArgument((!recipients.isEmpty() && !ConfigUtils.getList(recipients.get("to")).isEmpty()),
+        "Please specify at least one recipient in the notification group. If you wish to unsubscribe, set"
+            + " active to false.");
+
     // Properties cannot be empty
-    if (alertConfig.getProperties() == null || alertConfig.getProperties().isEmpty()) {
-      throw new ValidationException("Alert properties cannot be left empty. Please specify the recipients,"
-          + " subscribed detections, and type.");
-    }
+    Preconditions.checkArgument((alertConfig.getProperties() != null && !alertConfig.getProperties().isEmpty()),
+        "Alert properties cannot be left empty. Please specify the recipients, subscribed detections, and"
+            + " type.");
+
     // detectionConfigIds cannot be empty
     List<Long> detectionIds = ConfigUtils.getLongs(alertConfig.getProperties().get(PROP_DETECTION_CONFIG_IDS));
-    if (detectionIds == null || detectionIds.isEmpty()) {
-      throw new ValidationException("A notification group should subscribe to at least one alert. If you wish to"
-          + " unsubscribe, set active to false.");
-    }
-    // At least one recipient must be specified
-    Map<String, Object> recipients = ConfigUtils.getMap(alertConfig.getProperties().get(PROP_RECIPIENTS));
-    if (recipients.isEmpty() || ConfigUtils.getList(recipients.get("to")).isEmpty()) {
-      throw new ValidationException("Please specify at least one recipient in the notification group. If you wish to"
-          + " unsubscribe, set active to false.");
-    }
-    // Application name should be valid
-    if (DAORegistry.getInstance().getApplicationDAO().findByName(alertConfig.getApplication()).size() == 0) {
-      throw new ValidationException("Application name doesn't exist in our registry. Please use an existing"
-          + " application name. You may search for registered applications from the ThirdEye dashboard or reach out"
-          + " to ask_thirdeye if you wish to setup a new application.");
-    }
-    // Cron Validator
-    if (!CronExpression.isValidExpression(alertConfig.getCronExpression())) {
-      throw new ValidationException("The subscription cron specified is incorrect. Please verify your cron expression"
-          + " using online cron makers.");
-    }
+    Preconditions.checkArgument((detectionIds != null && !detectionIds.isEmpty()),
+        "A notification group should subscribe to at least one alert. If you wish to unsubscribe, set active"
+            + " to false.");
 
     // TODO add more checks like email validity, alert type check, scheme type check etc.
   }
 
+  // Validate the raw subscription yaml configuration
+  @Override
+  public void validateYaml(Map<String, Object> config) throws IllegalArgumentException {
+    // Add validations
+  }
+
   /**
    * Perform validation on the updated alert config. Check for fields which shouldn't be
    * updated by the user.
    */
   @Override
   public void validateUpdatedConfig(DetectionAlertConfigDTO updatedAlertConfig, DetectionAlertConfigDTO oldAlertConfig)
-      throws ValidationException {
+      throws IllegalArgumentException {
     validateConfig(updatedAlertConfig);
+    Preconditions.checkNotNull(oldAlertConfig);
 
+    Preconditions.checkArgument(updatedAlertConfig.getId().equals(oldAlertConfig.getId()));
     Long newHighWatermark = updatedAlertConfig.getHighWaterMark();
     Long oldHighWatermark = oldAlertConfig.getHighWaterMark();
     if (newHighWatermark != null && oldHighWatermark != null && newHighWatermark.longValue() != oldHighWatermark) {
-      throw new ValidationException("HighWaterMark has been modified. This is not allowed");
+      throw new IllegalArgumentException("HighWaterMark has been modified. This is not allowed");
     }
     if (updatedAlertConfig.getVectorClocks() != null) {
       for (Map.Entry<Long, Long> vectorClock : updatedAlertConfig.getVectorClocks().entrySet()) {
         if (!oldAlertConfig.getVectorClocks().containsKey(vectorClock.getKey())
             || oldAlertConfig.getVectorClocks().get(vectorClock.getKey()).longValue() != vectorClock.getValue()) {
-          throw new ValidationException("Vector clock has been modified. This is not allowed.");
+          throw new IllegalArgumentException("Vector clock has been modified. This is not allowed.");
         }
       }
     }
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/CompositePipelineConfigTranslator.java
index 1afeef8..5c30ae2 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/CompositePipelineConfigTranslator.java
@@ -27,7 +27,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -484,63 +483,4 @@ public class CompositePipelineConfigTranslator extends YamlDetectionConfigTransl
   private String makeComponentKey(String type, String name) {
     return "$" + name + ":" + type;
   }
-
-  @Override
-  protected void validateYAML(Map<String, Object> yamlConfig) {
-    super.validateYAML(yamlConfig);
-    Preconditions.checkArgument(yamlConfig.containsKey(PROP_METRIC), "Property missing " + PROP_METRIC);
-    Preconditions.checkArgument(yamlConfig.containsKey(PROP_DATASET), "Property missing " + PROP_DATASET);
-    Preconditions.checkArgument(yamlConfig.containsKey(PROP_RULES), "Property missing " + PROP_RULES);
-    Preconditions.checkArgument(!yamlConfig.containsKey(PROP_FILTER),
-        "Please double check the filter config. Adding dimensions filters should be in the yaml root level using 'filters' as the key. Anomaly filter should be added in to the indentation level of detection yaml it applies to.");
-    if (existingConfig != null) {
-      Map<String, Object> existingYamlConfig = (Map<String, Object>) this.yaml.load(existingConfig.getYaml());
-      Preconditions.checkArgument(MapUtils.getString(yamlConfig, PROP_METRIC).equals(MapUtils.getString(existingYamlConfig, PROP_METRIC)), "metric name cannot be modified");
-      Preconditions.checkArgument(MapUtils.getString(yamlConfig, PROP_DATASET).equals(MapUtils.getString(existingYamlConfig, PROP_DATASET)), "dataset name cannot be modified");
-    }
-
-    // Safety condition: Validate if maxDuration is greater than 15 minutes
-    Map<String, Object> mergerProperties = MapUtils.getMap(yamlConfig, PROP_MERGER, new HashMap());
-    if (mergerProperties.get(PROP_MAX_DURATION) != null) {
-      Preconditions.checkArgument(MapUtils.getLong(mergerProperties, PROP_MAX_DURATION) >= datasetConfig.bucketTimeGranularity().toMillis(),
-          "The maxDuration field set is not acceptable. Please check the the document  and set it correctly.");
-    }
-
-    // We support only one grouper per metric
-    Preconditions.checkArgument(getList(yamlConfig.get(PROP_GROUPER)).size() <= 1, "Multiple groupers detected for metric. We support only one grouper per metric.");
-
-    Set<String> names = new HashSet<>();
-    List<Map<String, Object>> ruleYamls = getList(yamlConfig.get(PROP_RULES));
-    for (int i = 0; i < ruleYamls.size(); i++) {
-      Map<String, Object> ruleYaml = ruleYamls.get(i);
-      Preconditions.checkArgument(ruleYaml.containsKey(PROP_DETECTION),
-          "In rule No." + (i + 1) + ", detection rule is missing. ");
-      List<Map<String, Object>> detectionStageYamls = ConfigUtils.getList(ruleYaml.get(PROP_DETECTION));
-      // check each detection rule
-      for (Map<String, Object> detectionStageYaml : detectionStageYamls) {
-        Preconditions.checkArgument(detectionStageYaml.containsKey(PROP_TYPE),
-            "In rule No." + (i + 1) + ", a detection rule type is missing. ");
-        String type = MapUtils.getString(detectionStageYaml, PROP_TYPE);
-        String name = MapUtils.getString(detectionStageYaml, PROP_NAME);
-        Preconditions.checkNotNull(name, "In rule No." + (i + 1) + ", a detection rule name for type " +  type + " is missing");
-        Preconditions.checkArgument(!names.contains(name), "In rule No." + (i + 1) +
-            ", found duplicate rule name, rule name must be unique." );
-        Preconditions.checkArgument(!name.contains(":"), "Sorry, rule name cannot contain \':\'");
-      }
-      if (ruleYaml.containsKey(PROP_FILTER)) {
-        List<Map<String, Object>> filterStageYamls = ConfigUtils.getList(ruleYaml.get(PROP_FILTER));
-        // check each filter rule
-        for (Map<String, Object> filterStageYaml : filterStageYamls) {
-          Preconditions.checkArgument(filterStageYaml.containsKey(PROP_TYPE),
-              "In rule No." + (i + 1) + ", a filter rule type is missing. ");
-          String type = MapUtils.getString(filterStageYaml, PROP_TYPE);
-          String name = MapUtils.getString(filterStageYaml, PROP_NAME);
-          Preconditions.checkNotNull(name, "In rule No." + (i + 1) + ", a filter rule name for type " + type + " is missing");
-          Preconditions.checkArgument(!names.contains(name), "In rule No." + (i + 1) +
-              ", found duplicate rule name, rule name must be unique." );
-          Preconditions.checkArgument(!name.contains(":"), "Sorry, rule name cannot contain \':\'");
-        }
-      }
-    }
-  }
 }
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/YamlDetectionAlertConfigTranslator.java
index 9f6d8d9..04501ab 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/YamlDetectionAlertConfigTranslator.java
@@ -20,19 +20,13 @@
 package org.apache.pinot.thirdeye.detection.yaml;
 
 import com.google.common.base.CaseFormat;
-import com.google.common.base.Preconditions;
 import java.util.stream.Collectors;
-import org.apache.pinot.thirdeye.datalayer.bao.DetectionAlertConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
-import org.apache.pinot.thirdeye.datalayer.dto.AnomalyFunctionDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
-import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.pojo.AlertConfigBean;
 import org.apache.pinot.thirdeye.datalayer.util.Predicate;
-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.annotation.registry.DetectionRegistry;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -42,6 +36,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.commons.collections.MapUtils;
+import org.apache.pinot.thirdeye.detection.validators.SubscriptionConfigValidator;
 import org.yaml.snakeyaml.DumperOptions;
 import org.yaml.snakeyaml.Yaml;
 
@@ -70,10 +65,11 @@ public class YamlDetectionAlertConfigTranslator {
   static final String PROP_TIME_WINDOWS = "timeWindows";
   static final String CRON_SCHEDULE_DEFAULT = "0 0/5 * * * ? *"; // Every 5 min
 
-  private static final String PROP_ONLY_FETCH_LEGACY_ANOMALIES = "onlyFetchLegacyAnomalies";
   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));
@@ -81,6 +77,7 @@ public class YamlDetectionAlertConfigTranslator {
 
   public YamlDetectionAlertConfigTranslator(DetectionConfigManager detectionConfigDAO) {
     this.detectionConfigDAO = detectionConfigDAO;
+    this.subscriptionValidator = new SubscriptionConfigValidator();
   }
 
   private Map<String, Object> buildAlerterProperties(Map<String, Object> alertYamlConfigs, Collection<Long> detectionConfigIds) {
@@ -161,7 +158,9 @@ public class YamlDetectionAlertConfigTranslator {
   /**
    * Generates the {@link DetectionAlertConfigDTO} from the YAML Alert Map
    */
-  public DetectionAlertConfigDTO translate(Map<String,Object> yamlAlertConfig) {
+  public DetectionAlertConfigDTO translate(Map<String,Object> yamlAlertConfig) throws IllegalArgumentException {
+    subscriptionValidator.validateYaml(yamlAlertConfig);
+
     DetectionAlertConfigDTO alertConfigDTO = new DetectionAlertConfigDTO();
 
     alertConfigDTO.setName(MapUtils.getString(yamlAlertConfig, PROP_SUBS_GROUP_NAME));
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/YamlDetectionConfigTranslator.java
index 54d766f..892a50d 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/YamlDetectionConfigTranslator.java
@@ -20,11 +20,12 @@
 package org.apache.pinot.thirdeye.detection.yaml;
 
 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.HashMap;
 import java.util.Map;
 import org.apache.commons.collections.MapUtils;
+import org.apache.pinot.thirdeye.detection.validators.DetectionConfigValidator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.DumperOptions;
@@ -38,21 +39,24 @@ import org.yaml.snakeyaml.Yaml;
 public abstract class YamlDetectionConfigTranslator {
   protected static final Logger LOG = LoggerFactory.getLogger(YamlDetectionConfigTranslator.class);
   private static final String PROP_NAME = "detectionName";
-  private static final String PROP_FILTER = "filter";
   private static final String PROP_DESC_NAME = "description";
   private static final String PROP_ACTIVE = "active";
 
-  protected Map<String, Object> yamlConfig;
+  DetectionConfigValidator detectionValidator;
+  Map<String, Object> yamlConfig;
+  DetectionConfigDTO existingConfig;
+  Map<String, Object> existingComponentSpecs;
+
   protected long startTime;
   protected long endTime;
   protected DataProvider dataProvider;
-  protected DetectionConfigDTO existingConfig;
-  protected Map<String, Object> existingComponentSpecs;
+
 
   public YamlDetectionConfigTranslator(Map<String, Object> yamlConfig, DataProvider provider) {
     this.yamlConfig = yamlConfig;
     this.dataProvider = provider;
     this.existingComponentSpecs = new HashMap<>();
+    this.detectionValidator = new DetectionConfigValidator(dataProvider);
   }
 
   public YamlDetectionConfigTranslator withTuningWindow(long startTime, long endTime) {
@@ -77,7 +81,7 @@ public abstract class YamlDetectionConfigTranslator {
    * Fill in common fields of detection config. Properties of the pipeline is filled by the subclass.
    */
   public DetectionConfigDTO generateDetectionConfig() {
-    validateYAML(yamlConfig);
+    detectionValidator.validateYaml(yamlConfig);
 
     DetectionConfigDTO config = new DetectionConfigDTO();
     config.setName(MapUtils.getString(yamlConfig, PROP_NAME));
@@ -103,30 +107,4 @@ public abstract class YamlDetectionConfigTranslator {
 
     return config;
   }
-
-  /**
-   * Check the yaml configuration is semantically valid. Throws an IllegalArgumentException if not.
-   * @param yamlConfig yamlConfiguration to be checked
-   */
-  protected void validateYAML(Map<String, Object> yamlConfig) {
-    validatePropertyExists(PROP_NAME);
-    validateFilter();
-  }
-
-  /**
-   * Validate property name exists.
-   * @param propName The property name to validate.
-   */
-  private void validatePropertyExists(String propName) {
-    Preconditions.checkArgument(yamlConfig.containsKey(propName), "Property missing " + propName);
-  }
-
-  /**
-   * Validate filter is set in correct level.
-   */
-  private void validateFilter() {
-    if (yamlConfig.containsKey(PROP_FILTER)) {
-      throw new IllegalArgumentException("Filter should be set under rules");
-    }
-  }
 }
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/YamlDetectionTranslatorLoader.java
index 151aa17..66a3ee2 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/YamlDetectionTranslatorLoader.java
@@ -19,7 +19,6 @@
 
 package org.apache.pinot.thirdeye.detection.yaml;
 
-import com.google.common.base.Preconditions;
 import org.apache.pinot.thirdeye.detection.DataProvider;
 import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionRegistry;
 import java.lang.reflect.Constructor;
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 db765fd..a60116f 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
@@ -45,7 +45,6 @@ import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import javax.xml.bind.ValidationException;
 import org.apache.commons.collections.MapUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.pinot.thirdeye.anomaly.task.TaskConstants;
@@ -120,8 +119,6 @@ public class YamlResource {
     this.detectionConfigDAO = DAORegistry.getInstance().getDetectionConfigManager();
     this.detectionAlertConfigDAO = DAORegistry.getInstance().getDetectionAlertConfigManager();
     this.translatorLoader = new YamlDetectionTranslatorLoader();
-    this.detectionValidator = new DetectionConfigValidator();
-    this.subscriptionValidator = new SubscriptionConfigValidator();
     this.alertConfigTranslator = new YamlDetectionAlertConfigTranslator(this.detectionConfigDAO);
     this.metricDAO = DAORegistry.getInstance().getMetricConfigDAO();
     this.datasetDAO = DAORegistry.getInstance().getDatasetConfigDAO();
@@ -141,17 +138,16 @@ public class YamlResource {
     this.loader = new DetectionPipelineLoader();
 
     this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, anomalyDAO, evaluationDAO, timeseriesLoader, aggregationLoader, loader);
-  }
 
-  public DetectionConfigDTO translateToDetectionConfig(Map<String, Object> yamlConfig) throws Exception {
-    return buildDetectionConfigFromYaml(0, 0, yamlConfig, null);
+    this.detectionValidator = new DetectionConfigValidator(this.provider);
+    this.subscriptionValidator = new SubscriptionConfigValidator();
   }
 
   /*
    * Build the detection config from a yaml.
    */
   private DetectionConfigDTO buildDetectionConfigFromYaml(long tuningStartTime, long tuningEndTime, Map<String, Object> yamlConfig,
-      DetectionConfigDTO existingDetectionConfig) throws Exception {
+      DetectionConfigDTO existingDetectionConfig) {
 
     // Configure the tuning window
     if (tuningStartTime == 0L && tuningEndTime == 0L) {
@@ -160,7 +156,12 @@ public class YamlResource {
       tuningStartTime = tuningEndTime - TimeUnit.DAYS.toMillis(28);
     }
 
-    YamlDetectionConfigTranslator translator = this.translatorLoader.from(yamlConfig, this.provider);
+    YamlDetectionConfigTranslator translator;
+    try {
+      translator = this.translatorLoader.from(yamlConfig, this.provider);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Unable to instantiate the detection pipeline. Please verify the pipelineType.", e);
+    }
     return translator.withTuningWindow(tuningStartTime, tuningEndTime)
         .withExistingDetectionConfig(existingDetectionConfig)
         .generateDetectionConfig();
@@ -212,16 +213,11 @@ public class YamlResource {
     try {
       yamls = OBJECT_MAPPER.readValue(payload, Map.class);
 
-      if (StringUtils.isBlank(payload)) {
-        throw new ValidationException("The Yaml Payload in the request is empty.");
-      }
-
-      if (!yamls.containsKey("detection")) {
-        throw new ValidationException("Detection pipeline yaml is missing");
-      }
+      Preconditions.checkArgument(StringUtils.isNotBlank(payload), "The Yaml Payload in the request is empty.");
+      Preconditions.checkArgument(yamls.containsKey("detection"), "Detection pipeline yaml is missing");
 
       detectionConfigId = createDetectionPipeline(yamls.get("detection"), startTime, endTime);
-    } catch (ValidationException e) {
+    } catch (IllegalArgumentException e) {
       LOG.warn("Validation error while creating detection pipeline with payload " + payload, e);
       responseMessage.put("detectionMsg", "Validation Error! " + e.getMessage());
       return Response.status(Response.Status.BAD_REQUEST).entity(responseMessage).build();
@@ -235,16 +231,14 @@ public class YamlResource {
     // Notification
     long detectionAlertConfigId;
     try {
-      if (!yamls.containsKey("subscription")) {
-        throw new ValidationException("Subscription group yaml is missing.");
-      }
+      Preconditions.checkArgument(yamls.containsKey("subscription"), "Subscription group yaml is missing.");
 
       String subscriptionYaml = yamls.get("subscription");
       Map<String, Object> subscriptionYamlConfig;
       try {
         subscriptionYamlConfig = ConfigUtils.getMap(this.yaml.load(subscriptionYaml));
       } catch (Exception e) {
-        throw new ValidationException(e.getMessage());
+        throw new IllegalArgumentException(e.getMessage());
       }
 
       // Check if existing or new subscription group
@@ -256,7 +250,7 @@ public class YamlResource {
       } else {
         detectionAlertConfigId = createSubscriptionGroup(subscriptionYaml);
       }
-    } catch (ValidationException e) {
+    } catch (IllegalArgumentException e) {
       LOG.warn("Validation error while creating subscription group with payload " + payload, e);
       this.detectionConfigDAO.deleteById(detectionConfigId);
       responseMessage.put("subscriptionMsg", "Validation Error! " + e.getMessage());
@@ -280,30 +274,24 @@ public class YamlResource {
     ).build();
   }
 
-  long createDetectionPipeline(String yamlDetectionConfig, long startTime, long endTime) throws ValidationException {
-    DetectionConfigDTO detectionConfig;
-    try {
-      Preconditions.checkArgument(StringUtils.isNotBlank(yamlDetectionConfig), "The Yaml Payload in the request is empty.");
-      // Translate config from YAML to detection config (JSON)
-      Map<String, Object> newDetectionConfigMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
-      newDetectionConfigMap.putAll(ConfigUtils.getMap(this.yaml.load(yamlDetectionConfig)));
-      detectionConfig = buildDetectionConfigFromYaml(startTime, endTime, newDetectionConfigMap, null);
-      detectionConfig.setYaml(yamlDetectionConfig);
-    } catch (Exception e) {
-      throw new ValidationException(e.getMessage());
-    }
+  long createDetectionPipeline(String yamlDetectionConfig, long startTime, long endTime)
+      throws IllegalArgumentException {
+    Preconditions.checkArgument(StringUtils.isNotBlank(yamlDetectionConfig), "The Yaml Payload in the request is empty.");
+    // Translate config from YAML to detection config (JSON)
+    Map<String, Object> newDetectionConfigMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+    newDetectionConfigMap.putAll(ConfigUtils.getMap(this.yaml.load(yamlDetectionConfig)));
+    DetectionConfigDTO detectionConfig = buildDetectionConfigFromYaml(startTime, endTime, newDetectionConfigMap, null);
+    detectionConfig.setYaml(yamlDetectionConfig);
 
     // Check for duplicates
     List<DetectionConfigDTO> detectionConfigDTOS = detectionConfigDAO
         .findByPredicate(Predicate.EQ("name", detectionConfig.getName()));
-    if (!detectionConfigDTOS.isEmpty()) {
-      throw new ValidationException("Detection name is already taken. Please use a different detectionName.");
-    }
+    Preconditions.checkArgument(detectionConfigDTOS.isEmpty(),
+        "Detection name is already taken. Please use a different detectionName.");
 
     // Validate the detection config before saving it
     detectionValidator.validateConfig(detectionConfig);
     // Save the detection config
-    Preconditions.checkNotNull(detectionConfig);
     Long id = this.detectionConfigDAO.save(detectionConfig);
     Preconditions.checkNotNull(id, "Error while saving the detection pipeline");
 
@@ -329,7 +317,7 @@ public class YamlResource {
     long detectionConfigId;
     try {
       detectionConfigId = createDetectionPipeline(payload, startTime, endTime);
-    } catch (ValidationException e) {
+    } catch (IllegalArgumentException e) {
       LOG.warn("Validation error while creating detection pipeline with payload " + payload, e);
       responseMessage.put("message", "Validation Error! " + e.getMessage());
       return Response.status(Response.Status.BAD_REQUEST).entity(responseMessage).build();
@@ -347,26 +335,20 @@ public class YamlResource {
   }
 
   void updateDetectionPipeline(long detectionID, String yamlDetectionConfig, long startTime, long endTime)
-      throws ValidationException {
+      throws IllegalArgumentException {
     DetectionConfigDTO existingDetectionConfig = this.detectionConfigDAO.findById(detectionID);
     DetectionConfigDTO detectionConfig;
-    if (existingDetectionConfig == null) {
-      throw new ValidationException("Cannot find detection pipeline " + detectionID);
-    }
-    try {
-      Preconditions.checkArgument(StringUtils.isNotBlank(yamlDetectionConfig), "The Yaml Payload in the request is empty.");
-      // Translate config from YAML to detection config (JSON)
-      TreeMap<String, Object> newDetectionConfigMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
-      newDetectionConfigMap.putAll(ConfigUtils.getMap(this.yaml.load(yamlDetectionConfig)));
-      detectionConfig = buildDetectionConfigFromYaml(startTime, endTime, newDetectionConfigMap, existingDetectionConfig);
-      detectionConfig.setYaml(yamlDetectionConfig);
-    } catch (Exception e) {
-      throw new ValidationException(e.getMessage());
-    }
+    Preconditions.checkNotNull(existingDetectionConfig, "Cannot find detection pipeline " + detectionID);
+    Preconditions.checkArgument(StringUtils.isNotBlank(yamlDetectionConfig), "The Yaml Payload in the request is empty.");
+    // Translate config from YAML to detection config (JSON)
+    TreeMap<String, Object> newDetectionConfigMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+    newDetectionConfigMap.putAll(ConfigUtils.getMap(this.yaml.load(yamlDetectionConfig)));
+    detectionConfig = buildDetectionConfigFromYaml(startTime, endTime, newDetectionConfigMap, existingDetectionConfig);
+    detectionConfig.setYaml(yamlDetectionConfig);
+
     // Validate updated config before saving it
     detectionValidator.validateUpdatedConfig(detectionConfig, existingDetectionConfig);
     // Save the detection config
-    Preconditions.checkNotNull(detectionConfig);
     Long id = this.detectionConfigDAO.save(detectionConfig);
     Preconditions.checkNotNull(id, "Error while saving the detection pipeline");
   }
@@ -392,7 +374,7 @@ public class YamlResource {
     Map<String, String> responseMessage = new HashMap<>();
     try {
       updateDetectionPipeline(id, payload, startTime, endTime);
-    } catch (ValidationException e) {
+    } catch (IllegalArgumentException e) {
       LOG.warn("Validation error while creating detection pipeline with payload " + payload, e);
       responseMessage.put("message", "Validation Error! " + e.getMessage());
       return Response.status(Response.Status.BAD_REQUEST).entity(responseMessage).build();
@@ -409,8 +391,9 @@ public class YamlResource {
     return Response.ok().entity(responseMessage).build();
   }
 
-  long createSubscriptionGroup(String yamlAlertConfig) throws ValidationException {
-    Preconditions.checkArgument(StringUtils.isNotBlank(yamlAlertConfig), "The Yaml Payload in the request is empty.");
+  long createSubscriptionGroup(String yamlAlertConfig) throws IllegalArgumentException {
+    Preconditions.checkArgument(StringUtils.isNotBlank(yamlAlertConfig),
+        "The Yaml Payload in the request is empty.");
 
     // Translate config from YAML to detection alert config (JSON)
     TreeMap<String, Object> newAlertConfigMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
@@ -421,15 +404,13 @@ public class YamlResource {
     // Check for duplicates
     List<DetectionAlertConfigDTO> alertConfigDTOS = detectionAlertConfigDAO
         .findByPredicate(Predicate.EQ("name", alertConfig.getName()));
-    if (!alertConfigDTOS.isEmpty()) {
-      throw new ValidationException("Subscription group name is already taken. Please use a different name.");
-    }
+    Preconditions.checkArgument(alertConfigDTOS.isEmpty(),
+        "Subscription group name is already taken. Please use a different name.");
 
     // Validate the config before saving it
     subscriptionValidator.validateConfig(alertConfig);
 
     // Save the detection alert config
-    Preconditions.checkNotNull(alertConfig);
     Long id = this.detectionAlertConfigDAO.save(alertConfig);
     Preconditions.checkNotNull(id, "Error while saving the subscription group");
 
@@ -448,7 +429,7 @@ public class YamlResource {
     long detectionAlertConfigId;
     try {
       detectionAlertConfigId = createSubscriptionGroup(yamlAlertConfig);
-    } catch (ValidationException e) {
+    } catch (IllegalArgumentException e) {
       LOG.warn("Validation error while creating subscription group with payload " + yamlAlertConfig, e);
       responseMessage.put("message", "Validation Error! " + e.getMessage());
       return Response.serverError().entity(responseMessage).build();
@@ -487,7 +468,7 @@ public class YamlResource {
     return oldAlertConfig;
   }
 
-  void updateSubscriptionGroup(long oldAlertConfigID, String yamlAlertConfig) throws ValidationException {
+  void updateSubscriptionGroup(long oldAlertConfigID, String yamlAlertConfig) {
     DetectionAlertConfigDTO oldAlertConfig = this.detectionAlertConfigDAO.findById(oldAlertConfigID);
     if (oldAlertConfig == null) {
       throw new RuntimeException("Cannot find subscription group " + oldAlertConfigID);
@@ -540,7 +521,7 @@ public class YamlResource {
     Map<String, String> responseMessage = new HashMap<>();
     try {
       updateSubscriptionGroup(id, yamlAlertConfig);
-    } catch (ValidationException e) {
+    } catch (IllegalArgumentException e) {
       LOG.warn("Validation error while updating subscription group " + id + " with payload " + yamlAlertConfig, e);
       responseMessage.put("message", "Validation Error! " + e.getMessage());
       return Response.serverError().entity(responseMessage).build();
@@ -611,7 +592,7 @@ public class YamlResource {
       DetectionPipeline pipeline = this.loader.from(this.provider, detectionConfig, start, end);
       result = pipeline.run();
 
-    } catch (ValidationException e) {
+    } catch (IllegalArgumentException e) {
       LOG.warn("Validation error while running preview with payload  " + payload, e);
       responseMessage.put("message", "Validation Error! " + e.getMessage());
       return Response.serverError().entity(responseMessage).build();
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
index 496f7c9..7f0587f 100644
--- 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
@@ -3,11 +3,13 @@ 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) {
+  public MockYamlDetectionConfigTranslator(Map<String, Object> yamlConfig, DataProvider provider, DetectionConfigValidator validator) {
     super(yamlConfig, provider);
+    super.detectionValidator = validator;
   }
 
   @Override
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
index ebdfd21..6b8c7cb 100644
--- 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
@@ -8,11 +8,14 @@ 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;
@@ -32,7 +35,10 @@ public class YamlDetectionConfigTranslatorTest {
     metricConfigDTO.setAlias("alias");
     this.metricDAO.save(metricConfigDTO);
 
-    YamlDetectionConfigTranslator translator = new MockYamlDetectionConfigTranslator(yamlConfigs, new MockDataProvider());
+    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");


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