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/02/01 23:13:51 UTC

[incubator-pinot] branch master updated: [TE] Replace response message with exceptions; Reuse notification endpoint in Create Alert (#3780)

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 1be4b5c  [TE] Replace response message with exceptions; Reuse notification endpoint in Create Alert (#3780)
1be4b5c is described below

commit 1be4b5ced82d669d095c1f74760b585499e66f3a
Author: Akshay Rai <ak...@gmail.com>
AuthorDate: Fri Feb 1 15:13:47 2019 -0800

    [TE] Replace response message with exceptions; Reuse notification endpoint in Create Alert (#3780)
---
 .../detection/validators/ConfigValidator.java      |  11 +-
 .../validators/DetectionAlertConfigValidator.java  |  60 ++--
 .../thirdeye/detection/yaml/YamlResource.java      | 331 +++++++++------------
 .../thirdeye/detection/yaml/YamlResourceTest.java  | 146 +++++----
 4 files changed, 257 insertions(+), 291 deletions(-)

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 d77f25f..4879674 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,6 +19,7 @@
 
 package org.apache.pinot.thirdeye.detection.validators;
 
+import javax.xml.bind.ValidationException;
 import org.apache.pinot.thirdeye.datalayer.bao.AlertConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.ApplicationManager;
 import org.apache.pinot.thirdeye.datasource.DAORegistry;
@@ -44,21 +45,17 @@ public abstract class ConfigValidator {
    * the yaml exists and is parsable
    */
   @SuppressWarnings("unchecked")
-  public boolean validateYAMLConfig(String yamlConfig, Map<String, String> responseMessage) {
+  public boolean validateYAMLConfig(String yamlConfig) throws ValidationException {
     // 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;
+      throw new ValidationException("The Yaml Payload in the request is empty.");
     }
 
     // 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;
+      throw new ValidationException("Error parsing the Yaml input. Check for syntax issues.");
     }
 
     return true;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionAlertConfigValidator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionAlertConfigValidator.java
index 8f2e0de..bd145ad 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionAlertConfigValidator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionAlertConfigValidator.java
@@ -19,10 +19,12 @@
 
 package org.apache.pinot.thirdeye.detection.validators;
 
+import javax.xml.bind.ValidationException;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
 import java.util.List;
 import java.util.Map;
 import org.apache.commons.lang.StringUtils;
+import org.apache.pinot.thirdeye.datalayer.util.Predicate;
 import org.apache.pinot.thirdeye.datasource.DAORegistry;
 import org.junit.Assert;
 
@@ -42,63 +44,57 @@ public class DetectionAlertConfigValidator extends ConfigValidator {
    * 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;
-
+  public void validateConfig(DetectionAlertConfigDTO alertConfig) throws ValidationException {
     // 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;
+      throw new ValidationException("Subscription group name field cannot be left empty.");
     }
     if (StringUtils.isEmpty(alertConfig.getApplication())) {
-      responseMessage.put("message", "Application field cannot be left empty");
-      return false;
+      throw new ValidationException("Application field cannot be left empty");
     }
     if (StringUtils.isEmpty(alertConfig.getFrom())) {
-      responseMessage.put("message", "From address field cannot be left empty");
-      return false;
+      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())) {
-      responseMessage.put("message", "'Type' field cannot be left empty.");
-      return false;
+      throw new ValidationException("'Type' field cannot be left empty.");
     }
     // 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;
+      throw new ValidationException("Alert scheme cannot be left empty");
     }
     // Properties cannot be empty
     if (alertConfig.getProperties() == null || alertConfig.getProperties().isEmpty()) {
-      responseMessage.put("message", "Alert properties cannot be left empty. Please specify the recipients,"
+      throw new ValidationException("Alert properties cannot be left empty. Please specify the recipients,"
           + " subscribed detections, 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"
+      throw new ValidationException("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"
+      throw new ValidationException("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 (DAORegistry.getInstance().getApplicationDAO().findByName(alertConfig.getApplication()).size() == 0) {
-      responseMessage.put("message", "Application name doesn't exist in our registry. Please use an existing"
+      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.");
-      return false;
     }
 
-    // TODO add more checks like cron validity, email validity, alert type check, scheme type check etc.
+    // Check for duplicates
+    List<DetectionAlertConfigDTO> alertConfigDTOS = DAORegistry.getInstance().getDetectionAlertConfigManager()
+        .findByPredicate(Predicate.EQ("name", alertConfig.getName()));
+    if (!alertConfigDTOS.isEmpty()) {
+      throw new ValidationException("Subscription group name is already taken. Please use a different name.");
+    }
 
-    return isValid;
+    // TODO add more checks like cron validity, email validity, alert type check, scheme type check etc.
   }
 
   /**
@@ -106,30 +102,22 @@ public class DetectionAlertConfigValidator extends ConfigValidator {
    * 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;
-    }
+  public void validateUpdatedConfig(DetectionAlertConfigDTO updatedAlertConfig, DetectionAlertConfigDTO oldAlertConfig)
+      throws ValidationException {
+    validateConfig(updatedAlertConfig);
 
     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;
+      throw new ValidationException("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()) {
-          responseMessage.put("message", "Vector clock has been modified. This is not allowed.");
-          isValid = false;
+          throw new ValidationException("Vector clock has been modified. This is not allowed.");
         }
       }
     }
-
-    return isValid;
   }
 }
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 d87386c..d1cdd6a 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
@@ -43,6 +43,7 @@ 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.api.Constants;
@@ -78,14 +79,13 @@ public class YamlResource {
   protected static final Logger LOG = LoggerFactory.getLogger(YamlResource.class);
   private static ObjectMapper OBJECT_MAPPER = new ObjectMapper();
 
-  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 DetectionAlertConfigValidator notificationValidator;
   private final DataProvider provider;
   private final MetricConfigManager metricDAO;
   private final DatasetConfigManager datasetDAO;
@@ -98,7 +98,7 @@ public class YamlResource {
     this.detectionConfigDAO = DAORegistry.getInstance().getDetectionConfigManager();
     this.detectionAlertConfigDAO = DAORegistry.getInstance().getDetectionAlertConfigManager();
     this.translatorLoader = new YamlDetectionTranslatorLoader();
-    this.alertValidator = DetectionAlertConfigValidator.getInstance();
+    this.notificationValidator = DetectionAlertConfigValidator.getInstance();
     this.alertConfigTranslator = new YamlDetectionAlertConfigTranslator(this.detectionConfigDAO);
     this.metricDAO = DAORegistry.getInstance().getMetricConfigDAO();
     this.datasetDAO = DAORegistry.getInstance().getDatasetConfigDAO();
@@ -118,6 +118,53 @@ public class YamlResource {
     this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, anomalyDAO, timeseriesLoader, aggregationLoader, loader);
   }
 
+  public DetectionConfigDTO translateToDetectionConfig(Map<String, Object> yamlConfig, Map<String, String> responseMessage) {
+    return buildDetectionConfigFromYaml(0, 0, yamlConfig, null, responseMessage);
+  }
+
+  /*
+   * Build the detection config from a yaml.
+   * Returns null if building or validation failed. Error messages stored in responseMessage.
+   */
+  private DetectionConfigDTO buildDetectionConfigFromYaml(long startTime, long endTime, Map<String, Object> yamlConfig,
+      DetectionConfigDTO existingDetectionConfig, Map<String, String> responseMessage) {
+    if (startTime == 0L && endTime == 0L) {
+      // default tuning window 28 days
+      endTime = System.currentTimeMillis();
+      startTime = endTime - TimeUnit.DAYS.toMillis(28);
+    }
+
+    try{
+      YamlDetectionConfigTranslator translator = this.translatorLoader.from(yamlConfig, this.provider);
+      DetectionConfigDTO detectionConfig = translator.withTrainingWindow(startTime, endTime)
+          .withExistingDetectionConfig(existingDetectionConfig)
+          .generateDetectionConfig();
+      validatePipeline(detectionConfig);
+      return detectionConfig;
+    } catch (InvocationTargetException e){
+      // exception thrown in validate pipeline via reflection
+      LOG.error("Validate pipeline error", e);
+      responseMessage.put("message", e.getCause().getMessage());
+    } catch (Exception e) {
+      LOG.error("yaml translation error", e);
+      responseMessage.put("message", e.getMessage());
+    }
+    return null;
+  }
+
+  /*
+   * Init the pipeline to check if detection pipeline property is valid semantically.
+   */
+  private void validatePipeline(DetectionConfigDTO detectionConfig) throws Exception {
+    Long id = detectionConfig.getId();
+    // swap out id
+    detectionConfig.setId(-1L);
+    // try to load the detection pipeline and init all the components
+    this.loader.from(provider, detectionConfig, 0, 0);
+    // set id back
+    detectionConfig.setId(id);
+  }
+
   @POST
   @Path("/create-alert")
   @Produces(MediaType.APPLICATION_JSON)
@@ -137,7 +184,7 @@ public class YamlResource {
       return Response.status(Response.Status.BAD_REQUEST).entity(ImmutableMap.of("message", "notification yaml is missing")).build();
     }
 
-    // get detection yaml
+    // Detection
     String detectionYaml = yamls.get("detection");
 
     Map<String, Object> detectionYamlConfig;
@@ -168,70 +215,17 @@ public class YamlResource {
     }
     Preconditions.checkNotNull(detectionConfigId, "Save detection config failed");
 
-    // notification
-    // TODO: Inject detectionConfigId into detection alert config
-    DetectionAlertConfigDTO alertConfig = createDetectionAlertConfig(yamls.get("notification"), responseMessage);
-    if (alertConfig == null) {
-      // revert detection DTO
-      this.detectionConfigDAO.deleteById(detectionConfigId);
-      return Response.status(Response.Status.BAD_REQUEST).entity(responseMessage).build();
-    }
-    Long detectionAlertConfigId = this.detectionAlertConfigDAO.save(alertConfig);
-    if (detectionAlertConfigId == null){
+    // Notification
+    String notificationYaml = yamls.get("notification");
+    Response response = createDetectionAlertConfigApi(notificationYaml);
+    if (response.getStatusInfo() != Response.Status.OK) {
       // revert detection DTO
       this.detectionConfigDAO.deleteById(detectionConfigId);
-      return Response.serverError().entity(ImmutableMap.of("message", "Save detection alert config failed")).build();
+      return Response.serverError().entity(response.getEntity()).build();
     }
-    LOG.info("saved detection alert config id {}", detectionAlertConfigId);
+    long alertId = Long.parseLong(((Map<String, String>) response.getEntity()).get("detectionAlertConfigId"));
 
-    return Response.ok().entity(ImmutableMap.of("detectionConfigId", detectionConfig.getId(), "detectionAlertConfigId", alertConfig.getId())).build();
-  }
-
-  public DetectionConfigDTO translateToDetectionConfig(Map<String, Object> yamlConfig, Map<String, String> responseMessage) {
-    return buildDetectionConfigFromYaml(0, 0, yamlConfig, null, responseMessage);
-  }
-
-  /*
-   * Build the detection config from a yaml.
-   * Returns null if building or validation failed. Error messages stored in responseMessage.
-   */
-  private DetectionConfigDTO buildDetectionConfigFromYaml(long startTime, long endTime, Map<String, Object> yamlConfig,
-      DetectionConfigDTO existingDetectionConfig, Map<String, String> responseMessage) {
-    if (startTime == 0L && endTime == 0L) {
-      // default tuning window 28 days
-      endTime = System.currentTimeMillis();
-      startTime = endTime - TimeUnit.DAYS.toMillis(28);
-    }
-
-    try{
-      YamlDetectionConfigTranslator translator = this.translatorLoader.from(yamlConfig, this.provider);
-      DetectionConfigDTO detectionConfig = translator.withTrainingWindow(startTime, endTime)
-          .withExistingDetectionConfig(existingDetectionConfig)
-          .generateDetectionConfig();
-      validatePipeline(detectionConfig);
-      return detectionConfig;
-    } catch (InvocationTargetException e){
-      // exception thrown in validate pipeline via reflection
-      LOG.error("Validate pipeline error", e);
-      responseMessage.put("message", e.getCause().getMessage());
-    } catch (Exception e) {
-      LOG.error("yaml translation error", e);
-      responseMessage.put("message", e.getMessage());
-    }
-    return null;
-  }
-
-  /*
-   * Init the pipeline to check if detection pipeline property is valid semantically.
-   */
-  private void validatePipeline(DetectionConfigDTO detectionConfig) throws Exception {
-    Long id = detectionConfig.getId();
-    // swap out id
-    detectionConfig.setId(-1L);
-    // try to load the detection pipeline and init all the components
-    this.loader.from(provider, detectionConfig, 0, 0);
-    // set id back
-    detectionConfig.setId(id);
+    return Response.ok().entity(ImmutableMap.of("detectionConfigId", detectionConfig.getId(), "detectionAlertConfigId", alertId)).build();
   }
 
   /**
@@ -323,154 +317,126 @@ public class YamlResource {
     return Response.ok(detectionConfig).build();
   }
 
+  @SuppressWarnings("unchecked")
+  DetectionAlertConfigDTO createDetectionAlertConfig(String yamlAlertConfig) throws ValidationException {
+    notificationValidator.validateYAMLConfig(yamlAlertConfig);
+
+    // Translate config from YAML to detection alert config (JSON)
+    TreeMap<String, Object> newAlertConfigMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+    newAlertConfigMap.putAll((Map<String, Object>) this.yaml.load(yamlAlertConfig));
+    DetectionAlertConfigDTO alertConfig = this.alertConfigTranslator.translate(newAlertConfigMap);
+    alertConfig.setYaml(yamlAlertConfig);
+
+    // Validate the config before saving it
+    notificationValidator.validateConfig(alertConfig);
+
+    return alertConfig;
+  }
+
+  @SuppressWarnings("unchecked")
+  DetectionAlertConfigDTO updateDetectionAlertConfig(long oldAlertConfigID, String yamlAlertConfig) throws ValidationException {
+    DetectionAlertConfigDTO oldAlertConfig = this.detectionAlertConfigDAO.findById(oldAlertConfigID);
+    if (oldAlertConfig == null) {
+      throw new RuntimeException("Cannot find subscription group " + oldAlertConfigID);
+    }
+    notificationValidator.validateYAMLConfig(yamlAlertConfig);
+
+    // Translate payload to detection alert config
+    TreeMap<String, Object> newAlertConfigMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+    newAlertConfigMap.putAll((Map<String, Object>) this.yaml.load(yamlAlertConfig));
+    DetectionAlertConfigDTO newAlertConfig = this.alertConfigTranslator.translate(newAlertConfigMap);
+
+    // Update existing alert config with the newly supplied config.
+    DetectionAlertConfigDTO updatedAlertConfig = updateDetectionAlertConfig(oldAlertConfig, newAlertConfig);
+    updatedAlertConfig.setYaml(yamlAlertConfig);
+
+    // Validate before updating the config
+    notificationValidator.validateUpdatedConfig(updatedAlertConfig, oldAlertConfig);
+
+    return updatedAlertConfig;
+  }
+
+  /**
+   * 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.
+   */
+  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;
+  }
+
   @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(
+  public Response createDetectionAlertConfigApi(
       @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();
-      }
+      DetectionAlertConfigDTO alertConfig = createDetectionAlertConfig(yamlAlertConfig);
+      Preconditions.checkNotNull(alertConfig);
 
       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();
-      }
+      Preconditions.checkNotNull(detectionAlertConfigId);
     } catch (Exception e) {
-      responseMessage.put("message", "Failed to save the detection alert config.");
-      responseMessage.put("more-info", "Exception = " + e);
+      LOG.error("Error creating notification group with payload " + yamlAlertConfig, e);
+      responseMessage.put("message", "Failed to create the notification group.");
+      responseMessage.put("more-info", "Error = " + e.getMessage());
       return Response.serverError().entity(responseMessage).build();
     }
 
-    responseMessage.put("message", "The YAML alert config was saved successfully.");
+    LOG.info("Notification group created with id " + detectionAlertConfigId + " using payload " + yamlAlertConfig);
+    responseMessage.put("message", "The notification group was created successfully.");
     responseMessage.put("more-info", "Record saved with id " + detectionAlertConfigId);
     return Response.ok().entity(responseMessage).build();
   }
 
-
-  @SuppressWarnings("unchecked")
-  public DetectionAlertConfigDTO createDetectionAlertConfig(String yamlAlertConfig, Map<String, String> responseMessage ) {
-    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));
-
-    // 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;
-    try {
-      alertConfig = this.alertConfigTranslator.translate(newAlertConfigMap);
-    } catch (Exception e){
-      responseMessage.put("message", e.getMessage());
-      return null;
-    }
-    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(
+  public Response updateDetectionAlertConfigApi(
       @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();
-      }
+      DetectionAlertConfigDTO updatedAlertConfig = updateDetectionAlertConfig(id, yamlAlertConfig);
+      Preconditions.checkNotNull(updatedAlertConfig);
 
       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();
+        throw new RuntimeException("Failed to update the detection alert config.");
       }
     } catch (Exception e) {
-      responseMessage.put("message", "Failed to update the detection alert config.");
-      responseMessage.put("more-info", "Exception = " + e);
+      LOG.error("Error updating notification group " + id + " with payload " + yamlAlertConfig, e);
+      responseMessage.put("message", "Failed to update the notification group " + id);
+      responseMessage.put("more-info", "Error = " + e.getMessage());
       return Response.serverError().entity(responseMessage).build();
     }
 
+    LOG.info("Notification group " + id + " updated successfully with payload " + yamlAlertConfig);
     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;
-    try {
-      newAlertConfig = this.alertConfigTranslator.translate(newAlertConfigMap);
-    } catch (Exception e){
-      responseMessage.put("message", e.getMessage());
-      return null;
-    }
-
-    // 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;
-  }
-
   @POST
   @Path("/preview")
   @Produces(MediaType.APPLICATION_JSON)
@@ -535,27 +501,4 @@ 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/org/apache/pinot/thirdeye/detection/yaml/YamlResourceTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/YamlResourceTest.java
index b09b548..7b53bfa 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/YamlResourceTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/YamlResourceTest.java
@@ -7,14 +7,15 @@ import org.apache.pinot.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
 import org.apache.pinot.thirdeye.datasource.DAORegistry;
 import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionAlertRegistry;
-import org.apache.pinot.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.AfterMethod;
 import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
 
@@ -24,7 +25,7 @@ public class YamlResourceTest {
   private YamlResource yamlResource;
   private DAORegistry daoRegistry;
 
-  @BeforeClass
+  @BeforeMethod
   public void beforeClass() {
     testDAOProvider = DAOTestBase.getInstance();
     this.yamlResource = new YamlResource();
@@ -40,96 +41,129 @@ public class YamlResourceTest {
     DetectionAlertRegistry.getInstance().registerAlertFilter("DIMENSIONAL_ALERTER_PIPELINE", "DimClass");
   }
 
-  @AfterClass(alwaysRun = true)
+  @AfterMethod(alwaysRun = true)
   void afterClass() {
     testDAOProvider.cleanup();
   }
 
-  @Test(priority=1)
+  @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.");
+    try {
+      this.yamlResource.createDetectionAlertConfig(blankYaml);
+      Assert.fail("Exception not thrown on empty yaml");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "The Yaml Payload in the request is empty.");
+    }
 
     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.");
+    try {
+      this.yamlResource.createDetectionAlertConfig(inValidYaml);
+      Assert.fail("Exception not thrown on empty yaml");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "Error parsing the Yaml input. 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.");
+    try {
+      this.yamlResource.createDetectionAlertConfig(noSubscriptGroupYaml);
+      Assert.fail("Exception not thrown on empty yaml");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "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");
+    try {
+      this.yamlResource.createDetectionAlertConfig(appFieldMissingYaml);
+      Assert.fail("Exception not thrown on empty yaml");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "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.");
+    try {
+      this.yamlResource.createDetectionAlertConfig(appMissingYaml);
+      Assert.fail("Exception not thrown on empty yaml");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "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 groupExists = IOUtils.toString(this.getClass().getResourceAsStream("alertconfig/alert-config-3.yaml"));
+    try {
+      this.yamlResource.createDetectionAlertConfig(groupExists);
+      Assert.fail("Exception not thrown on empty yaml");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "Subscription group name is already taken. Please use a different name.");
+    }
+
     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");
+    try {
+      DetectionAlertConfigDTO alert = this.yamlResource.createDetectionAlertConfig(validYaml);
+      Assert.assertNotNull(alert);
+      Assert.assertEquals(alert.getName(), "Subscription Group Name");
+    } catch (Exception e) {
+      Assert.fail("Exception should not be thrown for valid yaml");
+    }
   }
 
-  @Test(priority=2)
+  @Test
   public void testUpdateDetectionAlertConfig() throws IOException {
     DetectionAlertConfigDTO oldAlertDTO = new DetectionAlertConfigDTO();
     oldAlertDTO.setName("Subscription Group Name");
     oldAlertDTO.setApplication("Random Application");
-    daoRegistry.getDetectionAlertConfigManager().save(oldAlertDTO);
+    long oldId = 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");
+    try {
+      this.yamlResource.updateDetectionAlertConfig(-1, "");
+      Assert.fail("Exception not thrown on empty yaml");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "Cannot find subscription group -1");
+    }
 
     String blankYaml = "";
-    alertDTO = this.yamlResource.updateDetectionAlertConfig(oldAlertDTO, blankYaml, responseMessage);
-    Assert.assertNull(alertDTO);
-    Assert.assertEquals(responseMessage.get("message"), "The config file cannot be blank.");
+    try {
+      this.yamlResource.updateDetectionAlertConfig(oldId, blankYaml);
+      Assert.fail("Exception not thrown on empty yaml");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "The Yaml Payload in the request is empty.");
+    }
 
     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.");
+    try {
+      this.yamlResource.updateDetectionAlertConfig(oldId, inValidYaml);
+      Assert.fail("Exception not thrown on empty yaml");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "Error parsing the Yaml input. 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.");
+    try {
+      this.yamlResource.updateDetectionAlertConfig(oldId, noSubscriptGroupYaml);
+      Assert.fail("Exception not thrown on empty yaml");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "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");
+    try {
+      this.yamlResource.updateDetectionAlertConfig(oldId, appFieldMissingYaml);
+      Assert.fail("Exception not thrown on empty yaml");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "Application field cannot be left empty");
+    }
 
     ApplicationDTO request = new ApplicationDTO();
     request.setApplication("test_application");
@@ -137,10 +171,14 @@ public class YamlResourceTest {
     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");
+    try {
+      alertDTO = this.yamlResource.updateDetectionAlertConfig(oldId, validYaml);
+      Assert.assertNotNull(alertDTO);
+      Assert.assertEquals(alertDTO.getName(), "test_group");
+      Assert.assertEquals(alertDTO.getApplication(), "test_application");
+    } catch (Exception e) {
+      Assert.fail("Exception should not be thrown for valid yaml" + e.getMessage());
+    }
   }
 }
 


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