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 2019/06/17 17:16:44 UTC

[incubator-pinot] branch master updated: UI integration for ratio cube algorithm (#4311)

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

jihao 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 ea42ca6  UI integration for ratio cube algorithm (#4311)
ea42ca6 is described below

commit ea42ca69d8c67cf6441d2ed90263eafa18fe9afb
Author: Yen-Jung Chang <cy...@utexas.edu>
AuthorDate: Mon Jun 17 10:16:37 2019 -0700

    UI integration for ratio cube algorithm (#4311)
    
    Integrate the backend code of additive cube and ratio cube algorithm. Now the result of ratio cube algorithm can be shown on ThirdEye's existing UI without any frontend changes.
---
 .../dashboard/resources/SummaryResource.java       | 241 +++++++++++++++------
 .../dashboard/resources/SummaryResourceTest.java   |  57 +++++
 2 files changed, 234 insertions(+), 64 deletions(-)

diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/SummaryResource.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/SummaryResource.java
index c11a305..cc20543 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/SummaryResource.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/SummaryResource.java
@@ -21,6 +21,7 @@ package org.apache.pinot.thirdeye.dashboard.resources;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
@@ -28,6 +29,8 @@ import java.net.URLDecoder;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 import javax.ws.rs.DefaultValue;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
@@ -46,6 +49,9 @@ import org.apache.pinot.thirdeye.cube.entry.MultiDimensionalSummaryCLITool;
 import org.apache.pinot.thirdeye.cube.ratio.RatioDBClient;
 import org.apache.pinot.thirdeye.cube.summary.SummaryResponse;
 import org.apache.pinot.thirdeye.dashboard.Utils;
+import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
+import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
 import org.apache.pinot.thirdeye.datasource.ThirdEyeCacheRegistry;
 import org.apache.pinot.thirdeye.util.ThirdEyeUtils;
 import org.joda.time.DateTimeZone;
@@ -67,6 +73,13 @@ public class SummaryResource {
   private static final String JAVASCRIPT_NULL_STRING = "undefined";
   private static final String HTML_STRING_ENCODING = "UTF-8";
 
+  private static final String NUMERATOR_GROUP_NAME = "numerator";
+  private static final String DENOMINATOR_GROUP_NAME = "denominator";
+  // Only match string like "id123/id456" but not "id123*100/id456"
+  // The 1st metric id will be put into numerator group and the 2nd metric id will be in denominator group
+  private static final String SIMPLE_RATIO_METRIC_EXPRESSION_ID_PARSER =
+      "^id(?<" + NUMERATOR_GROUP_NAME + ">\\d*)\\/id(?<" + DENOMINATOR_GROUP_NAME + ">\\d*)$";
+
   @GET
   @Path(value = "/summary/autoDimensionOrder")
   @Produces(MediaType.APPLICATION_JSON)
@@ -113,15 +126,23 @@ public class SummaryResource {
           OBJECT_MAPPER.readValue(hierarchiesPayload, new TypeReference<List<List<String>>>() {
           });
 
-      CostFunction costFunction = new BalancedCostFunction();
-      DateTimeZone dateTimeZone = DateTimeZone.forID(timeZone);
-      AdditiveDBClient cubeDbClient = new AdditiveDBClient(CACHE_REGISTRY_INSTANCE.getQueryCache());
-      MultiDimensionalSummary mdSummary = new MultiDimensionalSummary(cubeDbClient, costFunction, dateTimeZone);
+      MetricConfigManager metricConfigDAO = DAORegistry.getInstance().getMetricConfigDAO();
+      MetricConfigDTO metricConfigDTO = metricConfigDAO.findByMetricAndDataset(metric, dataset);
 
-      response = mdSummary
-          .buildSummary(dataset, metric, currentStartInclusive, currentEndExclusive, baselineStartInclusive,
-              baselineEndExclusive, dimensions, filterSetMap, summarySize, depth, hierarchies, doOneSideError);
+      DateTimeZone dateTimeZone = DateTimeZone.forID(timeZone);
 
+      // Non simple ratio metrics
+      if (!isSimpleRatioMetric(metricConfigDTO)) {
+        response =
+            runAdditiveCubeAlgorithm(dateTimeZone, dataset, metric, currentStartInclusive,
+                currentEndExclusive, baselineStartInclusive, baselineEndExclusive, dimensions, filterSetMap,
+                summarySize, depth, hierarchies, doOneSideError);
+      } else {  // Simple ratio metric such as "A/B". On the contrary, "A*100/B" is not a simple ratio metric.
+        response =
+            runRatioCubeAlgorithm(dateTimeZone, dataset, metricConfigDTO, currentStartInclusive,
+                currentEndExclusive, baselineStartInclusive, baselineEndExclusive, dimensions, filterSetMap,
+                summarySize, depth, hierarchies, doOneSideError);
+      }
     } catch (Exception e) {
       LOG.error("Exception while generating difference summary", e);
       response = SummaryResponse.buildNotAvailableResponse(dataset, metric);
@@ -166,15 +187,23 @@ public class SummaryResource {
         filterSets = ThirdEyeUtils.convertToMultiMap(filterJsonPayload);
       }
 
-      CostFunction costFunction = new BalancedCostFunction();
+      MetricConfigManager metricConfigDAO = DAORegistry.getInstance().getMetricConfigDAO();
+      MetricConfigDTO metricConfigDTO = metricConfigDAO.findByMetricAndDataset(metric, dataset);
+
       DateTimeZone dateTimeZone = DateTimeZone.forID(timeZone);
-      AdditiveDBClient cubeDbClient = new AdditiveDBClient(CACHE_REGISTRY_INSTANCE.getQueryCache());
-      MultiDimensionalSummary mdSummary = new MultiDimensionalSummary(cubeDbClient, costFunction, dateTimeZone);
 
-      response = mdSummary
-          .buildSummary(dataset, metric, currentStartInclusive, currentEndExclusive, baselineStartInclusive,
-              baselineEndExclusive, dimensions, filterSets, summarySize, 0, Collections.<List<String>>emptyList(),
-              doOneSideError);
+      // Non simple ratio metrics
+      if (!isSimpleRatioMetric(metricConfigDTO)) {
+        response =
+            runAdditiveCubeAlgorithm(dateTimeZone, dataset, metric, currentStartInclusive,
+                currentEndExclusive, baselineStartInclusive, baselineEndExclusive, dimensions, filterSets,
+                summarySize, 0, Collections.emptyList(), doOneSideError);
+      } else {  // Simple ratio metric such as "A/B". On the contrary, "A*100/B" is not a simple ratio metric.
+        response =
+            runRatioCubeAlgorithm(dateTimeZone, dataset, metricConfigDTO, currentStartInclusive,
+                currentEndExclusive, baselineStartInclusive, baselineEndExclusive, dimensions, filterSets,
+                summarySize, 0, Collections.emptyList(), doOneSideError);
+      }
     } catch (Exception e) {
       LOG.error("Exception while generating difference summary", e);
       response = SummaryResponse.buildNotAvailableResponse(dataset, metric);
@@ -182,68 +211,152 @@ public class SummaryResource {
     return OBJECT_MAPPER.writeValueAsString(response);
   }
 
-  @GET
-  @Path(value = "/summary/autoRatioDimensionOrder")
-  @Produces(MediaType.APPLICATION_JSON)
-  public String buildRatioSummary(@QueryParam("dataset") String dataset,
-      @QueryParam("numeratorMetric") String numeratorMetric,
-      @QueryParam("denominatorMetric") String denominatorMetric,
-      @QueryParam("currentStart") long currentStartInclusive,
-      @QueryParam("currentEnd") long currentEndExclusive,
-      @QueryParam("baselineStart") long baselineStartInclusive,
-      @QueryParam("baselineEnd") long baselineEndExclusive,
-      @QueryParam("dimensions") String groupByDimensions,
-      @QueryParam("filters") String filterJsonPayload,
-      @QueryParam("summarySize") int summarySize,
-      @QueryParam("depth") @DefaultValue(DEFAULT_DEPTH) int depth,
-      @QueryParam("hierarchies") @DefaultValue(DEFAULT_HIERARCHIES) String hierarchiesPayload,
-      @QueryParam("oneSideError") @DefaultValue(DEFAULT_ONE_SIDE_ERROR) boolean doOneSideError,
-      @QueryParam("excludedDimensions") @DefaultValue(DEFAULT_EXCLUDED_DIMENSIONS) String excludedDimensions,
-      @QueryParam("timeZone") @DefaultValue(DEFAULT_TIMEZONE_ID) String timeZone) throws Exception {
-    if (summarySize < 1) summarySize = 1;
+  /**
+   * Executes cube algorithm for the given additive metric.
+   *
+   * @param dateTimeZone time zone of the data.
+   * @param dataset dataset name.
+   * @param metric metric name.
+   * @param currentStartInclusive timestamp of current start.
+   * @param currentEndExclusive timestamp of current end.
+   * @param baselineStartInclusive timestamp of baseline start.
+   * @param baselineEndExclusive timestamp of baseline end.
+   * @param dimensions ordered dimensions to be drilled down by the algorithm.
+   * @param dataFilters the filter to be applied on the data. Thus, the algorithm will only analyze a subset of data.
+   * @param summarySize the size of the summary result.
+   * @param depth the depth of the dimensions to be analyzed.
+   * @param hierarchies the hierarchy among the dimensions.
+   * @param doOneSideError flag to toggle if we only want one side results.
+   *
+   * @return the summary result of cube algorithm.
+   */
+  private SummaryResponse runAdditiveCubeAlgorithm(DateTimeZone dateTimeZone, String dataset, String metric,
+      long currentStartInclusive, long currentEndExclusive, long baselineStartInclusive, long baselineEndExclusive,
+      Dimensions dimensions, Multimap<String, String> dataFilters, int summarySize, int depth,
+      List<List<String>> hierarchies, boolean doOneSideError) throws Exception {
 
-    SummaryResponse response = null;
+    CostFunction costFunction = new BalancedCostFunction();
+    AdditiveDBClient cubeDbClient = new AdditiveDBClient(CACHE_REGISTRY_INSTANCE.getQueryCache());
+    MultiDimensionalSummary mdSummary = new MultiDimensionalSummary(cubeDbClient, costFunction, dateTimeZone);
 
-    try {
-      Dimensions dimensions;
-      if (StringUtils.isBlank(groupByDimensions) || JAVASCRIPT_NULL_STRING.equals(groupByDimensions)) {
-        dimensions =
-            MultiDimensionalSummaryCLITool.sanitizeDimensions(new Dimensions(Utils.getSchemaDimensionNames(dataset)));
-      } else {
-        dimensions = new Dimensions(Arrays.asList(groupByDimensions.trim().split(",")));
-      }
+    return mdSummary.buildSummary(dataset, metric, currentStartInclusive, currentEndExclusive, baselineStartInclusive,
+        baselineEndExclusive, dimensions, dataFilters, summarySize, depth, hierarchies, doOneSideError);
+  }
 
-      if (!Strings.isNullOrEmpty(excludedDimensions)) {
-        List<String> dimensionsToBeRemoved = Arrays.asList(excludedDimensions.trim().split(","));
-        dimensions = MultiDimensionalSummaryCLITool.removeDimensions(dimensions, dimensionsToBeRemoved);
-      }
+  /**
+   * Executes cube algorithm for the given ratio metric.
+   *
+   * @param dateTimeZone time zone of the data.
+   * @param dataset dataset name.
+   * @param metricConfigDTO the metric config of the ratio metric.
+   * @param currentStartInclusive timestamp of current start.
+   * @param currentEndExclusive timestamp of current end.
+   * @param baselineStartInclusive timestamp of baseline start.
+   * @param baselineEndExclusive timestamp of baseline end.
+   * @param dimensions ordered dimensions to be drilled down by the algorithm.
+   * @param dataFilters the filter to be applied on the data. Thus, the algorithm will only analyze a subset of data.
+   * @param summarySize the size of the summary result.
+   * @param depth the depth of the dimensions to be analyzed.
+   * @param hierarchies the hierarchy among the dimensions.
+   * @param doOneSideError flag to toggle if we only want one side results.
+   *
+   * @return the summary result of cube algorithm.
+   */
+  private SummaryResponse runRatioCubeAlgorithm(DateTimeZone dateTimeZone, String dataset,
+      MetricConfigDTO metricConfigDTO, long currentStartInclusive, long currentEndExclusive,
+      long baselineStartInclusive, long baselineEndExclusive, Dimensions dimensions,
+      Multimap<String, String> dataFilters, int summarySize, int depth, List<List<String>> hierarchies,
+      boolean doOneSideError) throws Exception {
+    Preconditions.checkNotNull(metricConfigDTO);
 
-      Multimap<String, String> filterSetMap;
-      if (StringUtils.isBlank(filterJsonPayload) || JAVASCRIPT_NULL_STRING.equals(filterJsonPayload)) {
-        filterSetMap = ArrayListMultimap.create();
-      } else {
-        filterJsonPayload = URLDecoder.decode(filterJsonPayload, HTML_STRING_ENCODING);
-        filterSetMap = ThirdEyeUtils.convertToMultiMap(filterJsonPayload);
-      }
+    MetricConfigManager metricConfigDAO = DAORegistry.getInstance().getMetricConfigDAO();
 
-      List<List<String>> hierarchies =
-          OBJECT_MAPPER.readValue(hierarchiesPayload, new TypeReference<List<List<String>>>() {
-          });
+    // Construct regular expression parser
+    String derivedMetricExpression = metricConfigDTO.getDerivedMetricExpression();
+    MatchedRatioMetricsResult matchedRatioMetricsResult = parseNumeratorDenominatorId(derivedMetricExpression);
 
+    if (matchedRatioMetricsResult.hasFound) {
+      // Extract numerator and denominator id
+      long numeratorId = matchedRatioMetricsResult.numeratorId;
+      long denominatorId = matchedRatioMetricsResult.denominatorId;
+      // Get numerator and denominator's metric name
+      String numeratorMetric = metricConfigDAO.findById(numeratorId).getName();
+      String denominatorMetric = metricConfigDAO.findById(denominatorId).getName();
+      // Generate cube result
       CostFunction costFunction = new RatioCostFunction();
-      DateTimeZone dateTimeZone = DateTimeZone.forID(timeZone);
       RatioDBClient dbClient = new RatioDBClient(CACHE_REGISTRY_INSTANCE.getQueryCache());
       MultiDimensionalRatioSummary mdSummary = new MultiDimensionalRatioSummary(dbClient, costFunction, dateTimeZone);
 
-      response = mdSummary
-          .buildRatioSummary(dataset, numeratorMetric, denominatorMetric, currentStartInclusive, currentEndExclusive, baselineStartInclusive,
-              baselineEndExclusive, dimensions, filterSetMap, summarySize, depth, hierarchies, doOneSideError);
+      return mdSummary.buildRatioSummary(dataset, numeratorMetric, denominatorMetric, currentStartInclusive,
+          currentEndExclusive, baselineStartInclusive, baselineEndExclusive, dimensions, dataFilters, summarySize,
+          depth, hierarchies, doOneSideError);
+    } else { // parser should find ids because of the guard of the if-condition.
+      LOG.error("Unable to parser numerator and denominator metric for metric" + metricConfigDTO.getName());
+      return SummaryResponse.buildNotAvailableResponse(dataset, metricConfigDTO.getName());
+    }
+  }
 
-    } catch (Exception e) {
-      LOG.error("Exception while generating difference summary", e);
-      response = SummaryResponse.buildNotAvailableResponse(dataset, numeratorMetric + "/" + denominatorMetric);
+  /**
+   * Returns if the given metric is a simple ratio metric such as "A/B" where A and B is a metric.
+   *
+   * @param metricConfigDTO the config of a metric.
+   *
+   * @return true if the given metric is a simple ratio metric.
+   */
+  static boolean isSimpleRatioMetric(MetricConfigDTO metricConfigDTO) {
+    if (metricConfigDTO != null) {
+      String metricExpression = metricConfigDTO.getDerivedMetricExpression();
+      if (!Strings.isNullOrEmpty(metricExpression)) {
+        Pattern pattern = Pattern.compile(SIMPLE_RATIO_METRIC_EXPRESSION_ID_PARSER);
+        Matcher matcher = pattern.matcher(metricExpression);
+        return matcher.matches();
+      }
     }
+    return false;
+  }
 
-    return OBJECT_MAPPER.writeValueAsString(response);
+  /**
+   * Parse numerator and denominator id from a given metric expression string.
+   *
+   * @param derivedMetricExpression the given metric expression.
+   *
+   * @return the parsed result, which is stored in MatchedRatioMetricsResult.
+   */
+  static MatchedRatioMetricsResult parseNumeratorDenominatorId(String derivedMetricExpression) {
+    if (Strings.isNullOrEmpty(derivedMetricExpression)) {
+      return new MatchedRatioMetricsResult(false, -1, -1);
+    }
+
+    Pattern pattern = Pattern.compile(SIMPLE_RATIO_METRIC_EXPRESSION_ID_PARSER);
+    Matcher matcher = pattern.matcher(derivedMetricExpression);
+    if (matcher.find()) {
+      // Extract numerator and denominator id
+      long numeratorId = Long.valueOf(matcher.group(NUMERATOR_GROUP_NAME));
+      long denominatorId = Long.valueOf(matcher.group(DENOMINATOR_GROUP_NAME));
+      return new MatchedRatioMetricsResult(true, numeratorId, denominatorId);
+    } else {
+      return new MatchedRatioMetricsResult(false, -1, -1);
+    }
+  }
+
+  /**
+   * The class to store the parsed numerator and denominator id.
+   */
+  static class MatchedRatioMetricsResult {
+    boolean hasFound; // false is parsing is failed.
+    long numeratorId; // numerator id if parsing is succeeded.
+    long denominatorId; // denominator id if parsing is succeeded.
+
+    /**
+     * Construct the object that stores the parsed numerator and denominator id.
+     * @param hasFound false is parsing is failed.
+     * @param numeratorId numerator id if parsing is succeeded.
+     * @param denominatorId denominator id if parsing is succeeded.
+     */
+    MatchedRatioMetricsResult(boolean hasFound, long numeratorId, long denominatorId) {
+      this.hasFound = hasFound;
+      this.numeratorId = numeratorId;
+      this.denominatorId = denominatorId;
+    }
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/dashboard/resources/SummaryResourceTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/dashboard/resources/SummaryResourceTest.java
new file mode 100644
index 0000000..8b5dfe7
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/dashboard/resources/SummaryResourceTest.java
@@ -0,0 +1,57 @@
+package org.apache.pinot.thirdeye.dashboard.resources;
+
+import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class SummaryResourceTest {
+  @Test
+  public void testIsSimpleRatioMetric() {
+    // False: null metric config
+    Assert.assertFalse(SummaryResource.isSimpleRatioMetric(null));
+    // False: empty metric config
+    Assert.assertFalse(SummaryResource.isSimpleRatioMetric(new MetricConfigDTO()));
+
+    MetricConfigDTO ratioMetricConfigDTO = new MetricConfigDTO();
+    // True: ratio metric
+    ratioMetricConfigDTO.setDerivedMetricExpression("id123/id456");
+    Assert.assertTrue(SummaryResource.isSimpleRatioMetric(ratioMetricConfigDTO));
+
+    // False: derived metric with empty expression
+    ratioMetricConfigDTO.setDerivedMetricExpression("");
+    Assert.assertFalse(SummaryResource.isSimpleRatioMetric(ratioMetricConfigDTO));
+
+    // False: derived metric with null expression
+    ratioMetricConfigDTO.setDerivedMetricExpression(null);
+    Assert.assertFalse(SummaryResource.isSimpleRatioMetric(ratioMetricConfigDTO));
+
+    // False: percentage metric
+    ratioMetricConfigDTO.setDerivedMetricExpression("id123*100/id456");
+    Assert.assertFalse(SummaryResource.isSimpleRatioMetric(ratioMetricConfigDTO));
+
+    // False: complex derived metric
+    ratioMetricConfigDTO.setDerivedMetricExpression("id123/id456/id789");
+    Assert.assertFalse(SummaryResource.isSimpleRatioMetric(ratioMetricConfigDTO));
+  }
+
+  @Test
+  public void testParseNumeratorDenominatorId() {
+    // Fail: empty metric expression
+    SummaryResource.MatchedRatioMetricsResult matchedRatioMetricsResult =
+        SummaryResource.parseNumeratorDenominatorId("");
+    Assert.assertFalse(matchedRatioMetricsResult.hasFound);
+
+    // Fail: null metric expression
+    matchedRatioMetricsResult = SummaryResource.parseNumeratorDenominatorId(null);
+    Assert.assertFalse(matchedRatioMetricsResult.hasFound);
+
+    // The other failure cases should be covered by testIsSimpleRatioMetric() since they use the same regex expression.
+
+    // Success: simple ratio metric expression
+    matchedRatioMetricsResult = SummaryResource.parseNumeratorDenominatorId("id123/id456");
+    Assert.assertTrue(matchedRatioMetricsResult.hasFound);
+    Assert.assertEquals(matchedRatioMetricsResult.numeratorId, 123);
+    Assert.assertEquals(matchedRatioMetricsResult.denominatorId, 456);
+  }
+}


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