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/27 18:48:49 UTC

[incubator-pinot] branch master updated: [TE] Endpoint to delete a complete application (#3888)

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 3b09d27  [TE] Endpoint to delete a complete application (#3888)
3b09d27 is described below

commit 3b09d27a285634c24b91c825dc7a908af46bc37b
Author: Akshay Rai <ak...@gmail.com>
AuthorDate: Wed Feb 27 10:48:44 2019 -0800

    [TE] Endpoint to delete a complete application (#3888)
---
 .../api/application/ApplicationResource.java       | 117 +++++++++++++++
 .../dashboard/ThirdEyeDashboardApplication.java    |   4 +
 .../api/application/ApplicationResourceTest.java   | 160 +++++++++++++++++++++
 3 files changed, 281 insertions(+)

diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/api/application/ApplicationResource.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/api/application/ApplicationResource.java
new file mode 100644
index 0000000..639e79a
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/api/application/ApplicationResource.java
@@ -0,0 +1,117 @@
+/*
+ * 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.api.application;
+
+import com.wordnik.swagger.annotations.ApiOperation;
+import com.wordnik.swagger.annotations.ApiParam;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import org.apache.pinot.thirdeye.datalayer.bao.ApplicationManager;
+import org.apache.pinot.thirdeye.datalayer.bao.DetectionAlertConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
+import org.apache.pinot.thirdeye.datalayer.dto.ApplicationDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import org.apache.pinot.thirdeye.datalayer.util.Predicate;
+import org.apache.pinot.thirdeye.detection.ConfigUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.pinot.thirdeye.detection.yaml.YamlDetectionAlertConfigTranslator.*;
+
+
+@Path(value = "/application")
+@Produces(MediaType.APPLICATION_JSON)
+public class ApplicationResource {
+  protected static final Logger LOG = LoggerFactory.getLogger(ApplicationResource.class);
+
+  private final ApplicationManager appDAO;
+  private final MergedAnomalyResultManager anomalyDAO;
+  private final DetectionConfigManager detectionDAO;
+  private final DetectionAlertConfigManager detectionAlertDAO;
+
+
+  public ApplicationResource(ApplicationManager appDAO, MergedAnomalyResultManager anomalyDAO,
+      DetectionConfigManager detectionDAO, DetectionAlertConfigManager detectionAlertDAO) {
+    this.appDAO = appDAO;
+    this.anomalyDAO = anomalyDAO;
+    this.detectionDAO = detectionDAO;
+    this.detectionAlertDAO = detectionAlertDAO;
+  }
+
+  @DELETE
+  @Path("/delete/{application}")
+  @ApiOperation(value = "Delete an application along with associated subscriptions, functions and anomalies")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Consumes(MediaType.APPLICATION_JSON)
+  public Response deleteApplication(
+      @ApiParam(value = "Name of the application to delete")
+      @PathParam("application") String application) throws Exception {
+    Map<String, String> responseMessage = new HashMap<>();
+    LOG.info("[APPLICATION] deleting application " + application);
+
+    try {
+      List<ApplicationDTO> appDTO = appDAO.findByPredicate(Predicate.EQ("application", application));
+      if (appDTO == null || appDTO.isEmpty()) {
+        responseMessage.put("message", "No application with name " + application + " was found.");
+        return Response.status(Response.Status.BAD_REQUEST).entity(responseMessage).build();
+      }
+
+      List<DetectionAlertConfigDTO> subsGroupList = detectionAlertDAO.findByPredicate(Predicate.EQ("application", application));
+      LOG.info("[APPLICATION] Found " + subsGroupList.size() + " subscription groups under application " + application);
+      for (DetectionAlertConfigDTO subsGroup : subsGroupList) {
+        for (long id : ConfigUtils.getLongs(subsGroup.getProperties().get(PROP_DETECTION_CONFIG_IDS))) {
+          DetectionConfigDTO function = this.detectionDAO.findById(id);
+          if (function != null) {
+            List<MergedAnomalyResultDTO> anomalies =
+                new ArrayList<>(anomalyDAO.findByPredicate(Predicate.EQ("detectionConfigId", function.getId())));
+            for (MergedAnomalyResultDTO anomaly : anomalies) {
+              anomalyDAO.delete(anomaly);
+            }
+            detectionDAO.delete(function);
+            LOG.info("[APPLICATION] detection function " + function.getName() + " and all anomalies have been deleted.");
+          }
+        }
+        detectionAlertDAO.delete(subsGroup);
+        LOG.info("[APPLICATION] subscription group " + subsGroup.getName() + " deleted.");
+      }
+      appDAO.delete(appDTO.get(0));
+    } catch (Exception e) {
+      LOG.error("[APPLICATION] Error while deleting application " + application, e);
+      responseMessage.put("message", "Error while deleting application. Error = " + e.getMessage());
+      return Response.serverError().entity(responseMessage).build();
+    }
+
+    LOG.info("[APPLICATION] Successfully deleted application " + application);
+    responseMessage.put("message", "Successfully deleted application.");
+    return Response.ok().entity(responseMessage).build();
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/ThirdEyeDashboardApplication.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/ThirdEyeDashboardApplication.java
index db8d237..d4ffcf4 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/ThirdEyeDashboardApplication.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/ThirdEyeDashboardApplication.java
@@ -24,6 +24,7 @@ import com.google.common.cache.CacheBuilder;
 import org.apache.pinot.thirdeye.anomaly.detection.DetectionJobScheduler;
 import org.apache.pinot.thirdeye.anomaly.onboard.DetectionOnboardResource;
 import org.apache.pinot.thirdeye.anomalydetection.alertFilterAutotune.AlertFilterAutotuneFactory;
+import org.apache.pinot.thirdeye.api.application.ApplicationResource;
 import org.apache.pinot.thirdeye.common.time.TimeGranularity;
 import org.apache.pinot.thirdeye.auth.Credentials;
 import org.apache.pinot.thirdeye.auth.ThirdEyeAuthFilter;
@@ -187,6 +188,9 @@ public class ThirdEyeDashboardApplication
         DAO_REGISTRY.getMergedAnomalyResultDAO(), DAO_REGISTRY.getAnomalyFunctionDAO(),
         DAO_REGISTRY.getMetricConfigDAO(), DAO_REGISTRY.getDatasetConfigDAO(), DAO_REGISTRY.getAlertConfigDAO(),
         DAO_REGISTRY.getDetectionConfigManager(), DAO_REGISTRY.getDetectionAlertConfigManager()));
+    env.jersey().register(new ApplicationResource(
+        DAO_REGISTRY.getApplicationDAO(), DAO_REGISTRY.getMergedAnomalyResultDAO(),
+        DAO_REGISTRY.getDetectionConfigManager(), DAO_REGISTRY.getDetectionAlertConfigManager()));
     env.jersey().register(new DetectionOnboardResource(
         DAO_REGISTRY.getTaskDAO(), DAO_REGISTRY.getAnomalyFunctionDAO()));
     env.jersey().register(new DetectionResource());
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/api/application/ApplicationResourceTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/api/application/ApplicationResourceTest.java
new file mode 100644
index 0000000..578f9ad
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/api/application/ApplicationResourceTest.java
@@ -0,0 +1,160 @@
+package org.apache.pinot.thirdeye.api.application;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import javax.ws.rs.core.Response;
+import org.apache.pinot.thirdeye.datalayer.bao.AlertConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.AnomalyFunctionManager;
+import org.apache.pinot.thirdeye.datalayer.bao.ApplicationManager;
+import org.apache.pinot.thirdeye.datalayer.bao.DAOTestBase;
+import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.DetectionAlertConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
+import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
+import org.apache.pinot.thirdeye.datalayer.dto.AlertConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.AnomalyFunctionDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.ApplicationDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import org.apache.pinot.thirdeye.datalayer.pojo.AlertConfigBean;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+
+public class ApplicationResourceTest {
+  DAOTestBase testBase;
+  ApplicationResource resource;
+
+  ApplicationManager appDAO;
+  MergedAnomalyResultManager anomalyDAO;
+  DetectionConfigManager detectionDAO;
+  DetectionAlertConfigManager detectionAlertDAO;
+  MetricConfigManager metricDAO;
+  DatasetConfigManager datasetDAO;
+  AnomalyFunctionManager functionDAO;
+
+  List<Long> anomalyIds;
+  List<Long> functionIds;
+  List<Long> alertIds;
+
+  @BeforeMethod
+  public void beforeMethod() {
+    this.testBase = DAOTestBase.getInstance();
+
+    this.metricDAO = DAORegistry.getInstance().getMetricConfigDAO();
+    this.datasetDAO = DAORegistry.getInstance().getDatasetConfigDAO();
+    this.functionDAO = DAORegistry.getInstance().getAnomalyFunctionDAO();
+
+    // Application
+    this.appDAO = DAORegistry.getInstance().getApplicationDAO();
+    this.appDAO.save(makeApplication("myApplicationA"));
+    this.appDAO.save(makeApplication("myApplicationB"));
+
+    // functions
+    this.functionDAO = DAORegistry.getInstance().getAnomalyFunctionDAO();
+    this.functionIds = new ArrayList<>();
+    this.functionIds.add(this.functionDAO.save(makeDetection("myFunctionA")));
+    this.functionIds.add(this.functionDAO.save(makeDetection("myFunctionB")));
+    this.functionIds.add(this.functionDAO.save(makeDetection("myFunctionC")));
+    for (Long id : this.functionIds) {
+      Assert.assertNotNull(id);
+    }
+
+    // anomalies
+    this.anomalyDAO = DAORegistry.getInstance().getMergedAnomalyResultDAO();
+    this.anomalyIds = new ArrayList<>();
+    this.anomalyIds.add(this.anomalyDAO.save(makeAnomaly(100, 500, this.functionIds.get(0), "test_metric", "test_dataset"))); // func A
+    this.anomalyIds.add(this.anomalyDAO.save(makeAnomaly(800, 1200, this.functionIds.get(0), "test_metric", "test_dataset"))); // func A
+    this.anomalyIds.add(this.anomalyDAO.save(makeAnomaly(300, 1500, this.functionIds.get(1), "test_metric", "test_dataset"))); // func B
+    this.anomalyIds.add(this.anomalyDAO.save(makeAnomaly(300, 1600, this.functionIds.get(2), "test_metric", "test_dataset"))); // func C
+    this.anomalyIds.add(this.anomalyDAO.save(makeAnomaly(300, 1600, this.functionIds.get(2), "test_metric_2", "test_dataset"))); // func C
+
+    for (Long id : this.anomalyIds) {
+      Assert.assertNotNull(id);
+    }
+
+    // alerts
+    this.detectionAlertDAO = DAORegistry.getInstance().getDetectionAlertConfigManager();
+    this.alertIds = new ArrayList<>();
+    this.alertIds.add(this.detectionAlertDAO.save(makeAlert("myGroupA", "myApplicationA", Arrays.asList(this.functionIds.get(0), this.functionIds.get(1))))); // funcA, funcB
+    this.alertIds.add(this.detectionAlertDAO.save(makeAlert("myGroupB", "myApplicationB", Collections.singletonList(this.functionIds.get(2))))); // none
+
+    for (Long id : this.alertIds) {
+      Assert.assertNotNull(id);
+    }
+
+    // new framework detectors
+    this.detectionDAO = DAORegistry.getInstance().getDetectionConfigManager();
+
+    // new framework alerts
+    this.detectionAlertDAO = DAORegistry.getInstance().getDetectionAlertConfigManager();
+
+    // resource
+    this.resource = new ApplicationResource(this.appDAO, this.anomalyDAO, this.detectionDAO, this.detectionAlertDAO);
+  }
+
+  @AfterMethod(alwaysRun = true)
+  public void afterMethod() {
+    if (this.testBase != null) {
+      this.testBase.cleanup();
+    }
+  }
+
+  @Test
+  public void testDeleteApplication() throws Exception {
+    Response response = this.resource.deleteApplication("myApplicationA");
+    Assert.assertEquals(response.getStatus(), 200);
+
+    // Expected:
+    // Application myApplicationA should be removed
+    // Subscription Group myGroupA should be removed
+    // Anomaly Functions myFunctionA and myFunctionB should be removed
+    // Anomalies detected by myFunctionA and myFunctionA should be removed
+    Assert.assertEquals(this.appDAO.findAll().size(), 1);
+    Assert.assertEquals(this.detectionAlertDAO.findAll().size(), 1);
+    Assert.assertEquals(this.functionDAO.findAll().size(), 1);
+    Assert.assertEquals(this.anomalyDAO.findAll().size(), 2);
+  }
+
+  private ApplicationDTO makeApplication(String applicationName) {
+    ApplicationDTO appDTO = new ApplicationDTO();
+    appDTO.setApplication(applicationName);
+    appDTO.setRecipients("test@test.com");
+    return appDTO;
+  }
+
+  private AnomalyFunctionDTO makeDetection(String name) {
+    AnomalyFunctionDTO function = new AnomalyFunctionDTO();
+    function.setFunctionName(name);
+    return function;
+  }
+
+  private MergedAnomalyResultDTO makeAnomaly(long start, long end, Long functionId, String metric, String dataset) {
+    MergedAnomalyResultDTO anomaly = new MergedAnomalyResultDTO();
+    anomaly.setStartTime(start);
+    anomaly.setEndTime(end);
+    anomaly.setMetric(metric);
+    anomaly.setCollection(dataset);
+    anomaly.setDetectionConfigId(functionId);
+    return anomaly;
+  }
+
+  private DetectionAlertConfigDTO makeAlert(String name, String application, List<Long> functionIds) {
+    DetectionAlertConfigDTO subsGroup = new DetectionAlertConfigDTO();
+    subsGroup.setName(name);
+    subsGroup.setApplication(application);
+    Map<String, Object> properties = new HashMap<>();
+    properties.put("detectionConfigIds", functionIds);
+    subsGroup.setProperties(properties);
+    return subsGroup;
+  }
+}


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