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/06/04 22:54:43 UTC

[incubator-pinot] branch master updated: Extend Detection Config Validator to validate composite(entity) alerts (#4271)

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 9129c06  Extend Detection Config Validator to validate composite(entity) alerts  (#4271)
9129c06 is described below

commit 9129c062bc88f66ab36dd50121118332735705dc
Author: Akshay Rai <ak...@gmail.com>
AuthorDate: Tue Jun 4 15:54:37 2019 -0700

    Extend Detection Config Validator to validate composite(entity) alerts  (#4271)
---
 .../validators/DetectionConfigValidator.java       | 152 +++++++++++----
 .../validators/DetectionConfigValidatorTest.java   | 211 +++++++++++++++++++++
 .../validators/entity-pipeline-config-1.yaml       |  44 +++++
 .../validators/entity-pipeline-config-2.yaml       |  68 +++++++
 4 files changed, 440 insertions(+), 35 deletions(-)

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 d34da6f..77c1cdf 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,6 +20,7 @@
 package org.apache.pinot.thirdeye.detection.validators;
 
 import com.google.common.base.Preconditions;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -49,12 +50,19 @@ public class DetectionConfigValidator implements ConfigValidator<DetectionConfig
   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_ALERTS = "alerts";
   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";
 
+  private static final String LEGACY_METRIC_ALERT = "COMPOSITE";
+  private static final String METRIC_ALERT = "METRIC_ALERT";
+  private static final String COMPOSITE_ALERT = "COMPOSITE_ALERT";
+  private static final Set<String> SUPPORTED_ALERT_TYPES = new HashSet<>(
+      Arrays.asList(LEGACY_METRIC_ALERT, METRIC_ALERT, COMPOSITE_ALERT));
+
   public DetectionConfigValidator(DataProvider provider) {
     this.provider = provider;
     this.loader = new DetectionPipelineLoader();
@@ -104,68 +112,84 @@ public class DetectionConfigValidator implements ConfigValidator<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) {
+  private void validateRule(String alertName, 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);
+        "Missing property " + ruleType + " (" + ruleType + ") for sub-alert " + alertName + " rule no. " + ruleIndex);
+
+    Preconditions.checkArgument(ruleYaml.containsKey(PROP_NAME),
+        "Missing property " + ruleType + " (" + PROP_NAME + ") for sub-alert " + alertName + " rule no. " + ruleIndex);
     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 \':\'");
+        "Duplicate rule name (" + name + ") found for sub-alert " + alertName + " rule no. " + ruleIndex + ". Names have to be unique within a config.");
+
+    Preconditions.checkArgument(!name.contains(":"),
+        "Illegal character (:) found in " + ruleType + " (" + PROP_NAME + ") for sub-alert " + alertName + " rule no. " + ruleIndex);
   }
 
-  /**
-   * Validate the the detection yaml configuration.
-   *
-   * @param detectionYaml the detection yaml configuration to be validated
-   */
-  @Override
-  public void validateYaml(Map<String, Object> detectionYaml) {
+  private void validateBasicAttributes(Map<String, Object> detectionYaml, String parentAlertName) {
+    Preconditions.checkArgument(detectionYaml.containsKey(PROP_NAME),
+        "Missing property ( " + PROP_NAME + " ) in one of the sub-alerts under " + parentAlertName);
+    String alertName = MapUtils.getString(detectionYaml, PROP_NAME);
+
+    Preconditions.checkArgument(detectionYaml.containsKey(PROP_TYPE),
+        "Missing property ( " + PROP_TYPE + " ) in sub-alert " + alertName);
+    String alertType = MapUtils.getString(detectionYaml, PROP_TYPE);
+
+    Preconditions.checkArgument(SUPPORTED_ALERT_TYPES.contains(alertType),
+        "Unsupported type (" + alertType + ") in sub-alert " + alertName);
+  }
+
+  private void validateMetricAlertConfig(Map<String, Object> detectionYaml, String parentAlertName)
+      throws IllegalArgumentException {
+    validateBasicAttributes(detectionYaml, parentAlertName);
+    String alertName = MapUtils.getString(detectionYaml, PROP_NAME);
+
     // 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);
+    Preconditions.checkArgument(detectionYaml.containsKey(PROP_METRIC),
+        "Missing property (" + PROP_METRIC + ") in sub-alert " + alertName);
+    Preconditions.checkArgument(detectionYaml.containsKey(PROP_DATASET),
+        "Missing property (" + PROP_DATASET + ") in sub-alert " + alertName);
+    Preconditions.checkArgument(detectionYaml.containsKey(PROP_RULES),
+        "Missing property (" + PROP_RULES + ") in sub-alert " + alertName);
 
-    // 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.");
+    // Validate fields which shouldn't be defined at this level
+    Preconditions.checkArgument(!detectionYaml.containsKey(PROP_FILTER),
+        "For sub-alert " + alertName + ", 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());
+    Preconditions.checkArgument(metricConfig != null,
+        "Invalid metric (not found) in sub-alert " + alertName);
 
     // 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.");
+        "Multiple groupers detected for metric in sub-alert " + alertName);
 
     // 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);
+    for (int ruleIndex = 1; ruleIndex <= ruleYamls.size(); ruleIndex++) {
+      Map<String, Object> ruleYaml = ruleYamls.get(ruleIndex - 1);
 
       // Validate detection rules
-      Preconditions.checkArgument(ruleYaml.containsKey(PROP_DETECTION), "In rule no." + (i) + ", detection rule is missing.");
+      Preconditions.checkArgument(ruleYaml.containsKey(PROP_DETECTION),
+          "Detection rule missing for sub-alert " + alertName + " rule no. " + ruleIndex);
       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));
+        validateRule(alertName, detectionRuleYaml, ruleIndex, "detection", names);
+        names.add(MapUtils.getString(detectionRuleYaml, 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));
+          validateRule(alertName, filterRuleYaml, ruleIndex, "filter", names);
+          names.add(MapUtils.getString(filterRuleYaml, PROP_NAME));
         }
       }
     }
@@ -173,11 +197,69 @@ public class DetectionConfigValidator implements ConfigValidator<DetectionConfig
     // 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(),
+      DatasetConfigDTO datasetConfig = provider
+          .fetchDatasets(Collections.singletonList(metricConfig.getDataset()))
+          .get(metricConfig.getDataset());
+      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.");
     }
   }
 
+  private void validateCompositeAlertConfig(Map<String, Object> detectionYaml, String parentAlertName) throws IllegalArgumentException {
+    validateBasicAttributes(detectionYaml, parentAlertName);
+    String alertName = MapUtils.getString(detectionYaml, PROP_NAME);
+
+    // Validate all compulsory fields
+    Preconditions.checkArgument(detectionYaml.containsKey(PROP_ALERTS),
+        "Missing property (" + PROP_ALERTS + ") in alert/sub-alert " + alertName);
+
+    // validate all the sub-alerts depending on the type (METRIC_ALERT OR COMPOSITE_ALERT)
+    List<Map<String, Object>> subDetectionYamls = ConfigUtils.getList(detectionYaml.get(PROP_ALERTS));
+    for (Map<String, Object> subDetectionYaml : subDetectionYamls) {
+      if (subDetectionYaml.containsKey(PROP_TYPE) && subDetectionYaml.get(PROP_TYPE).equals(COMPOSITE_ALERT)) {
+        validateCompositeAlertConfig(subDetectionYaml, alertName);
+      } else {
+        validateMetricAlertConfig(subDetectionYaml, alertName);
+      }
+    }
+
+    // TODO: Add more validations
+    // Validate Alert Condition
+    // Validate Merger
+  }
+
+  /**
+   * Validate the the detection yaml configuration.
+   *
+   * @param detectionYaml the detection yaml configuration to be validated
+   */
+  @Override
+  public void validateYaml(Map<String, Object> detectionYaml) throws IllegalArgumentException {
+    // Validate detectionName
+    Preconditions.checkArgument(detectionYaml.containsKey(PROP_DETECTION_NAME), "Property missing: " + PROP_DETECTION_NAME);
+    String alertName = MapUtils.getString(detectionYaml, PROP_DETECTION_NAME);
+
+    // Hack to support 'detectionName' attribute at root level and 'name' attribute elsewhere
+    // We consistently use 'name' as a convention to define the sub-alerts. However, at the root
+    // level, as a convention, we will use 'detectionName' which defines the name of the complete alert.
+    detectionYaml.put(PROP_NAME, alertName);
+
+    // By default if 'type' is not specified, we assume it as a METRIC_ALERT
+    if (!detectionYaml.containsKey(PROP_TYPE)) {
+      detectionYaml.put(PROP_TYPE, METRIC_ALERT);
+    }
+
+    // Validate config depending on the type (METRIC_ALERT OR COMPOSITE_ALERT)
+    if (detectionYaml.get(PROP_TYPE).equals(COMPOSITE_ALERT)) {
+      validateCompositeAlertConfig(detectionYaml, alertName);
+    } else {
+      // The legacy type 'COMPOSITE' will be treated as a metric alert along with the new convention METRIC_ALERT.
+      // This is applicable only at the root level to maintain backward compatibility.
+      validateMetricAlertConfig(detectionYaml, alertName);
+    }
+  }
+
   /**
    * Perform validation on the updated detection config. Check for fields which shouldn't be
    * updated by the user.
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidatorTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidatorTest.java
new file mode 100644
index 0000000..d6084ac
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidatorTest.java
@@ -0,0 +1,211 @@
+/*
+ * 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.validators;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.pinot.thirdeye.datalayer.bao.DAOTestBase;
+import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
+import org.apache.pinot.thirdeye.detection.ConfigUtils;
+import org.apache.pinot.thirdeye.detection.DataProvider;
+import org.apache.pinot.thirdeye.detection.MockDataProvider;
+import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionRegistry;
+import org.apache.pinot.thirdeye.detection.components.MockGrouper;
+import org.apache.pinot.thirdeye.detection.components.RuleBaselineProvider;
+import org.apache.pinot.thirdeye.detection.components.ThresholdRuleAnomalyFilter;
+import org.apache.pinot.thirdeye.detection.components.ThresholdRuleDetector;
+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;
+
+
+public class DetectionConfigValidatorTest {
+
+  private Yaml yaml;
+  private Map<String, Object> yamlConfig1;
+  private Map<String, Object> yamlConfig2;
+  private DataProvider provider;
+  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() {
+    MetricConfigDTO metricConfig = new MetricConfigDTO();
+    metricConfig.setAlias("alias");
+    metricConfig.setName("test_metric");
+    metricConfig.setDataset("test_dataset");
+    metricConfig.setId(1L);
+    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");
+    DetectionRegistry.registerComponent(RuleBaselineProvider.class.getName(), "RULE_BASELINE");
+    DetectionRegistry.registerComponent(MockGrouper.class.getName(), "MOCK_GROUPER");
+    this.provider = new MockDataProvider().setMetrics(Collections.singletonList(metricConfig)).setDatasets(Collections.singletonList(datasetConfigDTO));
+
+    this.yamlConfig1 = (Map<String, Object>) this.yaml.load(this.getClass().getResourceAsStream("entity-pipeline-config-1.yaml"));
+    this.yamlConfig2 = (Map<String, Object>) this.yaml.load(this.getClass().getResourceAsStream("entity-pipeline-config-2.yaml"));
+  }
+
+  @Test
+  public void testMetricAlertValidation() {
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig1);
+  }
+
+  // detectionName is a compulsory field at root level
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testNoDetectionNameValidation() {
+    this.yamlConfig1.remove("detectionName");
+
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig1);
+  }
+
+  // If type is not specified at root level, then assume it to be METRIC_ALERT and proceed
+  @Test
+  public void testNoTypeValidation() {
+    this.yamlConfig1.remove("type");
+
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig1);
+  }
+
+  // Validates if the type field is acceptable
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testInvalidTypeValidation() {
+    this.yamlConfig1.put("type", "INVALID_TYPE");
+
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig1);
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testNoMetricValidation() {
+    this.yamlConfig1.remove("metric");
+
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig1);
+  }
+
+  // Every METRIC_ALERT must have rules
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testNoRulesValidation() {
+    this.yamlConfig1.remove("rules");
+
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig1);
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testNonExistingMetricValidation() {
+    this.yamlConfig1.put("metric", "metric_doesnt_exist");
+
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig1);
+  }
+
+  // Every rule must have at least 1 detection
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testNoDetectionUnderRuleValidation() {
+    ((Map<String, Object>) ConfigUtils.getList(this.yamlConfig1.get("rules")).get(0)).remove("detection");
+
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig1);
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testNoDetectionTypeUnderRuleValidation() {
+    ((Map<String, Object>) ConfigUtils.getList(((Map<String, Object>) ConfigUtils.getList(this.yamlConfig1.get("rules")).get(0))
+        .get("detection")).get(0)).remove("type");
+
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig1);
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testNoDetectionNameUnderRuleValidation() {
+    ((Map<String, Object>) ConfigUtils.getList(((Map<String, Object>) ConfigUtils.getList(this.yamlConfig1.get("rules")).get(0))
+        .get("detection")).get(0)).remove("name");
+
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig1);
+  }
+
+  // Validate if detection/filter names are unique
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testDuplicateDetectionNameUnderRuleValidation() {
+    // Update rule 1 detection name to that of rule 2 detection name
+    ((Map<String, Object>) ConfigUtils.getList(
+        ((Map<String, Object>) ConfigUtils.getList(this.yamlConfig1.get("rules")).get(0)).get("detection")
+    ).get(0)).put("name", "maxThreshold_2");
+
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig1);
+  }
+
+  // Entity Example 2 Validations
+
+  @Test
+  public void testEntityAlertValidation() {
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig2);
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testNoAlertsValidation() {
+    this.yamlConfig2.remove("alerts");
+
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig2);
+  }
+
+  // type is a compulsory field at sub-alert levels
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testNoTypeUnderAlertsValidation() {
+    ((Map<String, Object>) ConfigUtils.getList(this.yamlConfig2.get("alerts")).get(0)).remove("type");
+
+    DetectionConfigValidator detectionConfigValidator = new DetectionConfigValidator(provider);
+    detectionConfigValidator.validateYaml(this.yamlConfig2);
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-1.yaml b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-1.yaml
new file mode 100644
index 0000000..70fc56b
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-1.yaml
@@ -0,0 +1,44 @@
+detectionName: testPipeline
+description: My test pipeline
+type: METRIC_ALERT
+metric: test_metric
+dataset: test_dataset
+filters:
+  D1:
+    - v1
+    - v2
+  D2:
+    - v3
+dimensionExploration:
+  dimensions:
+    - D1
+    - D2
+  minContribution: 0.05
+rules:
+  - detection:
+      - type: THRESHOLD
+        name: maxThreshold_1
+        params:
+          max: 100
+    filter:
+      - type: THRESHOLD_RULE_FILTER
+        name: thresholdFilter_1
+        params:
+          min: 50
+      - type: THRESHOLD_RULE_FILTER
+        name: thresholdFilter_2
+        params:
+          min: 50
+  - detection:
+      - type: THRESHOLD
+        name: maxThreshold_2
+        params:
+          max: 100
+    filter:
+      - type: THRESHOLD_RULE_FILTER
+        name: thresholdFilter_3
+        params:
+          min: 50
+merger:
+  maxGap: 0
+  maxDuration: 100000000
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-2.yaml b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-2.yaml
new file mode 100644
index 0000000..bba31ed
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-2.yaml
@@ -0,0 +1,68 @@
+detectionName: testPipeline
+description: My test pipeline
+type: COMPOSITE_ALERT
+
+alerts:
+  - type: METRIC_ALERT
+    name: metric alert on test_metric
+    metric: test_metric
+    dataset: test_dataset
+    filters:
+      D1:
+        - v1
+        - v2
+      D2:
+        - v3
+    dimensionExploration:
+      dimensions:
+        - D1
+        - D2
+      minContribution: 0.05
+    rules:
+      - detection:
+          - type: THRESHOLD
+            name: maxThreshold_1
+            params:
+              max: 100
+        filter:
+          - type: THRESHOLD_RULE_FILTER
+            name: thresholdFilter_1
+            params:
+              min: 50
+          - type: THRESHOLD_RULE_FILTER
+            name: thresholdFilter_2
+            params:
+              min: 100
+      - detection:
+          - type: THRESHOLD
+            name: maxThreshold_2
+            params:
+              max: 100
+        filter:
+          - type: THRESHOLD_RULE_FILTER
+            name: thresholdFilter_3
+            params:
+              min: 50
+  - type: COMPOSITE_ALERT
+    name: composite alert on entity
+    alerts:
+      - type: METRIC_ALERT
+        name: metric alert on test_metric
+        metric: test_metric
+        dataset: test_dataset
+        rules:
+          - detection:
+              - type: THRESHOLD
+                name: maxThreshold_1
+                params:
+                  max: 100
+      - type: METRIC_ALERT
+        name: metric alert on test_metric
+        metric: test_metric
+        dataset: test_dataset
+        rules:
+          - detection:
+              - type: THRESHOLD
+                name: maxThreshold_1
+                params:
+                  max: 100


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