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 2018/12/11 01:54:47 UTC

[incubator-pinot] branch master updated: [TE] Endpoints for create and edit alert yaml along with validators (#3603)

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 6a6c175  [TE] Endpoints for create and edit alert yaml along with validators (#3603)
6a6c175 is described below

commit 6a6c1758aae34239c1e1baf13978de0e6b7b7005
Author: Akshay Rai <ak...@gmail.com>
AuthorDate: Mon Dec 10 17:54:43 2018 -0800

    [TE] Endpoints for create and edit alert yaml along with validators (#3603)
---
 .../datalayer/pojo/DetectionAlertConfigBean.java   |  16 +-
 .../detection/validators/ConfigValidator.java      |  48 +++++
 .../validators/DetectionAlertConfigValidator.java  | 109 +++++++++++
 .../yaml/YamlDetectionAlertConfigTranslator.java   |  20 ++-
 .../thirdeye/detection/yaml/YamlResource.java      | 199 ++++++++++++++++-----
 .../YamlDetectionAlertConfigTranslatorTest.java    |   2 +-
 .../thirdeye/detection/yaml/YamlResourceTest.java  | 140 +++++++++++++++
 .../detection/yaml/alertconfig/alert-config-1.yaml |  40 +++++
 .../detection/yaml/alertconfig/alert-config-2.yaml |  42 +++++
 .../detection/yaml/alertconfig/alert-config-3.yaml |  42 +++++
 .../detection/yaml/alertconfig/alert-config-4.yaml |  42 +++++
 11 files changed, 648 insertions(+), 52 deletions(-)

diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/datalayer/pojo/DetectionAlertConfigBean.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/datalayer/pojo/DetectionAlertConfigBean.java
index 07984f2..902e635 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/datalayer/pojo/DetectionAlertConfigBean.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/datalayer/pojo/DetectionAlertConfigBean.java
@@ -36,6 +36,7 @@ public class DetectionAlertConfigBean extends AbstractBean {
   String from;
   String cronExpression;
   String application;
+  String yaml;
   boolean onlyFetchLegacyAnomalies;
 
   Map<String, Map<String, Object>> alertSchemes;
@@ -153,6 +154,14 @@ public class DetectionAlertConfigBean extends AbstractBean {
     this.refLinks = refLinks;
   }
 
+  public String getYaml() {
+    return yaml;
+  }
+
+  public void setYaml(String yaml) {
+    this.yaml = yaml;
+  }
+
   @Override
   public boolean equals(Object o) {
     if (this == o) {
@@ -165,12 +174,15 @@ public class DetectionAlertConfigBean extends AbstractBean {
     return active == that.active && Objects.equals(name, that.name) && Objects.equals(from, that.from)
         && Objects.equals(cronExpression, that.cronExpression) && Objects.equals(application, that.application)
         && subjectType == that.subjectType && Objects.equals(vectorClocks, that.vectorClocks) && Objects.equals(
-        highWaterMark, that.highWaterMark) && Objects.equals(properties, that.properties);
+        highWaterMark, that.highWaterMark) && Objects.equals(properties, that.properties)
+        && Objects.equals(alertSchemes, that.alertSchemes) && Objects.equals(alertSuppressors, that.alertSuppressors)
+        && Objects.equals(refLinks, that.refLinks) && onlyFetchLegacyAnomalies == that.onlyFetchLegacyAnomalies
+        && Objects.equals(yaml, that.yaml);
   }
 
   @Override
   public int hashCode() {
     return Objects.hash(active, name, from, cronExpression, application, subjectType, vectorClocks,
-        highWaterMark, properties);
+        highWaterMark, properties, alertSchemes, alertSuppressors, refLinks, onlyFetchLegacyAnomalies, yaml);
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/validators/ConfigValidator.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/validators/ConfigValidator.java
new file mode 100644
index 0000000..6720722
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/validators/ConfigValidator.java
@@ -0,0 +1,48 @@
+package com.linkedin.thirdeye.detection.validators;
+
+import com.linkedin.thirdeye.datalayer.bao.AlertConfigManager;
+import com.linkedin.thirdeye.datalayer.bao.ApplicationManager;
+import com.linkedin.thirdeye.datasource.DAORegistry;
+import java.util.Map;
+import org.apache.commons.lang.StringUtils;
+import org.yaml.snakeyaml.Yaml;
+
+
+public abstract class ConfigValidator {
+
+  protected static final Yaml YAML = new Yaml();
+
+  final AlertConfigManager alertDAO;
+  final ApplicationManager applicationDAO;
+
+  ConfigValidator() {
+    this.alertDAO = DAORegistry.getInstance().getAlertConfigDAO();
+    this.applicationDAO = DAORegistry.getInstance().getApplicationDAO();
+  }
+
+  /**
+   * Perform basic validations on a yaml file like verifying if
+   * the yaml exists and is parsable
+   */
+  @SuppressWarnings("unchecked")
+  public boolean validateYAMLConfig(String yamlConfig, Map<String, String> responseMessage) {
+    // Check if YAML is empty or not
+    if (StringUtils.isEmpty(yamlConfig)) {
+      responseMessage.put("message", "The config file cannot be blank.");
+      responseMessage.put("more-info", "Payload in the request is empty");
+      return false;
+    }
+
+    // Check if the YAML is parsable
+    try {
+      Map<String, Object> yamlConfigMap = (Map<String, Object>) YAML.load(yamlConfig);
+    } catch (Exception e) {
+      responseMessage.put("message", "There was an error parsing the yaml file. Check for syntax issues.");
+      responseMessage.put("more-info", "Error parsing YAML" + e);
+      return false;
+    }
+
+    return true;
+  }
+
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/validators/DetectionAlertConfigValidator.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/validators/DetectionAlertConfigValidator.java
new file mode 100644
index 0000000..2753d7a
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/validators/DetectionAlertConfigValidator.java
@@ -0,0 +1,109 @@
+package com.linkedin.thirdeye.detection.validators;
+
+import com.linkedin.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.lang.StringUtils;
+
+import static com.linkedin.thirdeye.detection.yaml.YamlDetectionAlertConfigTranslator.*;
+
+
+public class DetectionAlertConfigValidator extends ConfigValidator {
+
+  private static final DetectionAlertConfigValidator INSTANCE = new DetectionAlertConfigValidator();
+
+  public static DetectionAlertConfigValidator getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Perform validation on the alert config like verifying if all the required fields are set
+   */
+  @SuppressWarnings("unchecked")
+  public boolean validateConfig(DetectionAlertConfigDTO alertConfig,  Map<String, String> responseMessage) {
+    boolean isValid = true;
+
+    // Check for all the required fields in the alert
+    if (StringUtils.isEmpty(alertConfig.getName())) {
+      responseMessage.put("message", "Subscription group name field cannot be left empty.");
+      return false;
+    }
+    if (StringUtils.isEmpty(alertConfig.getApplication())) {
+      responseMessage.put("message", "Application field cannot be left empty");
+      return false;
+    }
+    if (StringUtils.isEmpty(alertConfig.getFrom())) {
+      responseMessage.put("message", "From address field cannot be left empty");
+      return false;
+    }
+
+    // At least one alertScheme is required
+    if (alertConfig.getAlertSchemes() == null || alertConfig.getAlertSchemes().size() == 0) {
+      responseMessage.put("message", "Alert scheme cannot be left empty");
+      return false;
+    }
+    // Properties cannot be empty
+    if (alertConfig.getProperties() == null || alertConfig.getProperties().isEmpty()) {
+      responseMessage.put("message", "Alert properties cannot be left empty. Please specify the recipients,"
+          + " detection ids, and type.");
+      return false;
+    }
+    // detectionConfigIds cannot be empty
+    List<Long> detectionIds = (List<Long>) alertConfig.getProperties().get(PROP_DETECTION_CONFIG_IDS);
+    if (detectionIds == null || detectionIds.isEmpty()) {
+      responseMessage.put("message", "A notification group should subscribe to at least one alert. If you wish to"
+          + " unsubscribe, set active to false.");
+      return false;
+    }
+    // At least one recipient must be specified
+    Map<String, Object> recipients = (Map<String, Object>) alertConfig.getProperties().get(PROP_RECIPIENTS);
+    if (recipients == null || recipients.isEmpty() || ((List<String>) recipients.get("to")).isEmpty()) {
+      responseMessage.put("message", "Please specify at least one recipient in the notification group. If you wish to"
+          + " unsubscribe, set active to false.");
+      return false;
+    }
+    // Application name should be valid
+    if (super.applicationDAO.findByName(alertConfig.getApplication()).size() == 0) {
+      responseMessage.put("message", "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.");
+      return false;
+    }
+
+    // TODO add more checks like cron validity, email validity, alert type check, scheme type check etc.
+
+    return isValid;
+  }
+
+  /**
+   * Perform validation on the updated alert config. Check for fields which shouldn't be
+   * updated by the user.
+   */
+  @SuppressWarnings("unchecked")
+  public boolean validateUpdatedConfig(DetectionAlertConfigDTO updatedAlertConfig,
+      DetectionAlertConfigDTO oldAlertConfig, Map<String, String> responseMessage) {
+    boolean isValid = true;
+
+    if (!validateConfig(updatedAlertConfig, responseMessage)) {
+      isValid = false;
+    }
+
+    Long newHighWatermark = updatedAlertConfig.getHighWaterMark();
+    Long oldHighWatermark = oldAlertConfig.getHighWaterMark();
+    if (newHighWatermark != null && oldHighWatermark != null && newHighWatermark.longValue() != oldHighWatermark) {
+      responseMessage.put("message", "HighWaterMark has been modified. This is not allowed");
+      isValid = false;
+    }
+    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()) {
+          responseMessage.put("message", "Vector clock has been modified. This is not allowed.");
+          isValid = false;
+        }
+      }
+    }
+
+    return isValid;
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslator.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslator.java
index c47d6a7..db87d80 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslator.java
@@ -22,6 +22,9 @@ import org.apache.commons.collections.MapUtils;
  * The translator converts the alert yaml config into a detection alert config
  */
 public class YamlDetectionAlertConfigTranslator {
+  public static final String PROP_DETECTION_CONFIG_IDS = "detectionConfigIds";
+  public static final String PROP_RECIPIENTS = "recipients";
+
   static final String PROP_SUBS_GROUP_NAME = "subscriptionGroupName";
   static final String PROP_CRON = "cron";
   static final String PROP_ACTIVE = "active";
@@ -29,11 +32,9 @@ public class YamlDetectionAlertConfigTranslator {
   static final String PROP_FROM = "fromAddress";
   static final String PROP_ONLY_FETCH_LEGACY_ANOMALIES = "onlyFetchLegacyAnomalies";
   static final String PROP_EMAIL_SUBJECT_TYPE = "emailSubjectStyle";
-  static final String PROP_DETECTION_CONFIG_IDS = "detectionConfigIds";
   static final String PROP_ALERT_SCHEMES = "alertSchemes";
   static final String PROP_ALERT_SUPPRESSORS = "alertSuppressors";
   static final String PROP_REFERENCE_LINKS = "referenceLinks";
-  static final String PROP_RECIPIENTS = "recipients";
 
   static final String PROP_TYPE = "type";
   static final String PROP_CLASS_NAME = "className";
@@ -176,8 +177,13 @@ public class YamlDetectionAlertConfigTranslator {
     alertConfigDTO.setOnlyFetchLegacyAnomalies(MapUtils.getBooleanValue(yamlAlertConfig, PROP_ONLY_FETCH_LEGACY_ANOMALIES, false));
     alertConfigDTO.setSubjectType((AlertConfigBean.SubjectType) MapUtils.getObject(yamlAlertConfig, PROP_EMAIL_SUBJECT_TYPE, AlertConfigBean.SubjectType.METRICS));
 
-    MapUtils.getMap(yamlAlertConfig, PROP_REFERENCE_LINKS).put("ThirdEye User Guide", "https://go/thirdeyeuserguide");
-    MapUtils.getMap(yamlAlertConfig, PROP_REFERENCE_LINKS).put("Add Reference Links", "https://go/thirdeyealertreflink");
+    Map<String, String> refLinks = MapUtils.getMap(yamlAlertConfig, PROP_REFERENCE_LINKS);
+    if (refLinks == null) {
+      refLinks = new HashMap<>();
+      yamlAlertConfig.put(PROP_REFERENCE_LINKS, refLinks);
+    }
+    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.setAlertSchemes(buildAlertSchemes(yamlAlertConfig));
@@ -185,10 +191,10 @@ public class YamlDetectionAlertConfigTranslator {
     alertConfigDTO.setProperties(buildAlerterProperties(yamlAlertConfig));
 
     // NOTE: The below fields will/should be hidden from the YAML/UI. They will only be updated by the backend pipeline.
-    List<Long> detectionConfigIds = ConfigUtils.getList(yamlAlertConfig.get(PROP_DETECTION_CONFIG_IDS));
+    List<Integer> detectionConfigIds = ConfigUtils.getList(yamlAlertConfig.get(PROP_DETECTION_CONFIG_IDS));
     Map<Long, Long> vectorClocks = new HashMap<>();
-    for (long detectionConfigId : detectionConfigIds) {
-      vectorClocks.put(detectionConfigId, 0L);
+    for (int detectionConfigId : detectionConfigIds) {
+      vectorClocks.put((long) detectionConfigId, 0L);
     }
     alertConfigDTO.setHighWaterMark(0L);
     alertConfigDTO.setVectorClocks(vectorClocks);
diff --git a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/yaml/YamlResource.java b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/yaml/YamlResource.java
index 6ffd36a..109da58 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/yaml/YamlResource.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/com/linkedin/thirdeye/detection/yaml/YamlResource.java
@@ -18,21 +18,20 @@ import com.linkedin.thirdeye.datasource.loader.AggregationLoader;
 import com.linkedin.thirdeye.datasource.loader.DefaultAggregationLoader;
 import com.linkedin.thirdeye.datasource.loader.DefaultTimeSeriesLoader;
 import com.linkedin.thirdeye.datasource.loader.TimeSeriesLoader;
-import com.linkedin.thirdeye.detection.ConfigUtils;
 import com.linkedin.thirdeye.detection.DataProvider;
 import com.linkedin.thirdeye.detection.DefaultDataProvider;
 import com.linkedin.thirdeye.detection.DetectionPipelineLoader;
 import com.wordnik.swagger.annotations.Api;
 import com.wordnik.swagger.annotations.ApiOperation;
+import com.linkedin.thirdeye.detection.validators.DetectionAlertConfigValidator;
 import com.wordnik.swagger.annotations.ApiParam;
 import java.lang.reflect.InvocationTargetException;
 import java.util.ArrayList;
 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 java.util.TreeMap;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
@@ -55,15 +54,15 @@ import org.yaml.snakeyaml.Yaml;
 public class YamlResource {
   protected static final Logger LOG = LoggerFactory.getLogger(YamlResource.class);
 
-  private static final String PROP_NAME = "detectionName";
-  private static final String PROP_TYPE = "type";
-  private static final String PROP_DETECTION_CONFIG_ID = "detectionConfigIds";
+  public static final String PROP_SUBS_GROUP_NAME = "subscriptionGroupName";
+  public static final String PROP_DETECTION_NAME = "detectionName";
 
 
   private final DetectionConfigManager detectionConfigDAO;
   private final DetectionAlertConfigManager detectionAlertConfigDAO;
   private final YamlDetectionTranslatorLoader translatorLoader;
   private final YamlDetectionAlertConfigTranslator alertConfigTranslator;
+  private final DetectionAlertConfigValidator alertValidator;
   private final DataProvider provider;
   private final MetricConfigManager metricDAO;
   private final DatasetConfigManager datasetDAO;
@@ -76,6 +75,7 @@ public class YamlResource {
     this.detectionConfigDAO = DAORegistry.getInstance().getDetectionConfigManager();
     this.detectionAlertConfigDAO = DAORegistry.getInstance().getDetectionAlertConfigManager();
     this.translatorLoader = new YamlDetectionTranslatorLoader();
+    this.alertValidator = DetectionAlertConfigValidator.getInstance();
     this.alertConfigTranslator = YamlDetectionAlertConfigTranslator.getInstance();
     this.metricDAO = DAORegistry.getInstance().getMetricConfigDAO();
     this.datasetDAO = DAORegistry.getInstance().getDatasetConfigDAO();
@@ -117,7 +117,7 @@ public class YamlResource {
 
       // retrieve id if detection config already exists
       List<DetectionConfigDTO> detectionConfigDTOs =
-          this.detectionConfigDAO.findByPredicate(Predicate.EQ("name", MapUtils.getString(yamlConfig, PROP_NAME)));
+          this.detectionConfigDAO.findByPredicate(Predicate.EQ("name", MapUtils.getString(yamlConfig, PROP_DETECTION_NAME)));
       DetectionConfigDTO existingDetectionConfig = null;
       if (!detectionConfigDTOs.isEmpty()) {
         existingDetectionConfig = detectionConfigDTOs.get(0);
@@ -191,6 +191,38 @@ public class YamlResource {
     return Response.status(400).entity(ImmutableMap.of("status", "400", "message", errorMessage)).build();
   }
 
+  @POST
+  @Path("/notification")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Consumes(MediaType.TEXT_PLAIN)
+  @ApiOperation("Create a notification group using a YAML config")
+  @SuppressWarnings("unchecked")
+  public Response createDetectionAlertConfig(
+      @ApiParam("payload") String yamlAlertConfig) {
+    Map<String, String> responseMessage = new HashMap<>();
+    Long detectionAlertConfigId;
+    try {
+      DetectionAlertConfigDTO alertConfig = createDetectionAlertConfig(yamlAlertConfig, responseMessage);
+      if (alertConfig == null) {
+        return Response.status(Response.Status.BAD_REQUEST).entity(responseMessage).build();
+      }
+
+      detectionAlertConfigId = this.detectionAlertConfigDAO.save(alertConfig);
+      if (detectionAlertConfigId == null) {
+        responseMessage.put("message", "Failed to save the detection alert config.");
+        responseMessage.put("more-info", "Check for potential DB issues. YAML alert config = " + yamlAlertConfig);
+        return Response.serverError().entity(responseMessage).build();
+      }
+    } catch (Exception e) {
+      responseMessage.put("message", "Failed to save the detection alert config.");
+      responseMessage.put("more-info", "Exception = " + e);
+      return Response.serverError().entity(responseMessage).build();
+    }
+
+    responseMessage.put("message", "The YAML alert config was saved successfully.");
+    responseMessage.put("more-info", "Record saved with id " + detectionAlertConfigId);
+    return Response.ok().entity(responseMessage).build();
+  }
 
   /*
    * Init the pipeline to check if detection pipeline property is valid semantically.
@@ -205,45 +237,105 @@ public class YamlResource {
     detectionConfig.setId(id);
   }
 
-  /**
-   translate alert yaml to detection alert config
-   */
-  private DetectionAlertConfigDTO getDetectionAlertConfig(Map<String, Object> alertYaml, Long detectionConfigId) {
-    Preconditions.checkArgument(alertYaml.containsKey(PROP_NAME), "alert name missing");
+  @SuppressWarnings("unchecked")
+  public DetectionAlertConfigDTO createDetectionAlertConfig(String yamlAlertConfig, Map<String, String> responseMessage ) {
+    if (!alertValidator.validateYAMLConfig(yamlAlertConfig, responseMessage)) {
+      return null;
+    }
 
-    // try to retrieve existing alert config
-    List<DetectionAlertConfigDTO> existingAlertConfigDTOs =
-        this.detectionAlertConfigDAO.findByPredicate(Predicate.EQ("name", MapUtils.getString(alertYaml, PROP_NAME)));
+    TreeMap<String, Object> newAlertConfigMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+    newAlertConfigMap.putAll((Map<String, Object>) this.yaml.load(yamlAlertConfig));
 
-    if (existingAlertConfigDTOs.isEmpty()) {
-      // if alert does not exist, create a new alerter
-      return this.alertConfigTranslator.generateDetectionAlertConfig(alertYaml, Collections.singletonList(detectionConfigId), new HashMap<>());
-    } else {
-      // get existing detection alerter
-      DetectionAlertConfigDTO existingAlertConfigDTO = existingAlertConfigDTOs.get(0);
-      if (alertYaml.containsKey(PROP_TYPE)) {
-        // if alert Yaml contains alert configuration, update existing alert config properties
-        Set<Long> detectionConfigIds =
-            new HashSet(ConfigUtils.getLongs(existingAlertConfigDTO.getProperties().get(PROP_DETECTION_CONFIG_ID)));
-        detectionConfigIds.add(detectionConfigId);
-        DetectionAlertConfigDTO alertConfigDTO =
-            this.alertConfigTranslator.generateDetectionAlertConfig(alertYaml, detectionConfigIds, existingAlertConfigDTO.getVectorClocks());
-        alertConfigDTO.setId(existingAlertConfigDTO.getId());
-        alertConfigDTO.setHighWaterMark(existingAlertConfigDTO.getHighWaterMark());
-        return alertConfigDTO;
-      } else {
-        // Yaml does not contains alert config properties, add the detection pipeline to a existing alerter
-        Map<Long, Long> existingVectorClocks = existingAlertConfigDTO.getVectorClocks();
-        if (!existingVectorClocks.containsKey(detectionConfigId)) {
-          existingVectorClocks.put(detectionConfigId, 0L);
-        }
-        Set<Long> detectionConfigIds =
-            new HashSet(ConfigUtils.getList(existingAlertConfigDTO.getProperties().get(PROP_DETECTION_CONFIG_ID)));
-        detectionConfigIds.add(detectionConfigId);
-        existingAlertConfigDTO.getProperties().put(PROP_DETECTION_CONFIG_ID, detectionConfigIds);
-        return existingAlertConfigDTO;
+    // Check if a subscription group with the name already exists
+    String subsGroupName = MapUtils.getString(newAlertConfigMap, PROP_SUBS_GROUP_NAME);
+    if (StringUtils.isEmpty(subsGroupName)) {
+      responseMessage.put("message", "Subscription group name field cannot be left empty.");
+      return null;
+    }
+    List<DetectionAlertConfigDTO> alertConfigDTOS = this.detectionAlertConfigDAO
+        .findByPredicate(Predicate.EQ("name", MapUtils.getString(newAlertConfigMap, PROP_SUBS_GROUP_NAME)));
+    if (!alertConfigDTOS.isEmpty()) {
+      responseMessage.put("message", "Subscription group name is already taken. Please use a different name.");
+      return null;
+    }
+
+    // Translate config from YAML to detection alert config (JSON)
+    DetectionAlertConfigDTO alertConfig = this.alertConfigTranslator.translate(newAlertConfigMap);
+    alertConfig.setYaml(yamlAlertConfig);
+
+    // Validate the config before saving it
+    if (!alertValidator.validateConfig(alertConfig, responseMessage)) {
+      return null;
+    }
+
+    return alertConfig;
+  }
+
+  @PUT
+  @Path("/notification/{id}")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Consumes(MediaType.TEXT_PLAIN)
+  @ApiOperation("Edit a notification group using a YAML config")
+  @SuppressWarnings("unchecked")
+  public Response updateDetectionAlertConfig(
+      @ApiParam("payload") String yamlAlertConfig,
+      @ApiParam("the detection alert config id to edit") @PathParam("id") long id) {
+    Map<String, String> responseMessage = new HashMap<>();
+    try {
+      DetectionAlertConfigDTO alertDTO = this.detectionAlertConfigDAO.findById(id);
+      DetectionAlertConfigDTO updatedAlertConfig = updateDetectionAlertConfig(alertDTO, yamlAlertConfig, responseMessage);
+      if (updatedAlertConfig == null) {
+        return Response.status(Response.Status.BAD_REQUEST).entity(responseMessage).build();
       }
+
+      int detectionAlertConfigId = this.detectionAlertConfigDAO.update(updatedAlertConfig);
+      if (detectionAlertConfigId <= 0) {
+        responseMessage.put("message", "Failed to update the detection alert config.");
+        responseMessage.put("more-info", "Zero records updated. Check for DB issues. YAML config = " + yamlAlertConfig);
+        return Response.serverError().entity(responseMessage).build();
+      }
+    } catch (Exception e) {
+      responseMessage.put("message", "Failed to update the detection alert config.");
+      responseMessage.put("more-info", "Exception = " + e);
+      return Response.serverError().entity(responseMessage).build();
+    }
+
+    responseMessage.put("message", "The YAML alert config was updated successfully.");
+    return Response.ok().entity(responseMessage).build();
+  }
+
+  public DetectionAlertConfigDTO updateDetectionAlertConfig(DetectionAlertConfigDTO oldAlertConfig, String yamlAlertConfig,
+      Map<String,String> responseMessage) {
+    if (oldAlertConfig == null) {
+      responseMessage.put("message", "Cannot find subscription group");
+      return null;
+    }
+
+    if (!alertValidator.validateYAMLConfig(yamlAlertConfig, responseMessage)) {
+      return null;
     }
+
+    TreeMap<String, Object> newAlertConfigMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+    newAlertConfigMap.putAll((Map<String, Object>) this.yaml.load(yamlAlertConfig));
+
+    // Search for the detection alert config's reference in the db
+    String subsGroupName = MapUtils.getString(newAlertConfigMap, PROP_SUBS_GROUP_NAME);
+    if (StringUtils.isEmpty(subsGroupName)) {
+      responseMessage.put("message", "Subscription group name field cannot be left empty.");
+      return null;
+    }
+    DetectionAlertConfigDTO newAlertConfig = this.alertConfigTranslator.translate(newAlertConfigMap);
+
+    // Translate config from YAML to detection alert config (JSON)
+    DetectionAlertConfigDTO updatedAlertConfig = updateDetectionAlertConfig(oldAlertConfig, newAlertConfig);
+    updatedAlertConfig.setYaml(yamlAlertConfig);
+
+    // Validate before updating the config
+    if (!alertValidator.validateUpdatedConfig(updatedAlertConfig, oldAlertConfig, responseMessage)) {
+      return null;
+    }
+
+    return updatedAlertConfig;
   }
 
   /**
@@ -275,4 +367,27 @@ public class YamlResource {
     }
     return yamlObjects;
   }
+
+  /**
+   * Update the existing {@code oldAlertConfig} with the new {@code newAlertConfig}
+   *
+   * Update all the fields except the vector clocks and high watermark. The clocks and watermarks
+   * are managed by the platform. They shouldn't be reset by the user.
+   */
+  public DetectionAlertConfigDTO updateDetectionAlertConfig(DetectionAlertConfigDTO oldAlertConfig,
+      DetectionAlertConfigDTO newAlertConfig) {
+    oldAlertConfig.setName(newAlertConfig.getName());
+    oldAlertConfig.setCronExpression(newAlertConfig.getCronExpression());
+    oldAlertConfig.setApplication(newAlertConfig.getApplication());
+    oldAlertConfig.setFrom(newAlertConfig.getFrom());
+    oldAlertConfig.setSubjectType(newAlertConfig.getSubjectType());
+    oldAlertConfig.setReferenceLinks(newAlertConfig.getReferenceLinks());
+    oldAlertConfig.setActive(newAlertConfig.isActive());
+    oldAlertConfig.setAlertSchemes(newAlertConfig.getAlertSchemes());
+    oldAlertConfig.setAlertSuppressors(newAlertConfig.getAlertSuppressors());
+    oldAlertConfig.setOnlyFetchLegacyAnomalies(newAlertConfig.isOnlyFetchLegacyAnomalies());
+    oldAlertConfig.setProperties(newAlertConfig.getProperties());
+
+    return oldAlertConfig;
+  }
 }
diff --git a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslatorTest.java b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslatorTest.java
index eac835b..238a9b6 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslatorTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslatorTest.java
@@ -72,7 +72,7 @@ public class YamlDetectionAlertConfigTranslatorTest {
     refLinks.put("Test Link", "test_url");
     alertYamlConfigs.put(PROP_REFERENCE_LINKS, refLinks);
 
-    Set<Long> detectionIds = new HashSet<>(Arrays.asList(1234L, 6789L));
+    Set<Integer> detectionIds = new HashSet<>(Arrays.asList(1234, 6789));
     alertYamlConfigs.put(PROP_DETECTION_CONFIG_IDS, detectionIds);
 
     Map<String, Object> alertSchemes = new HashMap<>();
diff --git a/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/yaml/YamlResourceTest.java b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/yaml/YamlResourceTest.java
new file mode 100644
index 0000000..f39a05f
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/com/linkedin/thirdeye/detection/yaml/YamlResourceTest.java
@@ -0,0 +1,140 @@
+package com.linkedin.thirdeye.detection.yaml;
+
+import com.linkedin.thirdeye.datalayer.bao.DAOTestBase;
+import com.linkedin.thirdeye.datalayer.dto.ApplicationDTO;
+import com.linkedin.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
+import com.linkedin.thirdeye.datasource.DAORegistry;
+import com.linkedin.thirdeye.detection.annotation.registry.DetectionAlertRegistry;
+import com.linkedin.thirdeye.detection.annotation.registry.DetectionRegistry;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.commons.io.IOUtils;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+public class YamlResourceTest {
+
+  private DAOTestBase testDAOProvider;
+  private YamlResource yamlResource;
+  private DAORegistry daoRegistry;
+
+  @BeforeClass
+  public void beforeClass() {
+    testDAOProvider = DAOTestBase.getInstance();
+    this.yamlResource = new YamlResource();
+    this.daoRegistry = DAORegistry.getInstance();
+
+    DetectionAlertRegistry.getInstance().registerAlertScheme("EMAIL", "EmailClass");
+    DetectionAlertRegistry.getInstance().registerAlertScheme("IRIS", "IrisClass");
+    DetectionAlertRegistry.getInstance().registerAlertSuppressor("TIME_WINDOW", "TimeWindowClass");
+    DetectionRegistry.registerComponent("TimeWindowClass", "DIMENSIONAL_ALERTER_PIPELINE");
+  }
+
+  @AfterClass(alwaysRun = true)
+  void afterClass() {
+    testDAOProvider.cleanup();
+  }
+
+  @Test
+  public void testCreateDetectionAlertConfig() throws IOException {
+    Map<String, String> responseMessage = new HashMap<>();
+    DetectionAlertConfigDTO alertDTO;
+
+    String blankYaml = "";
+    alertDTO = this.yamlResource.createDetectionAlertConfig(blankYaml, responseMessage);
+    Assert.assertNull(alertDTO);
+    Assert.assertEquals(responseMessage.get("message"), "The config file cannot be blank.");
+
+    String inValidYaml = "application:test:application";
+    alertDTO = this.yamlResource.createDetectionAlertConfig(inValidYaml, responseMessage);
+    Assert.assertNull(alertDTO);
+    Assert.assertEquals(responseMessage.get("message"), "There was an error parsing the yaml file. Check for syntax issues.");
+
+    String noSubscriptGroupYaml = "application: test_application";
+    alertDTO = this.yamlResource.createDetectionAlertConfig(noSubscriptGroupYaml, responseMessage);
+    Assert.assertNull(alertDTO);
+    Assert.assertEquals(responseMessage.get("message"), "Subscription group name field cannot be left empty.");
+
+    String appFieldMissingYaml = IOUtils.toString(this.getClass().getResourceAsStream("alertconfig/alert-config-1.yaml"));
+    alertDTO = this.yamlResource.createDetectionAlertConfig(appFieldMissingYaml, responseMessage);
+    Assert.assertNull(alertDTO);
+    Assert.assertEquals(responseMessage.get("message"), "Application field cannot be left empty");
+
+    String appMissingYaml = IOUtils.toString(this.getClass().getResourceAsStream("alertconfig/alert-config-2.yaml"));
+    alertDTO = this.yamlResource.createDetectionAlertConfig(appMissingYaml, responseMessage);
+    Assert.assertNull(alertDTO);
+    Assert.assertEquals(responseMessage.get("message"), "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.");
+
+    DetectionAlertConfigDTO oldAlertDTO = new DetectionAlertConfigDTO();
+    oldAlertDTO.setName("test_group");
+    daoRegistry.getDetectionAlertConfigManager().save(oldAlertDTO);
+
+    String groupExists = IOUtils.toString(this.getClass().getResourceAsStream("alertconfig/alert-config-3.yaml"));
+    alertDTO = this.yamlResource.createDetectionAlertConfig(groupExists, responseMessage);
+    Assert.assertNull(alertDTO);
+    Assert.assertEquals(responseMessage.get("message"), "Subscription group name is already taken. Please use a different name.");
+
+    ApplicationDTO request = new ApplicationDTO();
+    request.setApplication("test_application");
+    request.setRecipients("abc@abc.in");
+    daoRegistry.getApplicationDAO().save(request);
+
+    String validYaml = IOUtils.toString(this.getClass().getResourceAsStream("alertconfig/alert-config-4.yaml"));
+    alertDTO = this.yamlResource.createDetectionAlertConfig(validYaml, responseMessage);
+    Assert.assertNotNull(alertDTO);
+    Assert.assertEquals(alertDTO.getName(), "Subscription Group Name");
+  }
+
+  @Test
+  public void testUpdateDetectionAlertConfig() throws IOException {
+    DetectionAlertConfigDTO oldAlertDTO = new DetectionAlertConfigDTO();
+    oldAlertDTO.setName("Subscription Group Name");
+    oldAlertDTO.setApplication("Random Application");
+    daoRegistry.getDetectionAlertConfigManager().save(oldAlertDTO);
+
+    Map<String, String> responseMessage = new HashMap<>();
+    DetectionAlertConfigDTO alertDTO;
+
+    alertDTO = this.yamlResource.updateDetectionAlertConfig(null, "", responseMessage);
+    Assert.assertNull(alertDTO);
+    Assert.assertEquals(responseMessage.get("message"), "Cannot find subscription group");
+
+    String blankYaml = "";
+    alertDTO = this.yamlResource.updateDetectionAlertConfig(oldAlertDTO, blankYaml, responseMessage);
+    Assert.assertNull(alertDTO);
+    Assert.assertEquals(responseMessage.get("message"), "The config file cannot be blank.");
+
+    String inValidYaml = "application:test:application";
+    alertDTO = this.yamlResource.updateDetectionAlertConfig(oldAlertDTO, inValidYaml, responseMessage);
+    Assert.assertNull(alertDTO);
+    Assert.assertEquals(responseMessage.get("message"), "There was an error parsing the yaml file. Check for syntax issues.");
+
+    String noSubscriptGroupYaml = "application: test_application";
+    alertDTO = this.yamlResource.updateDetectionAlertConfig(oldAlertDTO, noSubscriptGroupYaml, responseMessage);
+    Assert.assertNull(alertDTO);
+    Assert.assertEquals(responseMessage.get("message"), "Subscription group name field cannot be left empty.");
+
+    String appFieldMissingYaml = IOUtils.toString(this.getClass().getResourceAsStream("alertconfig/alert-config-1.yaml"));
+    alertDTO = this.yamlResource.updateDetectionAlertConfig(oldAlertDTO, appFieldMissingYaml, responseMessage);
+    Assert.assertNull(alertDTO);
+    Assert.assertEquals(responseMessage.get("message"), "Application field cannot be left empty");
+
+    ApplicationDTO request = new ApplicationDTO();
+    request.setApplication("test_application");
+    request.setRecipients("abc@abc.in");
+    daoRegistry.getApplicationDAO().save(request);
+
+    String validYaml = IOUtils.toString(this.getClass().getResourceAsStream("alertconfig/alert-config-3.yaml"));
+    alertDTO = this.yamlResource.updateDetectionAlertConfig(oldAlertDTO, validYaml, responseMessage);
+    Assert.assertNotNull(alertDTO);
+    Assert.assertEquals(alertDTO.getName(), "test_group");
+    Assert.assertEquals(alertDTO.getApplication(), "test_application");
+  }
+}
+
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/com/linkedin/thirdeye/detection/yaml/alertconfig/alert-config-1.yaml b/thirdeye/thirdeye-pinot/src/test/resources/com/linkedin/thirdeye/detection/yaml/alertconfig/alert-config-1.yaml
new file mode 100644
index 0000000..6ee42f4
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/resources/com/linkedin/thirdeye/detection/yaml/alertconfig/alert-config-1.yaml
@@ -0,0 +1,40 @@
+subscriptionGroupName: "Subscription Group Name"
+cron: "0 0/5 * 1/1 * ? *"
+active: true
+
+type: DIMENSIONAL_ALERTER_PIPELINE
+dimensionRecipients:
+ "android":
+  - "android-thirdeye@thirdeye.com"
+ "ios":
+  - "ios-thirdeye@thirdeye.com"
+dimension: app_name
+
+detectionConfigIds:
+ - 5773069
+
+fromAddress: thirdeye-dev@linkedin.com
+
+recipients:
+ to:
+  - "thirdeye@thirdeye.com"
+ cc:
+  - "thirdeye-developers@thirdeye.com"
+
+alertSchemes:
+- type: EMAIL
+- type: IRIS
+  params:
+    plan: thirdye_test_plan
+
+alertSuppressors:
+- type: TIME_WINDOW
+  params:
+    windowStartTime: 1542888000000
+    windowEndTime: 1543215600000
+    isThresholdApplied: true
+    expectedChange: -0.25
+    acceptableDeviation: 0.35
+
+referenceLinks:
+ "Oncall Runbook": "test_url"
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/com/linkedin/thirdeye/detection/yaml/alertconfig/alert-config-2.yaml b/thirdeye/thirdeye-pinot/src/test/resources/com/linkedin/thirdeye/detection/yaml/alertconfig/alert-config-2.yaml
new file mode 100644
index 0000000..9bf1cc1
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/resources/com/linkedin/thirdeye/detection/yaml/alertconfig/alert-config-2.yaml
@@ -0,0 +1,42 @@
+subscriptionGroupName: "Subscription Group Name"
+cron: "0 0/5 * 1/1 * ? *"
+application: "test_application"
+active: true
+fromAddress: thirdeye@thirdeye.com
+
+type: DIMENSIONAL_ALERTER_PIPELINE
+dimensionRecipients:
+ "android":
+  - "android-thirdeye@thirdeye.com"
+ "ios":
+  - "ios-thirdeye@thirdeye.com"
+dimension: app_name
+
+detectionConfigIds:
+ - 5773069
+
+fromAddress: thirdeye-dev@linkedin.com
+
+recipients:
+ to:
+  - "thirdeye@thirdeye.com"
+ cc:
+  - "thirdeye-developers@thirdeye.com"
+
+alertSchemes:
+- type: EMAIL
+- type: IRIS
+  params:
+    plan: thirdye_test_plan
+
+alertSuppressors:
+- type: TIME_WINDOW
+  params:
+    windowStartTime: 1542888000000
+    windowEndTime: 1543215600000
+    isThresholdApplied: true
+    expectedChange: -0.25
+    acceptableDeviation: 0.35
+
+referenceLinks:
+ "Oncall Runbook": "test_url"
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/com/linkedin/thirdeye/detection/yaml/alertconfig/alert-config-3.yaml b/thirdeye/thirdeye-pinot/src/test/resources/com/linkedin/thirdeye/detection/yaml/alertconfig/alert-config-3.yaml
new file mode 100644
index 0000000..2325502
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/resources/com/linkedin/thirdeye/detection/yaml/alertconfig/alert-config-3.yaml
@@ -0,0 +1,42 @@
+subscriptionGroupName: "test_group"
+cron: "0 0/5 * 1/1 * ? *"
+application: "test_application"
+active: true
+fromAddress: thirdeye@thirdeye.com
+
+type: DIMENSIONAL_ALERTER_PIPELINE
+dimensionRecipients:
+ "android":
+  - "android-thirdeye@thirdeye.com"
+ "ios":
+  - "ios-thirdeye@thirdeye.com"
+dimension: app_name
+
+detectionConfigIds:
+ - 5773069
+
+fromAddress: thirdeye-dev@linkedin.com
+
+recipients:
+ to:
+  - "thirdeye@thirdeye.com"
+ cc:
+  - "thirdeye-developers@thirdeye.com"
+
+alertSchemes:
+- type: EMAIL
+- type: IRIS
+  params:
+    plan: thirdye_test_plan
+
+alertSuppressors:
+- type: TIME_WINDOW
+  params:
+    windowStartTime: 1542888000000
+    windowEndTime: 1543215600000
+    isThresholdApplied: true
+    expectedChange: -0.25
+    acceptableDeviation: 0.35
+
+referenceLinks:
+ "Oncall Runbook": "test_url"
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/com/linkedin/thirdeye/detection/yaml/alertconfig/alert-config-4.yaml b/thirdeye/thirdeye-pinot/src/test/resources/com/linkedin/thirdeye/detection/yaml/alertconfig/alert-config-4.yaml
new file mode 100644
index 0000000..9bf1cc1
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/resources/com/linkedin/thirdeye/detection/yaml/alertconfig/alert-config-4.yaml
@@ -0,0 +1,42 @@
+subscriptionGroupName: "Subscription Group Name"
+cron: "0 0/5 * 1/1 * ? *"
+application: "test_application"
+active: true
+fromAddress: thirdeye@thirdeye.com
+
+type: DIMENSIONAL_ALERTER_PIPELINE
+dimensionRecipients:
+ "android":
+  - "android-thirdeye@thirdeye.com"
+ "ios":
+  - "ios-thirdeye@thirdeye.com"
+dimension: app_name
+
+detectionConfigIds:
+ - 5773069
+
+fromAddress: thirdeye-dev@linkedin.com
+
+recipients:
+ to:
+  - "thirdeye@thirdeye.com"
+ cc:
+  - "thirdeye-developers@thirdeye.com"
+
+alertSchemes:
+- type: EMAIL
+- type: IRIS
+  params:
+    plan: thirdye_test_plan
+
+alertSuppressors:
+- type: TIME_WINDOW
+  params:
+    windowStartTime: 1542888000000
+    windowEndTime: 1543215600000
+    isThresholdApplied: true
+    expectedChange: -0.25
+    acceptableDeviation: 0.35
+
+referenceLinks:
+ "Oncall Runbook": "test_url"
\ No newline at end of file


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