You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ji...@apache.org on 2020/05/18 22:37:32 UTC

[incubator-pinot] branch alerts-pagination updated (c358db2 -> 2a7e703)

This is an automated email from the ASF dual-hosted git repository.

jihao pushed a change to branch alerts-pagination
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git.


 discard c358db2  [TE] alerts search and pagination endpoint
     new 2a7e703  [TE] alerts search and pagination endpoint

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (c358db2)
            \
             N -- N -- N   refs/heads/alerts-pagination (2a7e703)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../resources/v2/alerts/AlertResource.java         |  6 +-
 .../resources/v2/alerts/AlertSearchFilter.java     | 28 +++++---
 .../resources/v2/alerts/AlertSearcher.java         | 10 +--
 .../thirdeye/datalayer/bao/AbstractManager.java    | 13 +++-
 .../resources/v2/alerts/AlertSearcherTest.java     | 75 ++++++++++++++++++++++
 5 files changed, 115 insertions(+), 17 deletions(-)
 create mode 100644 thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertSearcherTest.java


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


[incubator-pinot] 01/01: [TE] alerts search and pagination endpoint

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jihao pushed a commit to branch alerts-pagination
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 2a7e70347c85979af47bdb54f3725f5504842ecc
Author: Jihao Zhang <ji...@linkedin.com>
AuthorDate: Thu May 14 15:26:41 2020 -0700

    [TE] alerts search and pagination endpoint
---
 .../dashboard/ThirdEyeDashboardApplication.java    |   2 +
 .../resources/v2/alerts/AlertResource.java         |  79 +++++++
 .../resources/v2/alerts/AlertSearchFilter.java     | 157 +++++++++++++
 .../resources/v2/alerts/AlertSearcher.java         | 259 +++++++++++++++++++++
 .../thirdeye/datalayer/bao/AbstractManager.java    |  29 ++-
 .../datalayer/bao/jdbc/AbstractManagerImpl.java    |  15 ++
 .../thirdeye/datalayer/dao/GenericPojoDao.java     |  82 +++++++
 .../datalayer/entity/DetectionConfigIndex.java     |  18 ++
 .../thirdeye/datalayer/util/SqlQueryBuilder.java   |  40 ++++
 .../thirdeye/detection/DetectionResource.java      |   1 -
 .../src/main/resources/schema/create-schema.sql    |   4 +
 .../resources/v2/alerts/AlertSearcherTest.java     |  75 ++++++
 12 files changed, 759 insertions(+), 2 deletions(-)

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 1ac0973..ccd70d3 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
@@ -44,6 +44,7 @@ import org.apache.pinot.thirdeye.dashboard.resources.CacheResource;
 import org.apache.pinot.thirdeye.dashboard.resources.CustomizedEventResource;
 import org.apache.pinot.thirdeye.dashboard.resources.DashboardResource;
 import org.apache.pinot.thirdeye.dashboard.resources.DatasetConfigResource;
+import org.apache.pinot.thirdeye.dashboard.resources.v2.alerts.AlertResource;
 import org.apache.pinot.thirdeye.dashboard.resources.DetectionJobResource;
 import org.apache.pinot.thirdeye.dashboard.resources.EmailResource;
 import org.apache.pinot.thirdeye.dashboard.resources.EntityManagerResource;
@@ -194,6 +195,7 @@ public class ThirdEyeDashboardApplication
     env.jersey().register(new YamlResource(config.getAlerterConfiguration(), config.getDetectionPreviewConfig(),
         config.getAlertOnboardingPermitPerSecond()));
     env.jersey().register(new SqlDataSourceResource());
+    env.jersey().register(new AlertResource());
 
     TimeSeriesLoader timeSeriesLoader = new DefaultTimeSeriesLoader(
         DAO_REGISTRY.getMetricConfigDAO(), DAO_REGISTRY.getDatasetConfigDAO(),
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertResource.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertResource.java
new file mode 100644
index 0000000..2e7cb3e
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertResource.java
@@ -0,0 +1,79 @@
+/*
+ * 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.dashboard.resources.v2.alerts;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import java.util.List;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import org.apache.pinot.thirdeye.api.Constants;
+
+
+/**
+ * The Alert resource.
+ */
+@Path(value = "/alerts")
+@Produces(MediaType.APPLICATION_JSON)
+@Api(tags = {Constants.DETECTION_TAG})
+public class AlertResource {
+  private final AlertSearcher alertSearcher;
+
+  /**
+   * Instantiates a new Alert resource.
+   */
+  public AlertResource() {
+    this.alertSearcher = new AlertSearcher();
+  }
+
+  /**
+   * Search the alerts with result pagination
+   *
+   * @param limit the returned result limit
+   * @param offset the offset of the start position
+   * @param applications the applications for the alerts
+   * @param subscriptionGroups the subscription groups for the alerts
+   * @param names the names for the alerts
+   * @param createdBy the owners for the alerts
+   * @param ruleTypes the rule types for the alerts
+   * @param metrics the metrics for the alerts
+   * @param datasets the datasets for the alerts
+   * @param active if the alert is active
+   * @return the response
+   */
+  @GET
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation("Search and paginate alerts according to the parameters")
+  public Response findAlerts(@QueryParam("limit") @DefaultValue("10") long limit,
+      @QueryParam("offset") @DefaultValue("0") long offset, @QueryParam("application") List<String> applications,
+      @QueryParam("subscriptionGroup") List<String> subscriptionGroups, @QueryParam("names") List<String> names,
+      @QueryParam("createdBy") List<String> createdBy, @QueryParam("ruleType") List<String> ruleTypes,
+      @QueryParam("metric") List<String> metrics, @QueryParam("dataset") List<String> datasets,
+      @QueryParam("active") Boolean active) {
+    AlertSearchFilter searchFilter = new AlertSearchFilter(applications, subscriptionGroups, names, createdBy, ruleTypes, metrics, datasets, active);
+    return Response.ok().entity(this.alertSearcher.search(searchFilter, limit, offset)).build();
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertSearchFilter.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertSearchFilter.java
new file mode 100644
index 0000000..0c488ff
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertSearchFilter.java
@@ -0,0 +1,157 @@
+/*
+ * 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.dashboard.resources.v2.alerts;
+
+import java.util.Collections;
+import java.util.List;
+
+
+/**
+ * The type Alert search filter.
+ */
+public class AlertSearchFilter {
+  private final List<String> applications;
+  private final List<String> subscriptionGroups;
+  private final List<String> createdBy;
+  private final List<String> ruleTypes;
+  private final List<String> metrics;
+  private final List<String> datasets;
+  private final List<String> names;
+  private final Boolean active;
+
+  public AlertSearchFilter() {
+    this.applications = Collections.emptyList();
+    this.subscriptionGroups = Collections.emptyList();
+    this.createdBy = Collections.emptyList();
+    this.ruleTypes = Collections.emptyList();
+    this.datasets = Collections.emptyList();
+    this.metrics = Collections.emptyList();
+    this.names = Collections.emptyList();
+    this.active = null;
+  }
+
+  /**
+   * Instantiates a new Alert search filter.
+   *
+   * @param applications the applications
+   * @param subscriptionGroups the subscription groups
+   * @param names the names
+   * @param createdBy the createdBy
+   * @param ruleTypes the rule types
+   * @param metrics the metrics
+   * @param datasets the datasets
+   * @param active the active
+   */
+  public AlertSearchFilter(List<String> applications, List<String> subscriptionGroups, List<String> names,
+      List<String> createdBy, List<String> ruleTypes, List<String> metrics, List<String> datasets, Boolean active) {
+    this.applications = applications;
+    this.subscriptionGroups = subscriptionGroups;
+    this.names = names;
+    this.createdBy = createdBy;
+    this.ruleTypes = ruleTypes;
+    this.metrics = metrics;
+    this.datasets = datasets;
+    this.active = active;
+  }
+
+  /**
+   * Gets applications.
+   *
+   * @return the applications
+   */
+  public List<String> getApplications() {
+    return applications;
+  }
+
+  /**
+   * Gets subscription groups.
+   *
+   * @return the subscription groups
+   */
+  public List<String> getSubscriptionGroups() {
+    return subscriptionGroups;
+  }
+
+  /**
+   * Gets createdBy.
+   *
+   * @return the owners
+   */
+  public List<String> getCreatedBy() {
+    return createdBy;
+  }
+
+  /**
+   * Gets rule types.
+   *
+   * @return the rule types
+   */
+  public List<String> getRuleTypes() {
+    return ruleTypes;
+  }
+
+  /**
+   * Gets metrics.
+   *
+   * @return the metrics
+   */
+  public List<String> getMetrics() {
+    return metrics;
+  }
+
+  /**
+   * Gets datasets.
+   *
+   * @return the datasets
+   */
+  public List<String> getDatasets() {
+    return datasets;
+  }
+
+  /**
+   * Gets names.
+   *
+   * @return the names
+   */
+  public List<String> getNames() {
+    return names;
+  }
+
+  /**
+   * Gets active.
+   *
+   * @return the active
+   */
+  public Boolean getActive() {
+    return active;
+  }
+
+  /**
+   * If all the search filters are empty.
+   *
+   * @return the boolean value of the result
+   */
+  public boolean isEmpty() {
+    return this.applications.isEmpty() && this.subscriptionGroups.isEmpty() && this.names.isEmpty()
+        && this.createdBy.isEmpty() && this.ruleTypes.isEmpty() && this.metrics.isEmpty() && this.datasets.isEmpty()
+        && active == null;
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertSearcher.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertSearcher.java
new file mode 100644
index 0000000..78fe3a1
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertSearcher.java
@@ -0,0 +1,259 @@
+/*
+ * 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.dashboard.resources.v2.alerts;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Multimap;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+import org.apache.commons.collections4.MapUtils;
+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.MetricConfigManager;
+import org.apache.pinot.thirdeye.datalayer.dto.AbstractDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.util.Predicate;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
+import org.apache.pinot.thirdeye.formatter.DetectionConfigFormatter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The Alert searcher.
+ */
+public class AlertSearcher {
+  private static final Logger LOG = LoggerFactory.getLogger(AlertSearcher.class.getName());
+  private final DetectionConfigManager detectionConfigDAO;
+  private final DetectionAlertConfigManager detectionAlertConfigDAO;
+  private final MetricConfigManager metricDAO;
+  private final DatasetConfigManager datasetDAO;
+  private final DetectionConfigFormatter detectionConfigFormatter;
+
+  /**
+   * The Alert search query.
+   */
+  static class AlertSearchQuery {
+    /**
+     * The Search filter.
+     */
+    final AlertSearchFilter searchFilter;
+    /**
+     * The Limit.
+     */
+    final long limit;
+    /**
+     * The Offset.
+     */
+    final long offset;
+
+    /**
+     * Instantiates a new Alert search query.
+     *
+     * @param searchFilter the search filter
+     * @param limit the limit
+     * @param offset the offset
+     */
+    public AlertSearchQuery(AlertSearchFilter searchFilter, long limit, long offset) {
+      this.searchFilter = searchFilter;
+      this.limit = limit;
+      this.offset = offset;
+    }
+  }
+
+  /**
+   * Instantiates a new Alert searcher.
+   */
+  public AlertSearcher() {
+    this.detectionConfigDAO = DAORegistry.getInstance().getDetectionConfigManager();
+    this.detectionAlertConfigDAO = DAORegistry.getInstance().getDetectionAlertConfigManager();
+    this.metricDAO = DAORegistry.getInstance().getMetricConfigDAO();
+    this.datasetDAO = DAORegistry.getInstance().getDatasetConfigDAO();
+    this.detectionConfigFormatter = new DetectionConfigFormatter(metricDAO, datasetDAO);
+  }
+
+  /**
+   * Search and retrive all the alerts matching to the search filter and limits.
+   *
+   * @param searchFilter the search filter
+   * @param limit the limit
+   * @param offset the offset
+   * @return the result
+   */
+  public Map<String, Object> search(AlertSearchFilter searchFilter, long limit, long offset) {
+    AlertSearchQuery searchQuery = new AlertSearchQuery(searchFilter, limit, offset);
+    List<DetectionAlertConfigDTO> subscriptionGroups = findRelatedSubscriptionGroups(searchQuery);
+    List<DetectionConfigDTO> detectionConfigs = findDetectionConfig(searchQuery, subscriptionGroups);
+    return getResult(searchQuery, subscriptionGroups, detectionConfigs);
+  }
+
+  private List<DetectionAlertConfigDTO> findRelatedSubscriptionGroups(AlertSearchQuery searchQuery) {
+    AlertSearchFilter searchFilter = searchQuery.searchFilter;
+    List<Predicate> predicates = new ArrayList<>();
+    if (!searchFilter.getApplications().isEmpty()) {
+      predicates.add(Predicate.IN("application", searchFilter.getApplications().toArray()));
+    }
+    if (!searchFilter.getSubscriptionGroups().isEmpty()) {
+      predicates.add(Predicate.IN("name", searchFilter.getSubscriptionGroups().toArray()));
+    }
+    if (!predicates.isEmpty()) {
+      return this.detectionAlertConfigDAO.findByPredicate(Predicate.AND(predicates.toArray(new Predicate[0])));
+    } else {
+      return this.detectionAlertConfigDAO.findAll();
+    }
+  }
+
+  private List<DetectionConfigDTO> findDetectionConfig(AlertSearchQuery searchQuery,
+      List<DetectionAlertConfigDTO> subscriptionGroups) {
+    AlertSearchFilter searchFilter = searchQuery.searchFilter;
+    if (searchFilter.isEmpty()) {
+      // if no search filter is applied, by default, retrieve the paginated result from db
+      return this.detectionConfigDAO.list(searchQuery.limit, searchQuery.offset);
+    }
+
+    // look up and run the search filters on the detection config index
+    List<DetectionConfigDTO> indexedResult = new ArrayList<>();
+    List<Predicate> indexPredicates = new ArrayList<>();
+    if (!searchFilter.getApplications().isEmpty() || !searchFilter.getSubscriptionGroups().isEmpty()) {
+      Set<Long> detectionConfigIds = new TreeSet<>();
+      for (DetectionAlertConfigDTO subscriptionGroup : subscriptionGroups) {
+        detectionConfigIds.addAll(subscriptionGroup.getVectorClocks().keySet());
+      }
+      indexPredicates.add(Predicate.IN("baseId", detectionConfigIds.toArray()));
+    }
+    if (!searchFilter.getCreatedBy().isEmpty()) {
+      indexPredicates.add(Predicate.IN("createdBy", searchFilter.getCreatedBy().toArray()));
+    }
+    if (!searchFilter.getNames().isEmpty()) {
+      indexPredicates.add(Predicate.IN("name", searchFilter.getNames().toArray()));
+    }
+    if (searchFilter.getActive() != null) {
+      indexPredicates.add(Predicate.EQ("active", searchFilter.getActive() ? 1 : 0));
+    }
+    if (!indexPredicates.isEmpty()) {
+      indexedResult = this.detectionConfigDAO.findByPredicate(Predicate.AND(indexPredicates.toArray(new Predicate[0])));
+    }
+
+    // for metrics, datasets, rule types filters, run the search filters in the generic table
+    List<DetectionConfigDTO> jsonValResult = new ArrayList<>();
+    List<Predicate> jsonValPredicates = new ArrayList<>();
+    if (!searchFilter.getRuleTypes().isEmpty()) {
+      List<Predicate> ruleTypePredicates = new ArrayList<>();
+      for (String ruleType : searchFilter.getRuleTypes()) {
+        ruleTypePredicates.add(Predicate.LIKE("jsonVal", "%componentSpecs%:" + ruleType + "\"%"));
+      }
+      jsonValPredicates.add(Predicate.OR(ruleTypePredicates.toArray(new Predicate[0])));
+    }
+
+    Set<Long> metricIds = new HashSet<>();
+    if (!searchFilter.getMetrics().isEmpty()) {
+      for (String metric : searchFilter.getMetrics()) {
+        metricIds =
+            this.metricDAO.findByMetricName(metric).stream().map(AbstractDTO::getId).collect(Collectors.toSet());
+      }
+    }
+
+    if (!searchFilter.getDatasets().isEmpty()) {
+      for (String dataset : searchFilter.getDatasets()) {
+        metricIds.retainAll(
+            this.metricDAO.findByDataset(dataset).stream().map(AbstractDTO::getId).collect(Collectors.toSet()));
+      }
+    }
+
+    if (!metricIds.isEmpty()) {
+      List<Predicate> metricUrnPredicates = new ArrayList<>();
+      for (Long id : metricIds) {
+        metricUrnPredicates.add(Predicate.LIKE("jsonVal", "%thirdeye:metric:" + id + "%"));
+      }
+      jsonValPredicates.add(Predicate.OR(metricUrnPredicates.toArray(new Predicate[0])));
+    }
+
+    if (!jsonValPredicates.isEmpty()) {
+      jsonValResult =
+          this.detectionConfigDAO.findByPredicateJsonVal(Predicate.AND(jsonValPredicates.toArray(new Predicate[0])));
+    }
+
+    List<DetectionConfigDTO> result;
+    if (!jsonValPredicates.isEmpty() && !indexPredicates.isEmpty()) {
+      // merge the result from both tables
+      result = jsonValResult.stream().filter(indexedResult::contains).collect(Collectors.toList());
+    } else {
+      jsonValResult.addAll(indexedResult);
+      result = jsonValResult;
+    }
+    return result;
+  }
+
+  /**
+   * Format and generate the final search result
+   */
+  private Map<String, Object> getResult(AlertSearchQuery searchQuery, List<DetectionAlertConfigDTO> subscriptionGroups,
+      List<DetectionConfigDTO> detectionConfigs) {
+    long count;
+    if (searchQuery.searchFilter.isEmpty()) {
+      // if not filter is applied, execute count query
+      count = this.detectionConfigDAO.count();
+    } else {
+      // count and limit the filtered results
+      count = detectionConfigs.size();
+      if (searchQuery.offset >= count) {
+        // requested page is out of bound
+        detectionConfigs.clear();
+      } else {
+        detectionConfigs = detectionConfigs.subList((int) searchQuery.offset,
+            (int) Math.min(searchQuery.offset + searchQuery.limit, count));
+      }
+    }
+
+    // format the results
+    List<Map<String, Object>> alerts = detectionConfigs.parallelStream().map(config -> {
+      try {
+        return this.detectionConfigFormatter.format(config);
+      } catch (Exception e) {
+        LOG.warn("formatting detection config failed {}", config.getId(), e);
+        return null;
+      }
+    }).filter(Objects::nonNull).collect(Collectors.toList());
+
+    // join detections with subscription groups
+    Multimap<Long, String> detectionIdToSubscriptionGroups = ArrayListMultimap.create();
+    for (DetectionAlertConfigDTO subscriptionGroup : subscriptionGroups) {
+      for (long detectionConfigId : subscriptionGroup.getVectorClocks().keySet()) {
+        detectionIdToSubscriptionGroups.put(detectionConfigId, subscriptionGroup.getName());
+      }
+    }
+    for (Map<String, Object> alert : alerts) {
+      alert.put("subscriptionGroup", detectionIdToSubscriptionGroups.get(MapUtils.getLong(alert, "id")));
+    }
+
+    return ImmutableMap.of("count", count, "limit", searchQuery.limit, "offset", searchQuery.offset, "elements",
+        alerts);
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/AbstractManager.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/AbstractManager.java
index 69e6354..9faa14d 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/AbstractManager.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/AbstractManager.java
@@ -21,7 +21,7 @@ package org.apache.pinot.thirdeye.datalayer.bao;
 
 import java.util.List;
 import java.util.Map;
-
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.pinot.thirdeye.datalayer.dto.AbstractDTO;
 import org.apache.pinot.thirdeye.datalayer.util.Predicate;
 
@@ -57,4 +57,31 @@ public interface AbstractManager<E extends AbstractDTO> {
   List<Long> findIdsByPredicate(Predicate predicate);
 
   int update(E entity, Predicate predicate);
+
+  /**
+   * Find the entities based on the JSON value predicate
+   * @param predicate the predicate
+   * @return the list of entities that match with the predicate
+   */
+  default List<E> findByPredicateJsonVal(Predicate predicate) {
+    throw new NotImplementedException("Not Implemented");
+  }
+
+  /**
+   * List the entities with pagination
+   * @param limit the limit for the number of elements returned
+   * @param offset the offset position
+   * @return the list of entities ordered by id in descending order
+   */
+  default List<E> list(long limit, long offset) {
+    throw new NotImplementedException("Not Implemented");
+  }
+
+  /**
+   * Count how many entities are there in the table
+   * @return the number of total entities
+   */
+  default long count()  {
+    throw new NotImplementedException("Not Implemented");
+  }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/jdbc/AbstractManagerImpl.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/jdbc/AbstractManagerImpl.java
index c13cea9..81a8535 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/jdbc/AbstractManagerImpl.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/bao/jdbc/AbstractManagerImpl.java
@@ -171,6 +171,21 @@ public abstract class AbstractManagerImpl<E extends AbstractDTO> implements Abst
     return genericPojoDao.getIdsByPredicate(predicate, beanClass);
   }
 
+  @Override
+  public List<E> list(long limit, long offset) {
+    return convertBeanListToDTOList(genericPojoDao.list(beanClass, limit, offset));
+  }
+
+  @Override
+  public List<E> findByPredicateJsonVal(Predicate predicate) {
+    return convertBeanListToDTOList(genericPojoDao.getByPredicateJsonVal(predicate, beanClass));
+  }
+
+  @Override
+  public long count() {
+    return genericPojoDao.count(beanClass);
+  }
+
   protected List<E> convertBeanListToDTOList(List<? extends AbstractBean> beans) {
     List<E> result = new ArrayList<>();
     for (AbstractBean bean : beans) {
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dao/GenericPojoDao.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dao/GenericPojoDao.java
index 8e983c5..227c9b1 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dao/GenericPojoDao.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/dao/GenericPojoDao.java
@@ -442,6 +442,88 @@ public class GenericPojoDao {
     }
   }
 
+  public <E extends AbstractBean> List<E> list(final Class<E> beanClass, long limit, long offset) {
+    long tStart = System.nanoTime();
+    try {
+      return runTask(connection -> {
+        List<GenericJsonEntity> entities;
+        Predicate predicate = Predicate.EQ("beanClass", beanClass.getName());
+        try (PreparedStatement selectStatement =
+            sqlQueryBuilder.createfindByParamsStatementWithLimit(connection, GenericJsonEntity.class, predicate, limit, offset)) {
+          try (ResultSet resultSet = selectStatement.executeQuery()) {
+            entities = genericResultSetMapper.mapAll(resultSet, GenericJsonEntity.class);
+          }
+        }
+        List<E> result = new ArrayList<>();
+        if (entities != null) {
+          for (GenericJsonEntity entity : entities) {
+            ThirdeyeMetricsUtil.dbReadByteCounter.inc(entity.getJsonVal().length());
+            E e = OBJECT_MAPPER.readValue(entity.getJsonVal(), beanClass);
+            e.setId(entity.getId());
+            e.setUpdateTime(entity.getUpdateTime());
+            result.add(e);
+          }
+        }
+        return result;
+      }, Collections.emptyList());
+    } finally {
+      ThirdeyeMetricsUtil.dbReadCallCounter.inc();
+      ThirdeyeMetricsUtil.dbReadDurationCounter.inc(System.nanoTime() - tStart);
+    }
+  }
+
+  public <E extends AbstractBean> List<E> getByPredicateJsonVal(Predicate predicate, final Class<E> beanClass) {
+    long tStart = System.nanoTime();
+    try {
+      return runTask(connection -> {
+        List<GenericJsonEntity> entities;
+        Predicate p = Predicate.AND(predicate, Predicate.EQ("beanClass", beanClass.getName()));
+        try (PreparedStatement selectStatement =
+            sqlQueryBuilder.createFindByParamsStatement(connection, GenericJsonEntity.class, p)) {
+          try (ResultSet resultSet = selectStatement.executeQuery()) {
+            entities = genericResultSetMapper.mapAll(resultSet, GenericJsonEntity.class);
+          }
+        }
+        List<E> result = new ArrayList<>();
+        if (entities != null) {
+          for (GenericJsonEntity entity : entities) {
+            ThirdeyeMetricsUtil.dbReadByteCounter.inc(entity.getJsonVal().length());
+            E e = OBJECT_MAPPER.readValue(entity.getJsonVal(), beanClass);
+            e.setId(entity.getId());
+            e.setUpdateTime(entity.getUpdateTime());
+            result.add(e);
+          }
+        }
+        return result;
+      }, Collections.emptyList());
+    } finally {
+      ThirdeyeMetricsUtil.dbReadCallCounter.inc();
+      ThirdeyeMetricsUtil.dbReadDurationCounter.inc(System.nanoTime() - tStart);
+    }
+  }
+
+  public <E extends AbstractBean> long count(final Class<E> beanClass) {
+    long tStart = System.nanoTime();
+    try {
+      return runTask(connection -> {
+        PojoInfo pojoInfo = pojoInfoMap.get(beanClass);
+        try (PreparedStatement selectStatement =
+            sqlQueryBuilder.createCountStatement(connection, pojoInfo.indexEntityClass)) {
+          try (ResultSet resultSet = selectStatement.executeQuery()) {
+            if (resultSet.next()) {
+              return resultSet.getInt(1);
+            } else {
+              throw new IllegalStateException("can't parse count query response");
+            }
+          }
+        }
+      }, -1);
+    } finally {
+      ThirdeyeMetricsUtil.dbReadCallCounter.inc();
+      ThirdeyeMetricsUtil.dbReadDurationCounter.inc(System.nanoTime() - tStart);
+    }
+  }
+
   public <E extends AbstractBean> E get(final Long id, final Class<E> pojoClass) {
     long tStart = System.nanoTime();
     try {
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/entity/DetectionConfigIndex.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/entity/DetectionConfigIndex.java
index 39e9d64..2c4011f 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/entity/DetectionConfigIndex.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/entity/DetectionConfigIndex.java
@@ -21,6 +21,8 @@ package org.apache.pinot.thirdeye.datalayer.entity;
 
 public class DetectionConfigIndex extends AbstractIndexEntity {
   String name;
+  boolean active;
+  String createdBy;
 
   public String getName() {
     return name;
@@ -29,4 +31,20 @@ public class DetectionConfigIndex extends AbstractIndexEntity {
   public void setName(String name) {
     this.name = name;
   }
+
+  public boolean isActive() {
+    return active;
+  }
+
+  public void setActive(boolean active) {
+    this.active = active;
+  }
+
+  public String getCreatedBy() {
+    return createdBy;
+  }
+
+  public void setCreatedBy(String createdBy) {
+    this.createdBy = createdBy;
+  }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/util/SqlQueryBuilder.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/util/SqlQueryBuilder.java
index 88df8de..6f4076c 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/util/SqlQueryBuilder.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/datalayer/util/SqlQueryBuilder.java
@@ -338,6 +338,46 @@ public class SqlQueryBuilder {
     return prepareStatement;
   }
 
+  public PreparedStatement createfindByParamsStatementWithLimit(Connection connection,
+      Class<? extends AbstractEntity> entityClass, Predicate predicate, Long limit, Long offset) throws Exception {
+    String tableName = entityMappingHolder.tableToEntityNameMap.inverse().get(entityClass.getSimpleName());
+    BiMap<String, String> entityNameToDBNameMapping =
+        entityMappingHolder.columnMappingPerTable.get(tableName).inverse();
+    StringBuilder sqlBuilder = new StringBuilder("SELECT * FROM " + tableName);
+    StringBuilder whereClause = new StringBuilder(" WHERE ");
+    List<Pair<String, Object>> parametersList = new ArrayList<>();
+    generateWhereClause(entityNameToDBNameMapping, predicate, parametersList, whereClause);
+    sqlBuilder.append(whereClause.toString());
+    sqlBuilder.append(" ORDER BY id DESC");
+    if (limit != null) {
+      sqlBuilder.append(" LIMIT ").append(limit);
+    }
+    if (offset != null) {
+      sqlBuilder.append(" OFFSET ").append(offset);
+    }
+    PreparedStatement prepareStatement = connection.prepareStatement(sqlBuilder.toString());
+    int parameterIndex = 1;
+    LinkedHashMap<String, ColumnInfo> columnInfoMap =
+        entityMappingHolder.columnInfoPerTable.get(tableName);
+    for (Pair<String, Object> pair : parametersList) {
+      String dbFieldName = pair.getKey();
+      ColumnInfo info = columnInfoMap.get(dbFieldName);
+      Preconditions.checkNotNull(info, String.format("Found field '%s' but expected %s", dbFieldName, columnInfoMap.keySet()));
+      prepareStatement.setObject(parameterIndex++, pair.getValue(), info.sqlType);
+      LOG.debug("Setting {} to {}", pair.getKey(), pair.getValue());
+    }
+    return prepareStatement;
+  }
+
+  public PreparedStatement createCountStatement(Connection connection,
+      Class<? extends AbstractIndexEntity> indexEntityClass) throws Exception {
+    String tableName =
+        entityMappingHolder.tableToEntityNameMap.inverse().get(indexEntityClass.getSimpleName());
+    String sql = "Select count(*) from " + tableName;
+    PreparedStatement prepareStatement = connection.prepareStatement(sql);
+    return prepareStatement;
+  }
+
   private void generateWhereClause(BiMap<String, String> entityNameToDBNameMapping,
       Predicate predicate, List<Pair<String, Object>> parametersList, StringBuilder whereClause) {
     String columnName = null;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionResource.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionResource.java
index 6c0637f..e861cec 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionResource.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionResource.java
@@ -85,7 +85,6 @@ import org.apache.pinot.thirdeye.formatter.DetectionAlertConfigFormatter;
 import org.apache.pinot.thirdeye.formatter.DetectionConfigFormatter;
 import org.apache.pinot.thirdeye.rootcause.impl.MetricEntity;
 import org.apache.pinot.thirdeye.util.AnomalyOffset;
-import org.h2.command.dml.Merge;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.Interval;
diff --git a/thirdeye/thirdeye-pinot/src/main/resources/schema/create-schema.sql b/thirdeye/thirdeye-pinot/src/main/resources/schema/create-schema.sql
index 00e6180..c93b922 100644
--- a/thirdeye/thirdeye-pinot/src/main/resources/schema/create-schema.sql
+++ b/thirdeye/thirdeye-pinot/src/main/resources/schema/create-schema.sql
@@ -380,6 +380,8 @@ create index session_principal_type_idx ON session_index(principal_type);
 create table if not exists detection_config_index (
     base_id bigint(20) not null,
     `name` VARCHAR(256) not null,
+    active BOOLEAN,
+    created_by VARCHAR(256),
     create_time timestamp,
     update_time timestamp default current_timestamp,
     version int(10)
@@ -387,6 +389,8 @@ create table if not exists detection_config_index (
 ALTER TABLE `detection_config_index` ADD UNIQUE `detection_config_unique_index`(`name`);
 create index detection_config_base_id_idx ON detection_config_index(base_id);
 create index detection_config_name_idx ON detection_config_index(`name`);
+create index detection_config_active_idx ON detection_config_index(active);
+create index detection_config_created_by_index ON detection_config_index(created_by);
 
 create table if not exists detection_alert_config_index (
     base_id bigint(20) not null,
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertSearcherTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertSearcherTest.java
new file mode 100644
index 0000000..3e25461
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/dashboard/resources/v2/alerts/AlertSearcherTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.dashboard.resources.v2.alerts;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.pinot.thirdeye.datalayer.bao.DAOTestBase;
+import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+
+public class AlertSearcherTest {
+  private DAOTestBase testDAOProvider;
+
+  @BeforeMethod
+  public void setUp() {
+    testDAOProvider = DAOTestBase.getInstance();
+    DetectionConfigManager detectionDAO = DAORegistry.getInstance().getDetectionConfigManager();
+    DetectionConfigDTO detectionConfig1 = new DetectionConfigDTO();
+    detectionConfig1.setName("test_detection1");
+    DetectionConfigDTO detectionConfig2 = new DetectionConfigDTO();
+    detectionConfig2.setName("test_detection2");
+    DetectionConfigDTO detectionConfig3 = new DetectionConfigDTO();
+    detectionConfig3.setName("test_detection3");
+    detectionConfig3.setCreatedBy("test@example.com");
+    DetectionConfigDTO detectionConfig4 = new DetectionConfigDTO();
+    detectionConfig4.setActive(true);
+    detectionConfig4.setName("test_detection4");
+
+    detectionDAO.save(detectionConfig1);
+    detectionDAO.save(detectionConfig2);
+    detectionDAO.save(detectionConfig3);
+    detectionDAO.save(detectionConfig4);
+  }
+
+  @Test
+  public void testSearch() {
+    AlertSearcher searcher = new AlertSearcher();
+    Map<String, Object> result = searcher.search(new AlertSearchFilter(), 10 ,0);
+    Assert.assertEquals(result.get("count"), 4L);
+    Assert.assertEquals(result.get("limit"), 10L);
+    Assert.assertEquals(result.get("offset"), 0L);
+  }
+
+  @Test
+  public void testSearchActive() {
+    AlertSearcher searcher = new AlertSearcher();
+    Map<String, Object> result = searcher.search(new AlertSearchFilter(Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), true), 10 ,0);
+    Assert.assertEquals(result.get("count"), 1L);
+    Assert.assertEquals(result.get("limit"), 10L);
+    Assert.assertEquals(result.get("offset"), 0L);
+  }
+}


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