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

[incubator-pinot] branch master updated: [TE] Refactor cube algorithm code for adding ratio cube in the future. (#4213)

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 7abbcd2  [TE] Refactor cube algorithm code for adding ratio cube in the future. (#4213)
7abbcd2 is described below

commit 7abbcd2a7845b6425700aca02a35c828f9bd3e50
Author: Yen-Jung Chang <cy...@utexas.edu>
AuthorDate: Thu May 23 16:02:07 2019 -0700

    [TE] Refactor cube algorithm code for adding ratio cube in the future. (#4213)
    
    Major changes:
    
    Move cube algorithm related code to package cube.
    Extract multiple classes to interface and implementation classes:
    ThirdEyeSummaryClient --> CubeDBClient (interface) and AdditiveCubeDBClient (Impl)
    Row --> Row (Interface) and AdditiveRow (Impl)
    HierarchyNode --> CubeNode (Interface) and AdditiveCubeNode (Impl)
    Update related classes for future support of ratio metrics.
    Update unit tests for the new changes.
    Tests:
    
    Passed the updated unit tests.
    The integration test is done through ThirdEye API.
---
 .../thirdeye/client/diffsummary/HierarchyNode.java | 296 ---------------------
 .../client/diffsummary/ThirdEyeSummaryClient.java  | 278 -------------------
 .../thirdeye/cube/additive/AdditiveCubeNode.java   | 183 +++++++++++++
 .../thirdeye/cube/additive/AdditiveDBClient.java   |  98 +++++++
 .../Row.java => cube/additive/AdditiveRow.java}    |  94 ++++---
 .../additive}/MultiDimensionalSummary.java         |  37 ++-
 .../additive}/MultiDimensionalSummaryCLITool.java  |  14 +-
 .../cost}/BalancedCostFunction.java                |  42 +--
 .../cost}/ChangeRatioCostFunction.java             |  28 +-
 .../ContributionToOverallChangeCostFunction.java   |  11 +-
 .../pinot/thirdeye/cube/cost/CostFunction.java     |  41 +++
 .../diffsummary => cube/data/cube}/Cube.java       | 182 ++++++-------
 .../pinot/thirdeye/cube/data/cube/CubeUtils.java   |  85 ++++++
 .../data/cube}/DimNameValueCostEntry.java          |  38 ++-
 .../cube/data/dbclient/BaseCubePinotClient.java    | 261 ++++++++++++++++++
 .../thirdeye/cube/data/dbclient/CubeClient.java    |  70 +++++
 .../data/dbclient/CubePinotClient.java}            |  49 ++--
 .../thirdeye/cube/data/dbclient/CubeSpec.java      | 123 +++++++++
 .../data/dbclient/CubeTag.java}                    |  10 +-
 .../dbclient/ThirdEyeRequestMetricExpressions.java |  63 +++++
 .../data/dbrow/BaseRow.java}                       |  36 ++-
 .../data/dbrow}/DimensionValues.java               |   2 +-
 .../data/dbrow}/Dimensions.java                    |   5 +-
 .../apache/pinot/thirdeye/cube/data/dbrow/Row.java |  78 ++++++
 .../thirdeye/cube/data/node/BaseCubeNode.java      | 144 ++++++++++
 .../pinot/thirdeye/cube/data/node/CubeNode.java    | 211 +++++++++++++++
 .../thirdeye/cube/data/node/CubeNodeUtils.java     |  72 +++++
 .../summary}/BaseResponseRow.java                  |   4 +-
 .../diffsummary => cube/summary}/DPArray.java      |   8 +-
 .../diffsummary => cube/summary}/Summary.java      | 164 ++++++------
 .../summary}/SummaryGainerLoserResponseRow.java    |   2 +-
 .../summary}/SummaryResponse.java                  |  46 ++--
 .../summary}/SummaryResponseRow.java               |   2 +-
 .../summary}/SummaryResponseTree.java              | 106 ++++----
 .../dashboard/resources/SummaryResource.java       |  31 +--
 .../impl/MetricComponentAnalysisPipeline.java      |   3 +-
 .../client/diffsummary/HierarchyNodeTest.java      | 182 -------------
 .../MultiDimensionalSummaryCLIToolTest.java        |   5 +-
 .../cost}/BalancedCostFunctionTest.java            |   5 +-
 .../diffsummary => cube/data/cube}/CubeTest.java   |  75 +++---
 .../data/cube}/DimNameValueCostEntryTest.java      |   9 +-
 .../data/dbrow}/DimensionValuesTest.java           |   3 +-
 .../data/dbrow}/DimensionsTest.java                |   3 +-
 .../thirdeye/cube/data/node/CubeNodeTest.java      | 187 +++++++++++++
 44 files changed, 2201 insertions(+), 1185 deletions(-)

diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/HierarchyNode.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/HierarchyNode.java
deleted file mode 100644
index 48284d4..0000000
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/HierarchyNode.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/*
- * 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.client.diffsummary;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import java.util.Objects;
-import org.apache.commons.lang.ObjectUtils;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-
-public class HierarchyNode {
-  private int level;
-  private int index;
-  private double baselineValue;
-  private double currentValue;
-  private double cost;
-  private Row data;
-  private HierarchyNode parent;
-  private List<HierarchyNode> children = new ArrayList<>();
-
-  /**
-   * Constructs a root HierarchyNode whose level and index is 0 and parent pointer is null.
-   *
-   * @param data the data of this root node.
-   */
-  public HierarchyNode(Row data) {
-    this.data = Preconditions.checkNotNull(data);
-    this.baselineValue = data.getBaselineValue();
-    this.currentValue = data.getCurrentValue();
-  }
-
-  /**
-   * Constructs a HierarchyNode which is specified information.
-   *
-   * @param level the level of this node.
-   * @param index the index of this node that is located in its parent's children list.
-   * @param data the data of this node.
-   * @param parent the parent of this node.
-   */
-  public HierarchyNode(int level, int index, Row data, HierarchyNode parent) {
-    this(data);
-    this.level = level;
-    this.index = index;
-    this.parent = Preconditions.checkNotNull(parent);
-    Dimensions parentDimension = new Dimensions(parent.data.getDimensions().namesToDepth(parent.getLevel()));
-    Dimensions childDimension = new Dimensions(data.getDimensions().namesToDepth(level));
-    Preconditions.checkState(parentDimension.isParentOf(childDimension),
-        "Current node is not a child node of the given parent node. Current and parent dimensions: ",
-        data.getDimensions(), parent.data.getDimensions());
-    parent.children.add(this);
-  }
-
-  public int getLevel() {
-    return level;
-  }
-
-  public void resetValues() {
-    this.baselineValue = this.data.getBaselineValue();
-    this.currentValue = this.data.getCurrentValue();
-  }
-
-  public void removeNodeValues(HierarchyNode node) {
-    this.baselineValue -= node.baselineValue;
-    this.currentValue -= node.currentValue;
-  }
-
-  public void addNodeValues(HierarchyNode node) {
-    this.baselineValue += node.baselineValue;
-    this.currentValue += node.currentValue;
-  }
-
-  public double getBaselineValue() {
-    return baselineValue;
-  }
-
-  public double getCurrentValue() {
-    return currentValue;
-  }
-
-  @JsonIgnore
-  public double getOriginalBaselineValue() {
-    return data.getBaselineValue();
-  }
-
-  @JsonIgnore
-  public double getOriginalCurrentValue() {
-    return data.getCurrentValue();
-  }
-
-  public double getCost() {
-    return cost;
-  }
-
-  public void setCost(double cost) {
-    this.cost = cost;
-  }
-
-  @JsonIgnore
-  public Dimensions getDimensions() {
-    return data.getDimensions();
-  }
-
-  @JsonIgnore
-  public DimensionValues getDimensionValues() {
-    return data.getDimensionValues();
-  }
-
-  public HierarchyNode getParent() {
-    return parent;
-  }
-
-  public int childrenSize() {
-    return children.size();
-  }
-
-  public List<HierarchyNode> getChildren() {
-    return Collections.unmodifiableList(children);
-  }
-
-  /**
-   * Returns the ratio that is calculated by the aggregate current and aggregate baseline values of all children node.
-   *
-   * @return aggregated current value of all children / aggregated baseline value of all children;
-   */
-  public double aggregatedRatio() {
-    return data.ratio();
-  }
-
-  /**
-   * Returns the ratio that is calculated by currentValue and baselineValue.
-   * @return currentValue / baselineValue;
-   */
-  public double currentRatio() {
-    return currentValue / baselineValue;
-  }
-
-  /**
-   * Return the ratio of the node. If the ratio is not a finite number, then it returns the aggregatedRatio.
-   * If the aggregatedRatio is not a finite number, then it bootstraps to the parents until it finds a finite
-   * ratio. If no finite ratio available, then it returns 1.
-   */
-  public double targetRatio() {
-    double ratio = currentRatio();
-    if (!Double.isInfinite(ratio) && Double.compare(ratio, 0d) != 0) {
-      return ratio;
-    } else {
-      ratio = aggregatedRatio();
-      if (!Double.isInfinite(ratio) && Double.compare(ratio, 0d) != 0) {
-        return ratio;
-      } else {
-        if (parent != null) {
-          return parent.targetRatio();
-        } else {
-          return 1.;
-        }
-      }
-    }
-  }
-
-  /**
-   * Returns the current ratio of this node is increased or decreased, i.e., returns true if ratio of the node >= 1.0.
-   * If the current ratio is NAN, then the ratio of the aggregated values is used.
-   *
-   * Precondition: the aggregated baseline and current values cannot both be zero.
-   */
-  public boolean side() {
-    double ratio = currentRatio();
-    if (!Double.isNaN(ratio)) {
-      return Double.compare(1., currentRatio()) <= 0;
-    } else {
-      return Double.compare(1., aggregatedRatio()) <= 0;
-    }
-  }
-
-  /**
-   * Check if the hierarchical tree of the given two root nodes are the same.
-   *
-   * @param node1 the root node of the first hierarchical tree.
-   * @param node2 the root node of the second hierarchical tree.
-   *
-   * @return true if both hierarchical tree are the same.
-   */
-  public static boolean equalHierarchy(HierarchyNode node1, HierarchyNode node2) {
-    return equalHierarchy(node1, null, node2, null);
-  }
-
-  private static boolean equalHierarchy(HierarchyNode node1, HierarchyNode node1Parent, HierarchyNode node2,
-      HierarchyNode node2Parent) {
-    boolean sameData = ObjectUtils.equals(node1, node2);
-    if (sameData) {
-      // Check parent reference
-      if (node1Parent != null && node1.getParent() != node1Parent) {
-        return false;
-      }
-      if (node2Parent != null && node2.getParent() != node2Parent) {
-        return false;
-      }
-
-      // Check children reference
-      List<HierarchyNode> children1 = node1.children;
-      List<HierarchyNode> children2 = node2.children;
-      if (children1.size() != children2.size()) {
-        return false;
-      }
-      int size = children1.size();
-      for (int i = 0; i < size; i++) {
-        HierarchyNode child1 = children1.get(i);
-        HierarchyNode child2 = children2.get(i);
-        boolean sameChild = equalHierarchy(child1, node1, child2, node2);
-        if (!sameChild) {
-          return false;
-        }
-      }
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  /**
-   * Returns if the data of current node equals to the data of other node. The parent and children nodes are not
-   * compared due to the cyclic references between parent and children nodes.
-   *
-   * @param o the other node.
-   *
-   * @return true if the data of current node equals to the data of other node.
-   */
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    HierarchyNode that = (HierarchyNode) o;
-    return getLevel() == that.getLevel() && index == that.index
-        && Double.compare(that.getBaselineValue(), getBaselineValue()) == 0
-        && Double.compare(that.getCurrentValue(), getCurrentValue()) == 0
-        && Double.compare(that.getCost(), getCost()) == 0 && Objects.equals(data, that.data);
-  }
-
-  /**
-   * Returns the hash code that is generated base on the data of this node.
-   *
-   * @return the hash code that is generated base on the data of this node.
-   */
-  @Override
-  public int hashCode() {
-    return Objects
-        .hash(getLevel(), index, getBaselineValue(), getCurrentValue(), getCost(), data);
-  }
-
-  private String toStringAsParent() {
-    return MoreObjects.toStringHelper(this).add("level", level).add("index", index).add("baselineValue", baselineValue)
-        .add("currentValue", currentValue).add("cost", cost).add("data", data).toString();
-  }
-
-  private static String toStringAsParent(HierarchyNode node) {
-    if (node == null) {
-      return "null";
-    } else {
-      return node.toStringAsParent();
-    }
-  }
-
-  @Override
-  public String toString() {
-    return MoreObjects.toStringHelper(this).add("level", level).add("index", index).add("baselineValue", baselineValue)
-        .add("currentValue", currentValue).add("cost", cost).add("data", data).add("parent", toStringAsParent(parent))
-        .toString();
-  }
-}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/ThirdEyeSummaryClient.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/ThirdEyeSummaryClient.java
deleted file mode 100644
index c995181..0000000
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/ThirdEyeSummaryClient.java
+++ /dev/null
@@ -1,278 +0,0 @@
-/*
- * 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.client.diffsummary;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multimap;
-import org.apache.pinot.thirdeye.datasource.MetricExpression;
-import org.apache.pinot.thirdeye.datasource.MetricFunction;
-import org.apache.pinot.thirdeye.datasource.ThirdEyeRequest;
-import org.apache.pinot.thirdeye.datasource.ThirdEyeRequest.ThirdEyeRequestBuilder;
-import org.apache.pinot.thirdeye.datasource.ThirdEyeResponse;
-import org.apache.pinot.thirdeye.datasource.cache.QueryCache;
-import org.apache.pinot.thirdeye.util.ThirdEyeUtils;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import org.jfree.util.Log;
-import org.joda.time.DateTime;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class generates query requests to the backend database and retrieve the data for summary algorithm.
- *
- * The generated requests are organized the following tree structure:
- *   Root level by GroupBy dimensions.
- *   Mid  level by "baseline" or "current"; The "baseline" request is ordered before the "current" request.
- *   Leaf level by metric functions; This level is handled by the request itself, i.e., a request can gather multiple
- *   metric functions at the same time.
- * The generated requests are store in a List. Because of the tree structure, the requests belong to the same
- * timeline (baseline or current) are located together. Then, the requests belong to the same GroupBy dimension are
- * located together.
- */
-public class ThirdEyeSummaryClient implements OLAPDataBaseClient {
-  private static final Logger LOG = LoggerFactory.getLogger(ThirdEyeSummaryClient.class);
-
-  private final static DateTime NULL_DATETIME = new DateTime();
-  private final static int TIME_OUT_VALUE = 1200;
-  private final static TimeUnit TIME_OUT_UNIT = TimeUnit.SECONDS;
-
-  private QueryCache queryCache;
-  private String collection;
-  private DateTime baselineStartInclusive = NULL_DATETIME;
-  private DateTime baselineEndExclusive = NULL_DATETIME;
-  private DateTime currentStartInclusive = NULL_DATETIME;
-  private DateTime currentEndExclusive = NULL_DATETIME;
-
-  private MetricExpression metricExpression;
-  private List<MetricFunction> metricFunctions;
-  private MetricExpressionsContext context;
-
-  public ThirdEyeSummaryClient(QueryCache queryCache) {
-    this.queryCache = queryCache;
-  }
-
-
-  @Override
-  public void setCollection(String collection) {
-    this.collection = collection;
-  }
-
-  @Override
-  public void setMetricExpression(MetricExpression metricExpression) {
-    this.metricExpression = metricExpression;
-    metricFunctions = metricExpression.computeMetricFunctions();
-    if (metricFunctions.size() > 1) {
-      context = new MetricExpressionsContext();
-    } else {
-      context = null;
-    }
-  }
-
-  @Override
-  public void setBaselineStartInclusive(DateTime dateTime) {
-    baselineStartInclusive = dateTime;
-  }
-
-  @Override
-  public void setBaselineEndExclusive(DateTime dateTime) {
-    baselineEndExclusive = dateTime;
-  }
-
-  @Override
-  public void setCurrentStartInclusive(DateTime dateTime) {
-    currentStartInclusive = dateTime;
-  }
-
-  @Override
-  public void setCurrentEndExclusive(DateTime dateTime) {
-    currentEndExclusive = dateTime;
-  }
-
-  @Override
-  public Row getTopAggregatedValues(Multimap<String, String> filterSets) throws Exception {
-    List<String> groupBy = Collections.emptyList();
-      List<ThirdEyeRequest> timeOnTimeBulkRequests = constructTimeOnTimeBulkRequests(groupBy, filterSets);
-      Row row = constructAggregatedValues(new Dimensions(), timeOnTimeBulkRequests).get(0).get(0);
-      return row;
-  }
-
-  @Override
-  public List<List<Row>> getAggregatedValuesOfDimension(Dimensions dimensions, Multimap<String, String> filterSets)
-      throws Exception {
-      List<ThirdEyeRequest> timeOnTimeBulkRequests = new ArrayList<>();
-      for (int level = 0; level < dimensions.size(); ++level) {
-        List<String> groupBy = Lists.newArrayList(dimensions.get(level));
-        timeOnTimeBulkRequests.addAll(constructTimeOnTimeBulkRequests(groupBy, filterSets));
-      }
-      List<List<Row>> rows = constructAggregatedValues(dimensions, timeOnTimeBulkRequests);
-      return rows;
-  }
-
-  @Override
-  public List<List<Row>> getAggregatedValuesOfLevels(Dimensions dimensions, Multimap<String, String> filterSets)
-      throws Exception {
-      List<ThirdEyeRequest> timeOnTimeBulkRequests = new ArrayList<>();
-      for (int level = 0; level < dimensions.size() + 1; ++level) {
-        List<String> groupBy = Lists.newArrayList(dimensions.namesToDepth(level));
-        timeOnTimeBulkRequests.addAll(constructTimeOnTimeBulkRequests(groupBy, filterSets));
-      }
-      List<List<Row>> rows = constructAggregatedValues(dimensions, timeOnTimeBulkRequests);
-      return rows;
-  }
-
-  /**
-   * Returns the baseline and current requests for the given GroupBy dimensions.
-   *
-   * @param groupBy the dimensions to do GroupBy queries
-   * @param filterSets the filter to apply on the DB queries (e.g., country=US, etc.)
-   * @return Baseline and Current requests.
-   */
-  private List<ThirdEyeRequest> constructTimeOnTimeBulkRequests(List<String> groupBy,
-      Multimap<String, String> filterSets) {
-    List<ThirdEyeRequest> requests = new ArrayList<>();;
-
-    // baseline requests
-    ThirdEyeRequestBuilder builder = ThirdEyeRequest.newBuilder();
-    builder.setMetricFunctions(metricFunctions);
-    builder.setGroupBy(groupBy);
-    builder.setStartTimeInclusive(baselineStartInclusive);
-    builder.setEndTimeExclusive(baselineEndExclusive);
-    builder.setDataSource(ThirdEyeUtils.getDataSourceFromMetricFunctions(metricFunctions));
-    builder.setFilterSet(filterSets);
-    ThirdEyeRequest baselineRequest = builder.build("baseline");
-    requests.add(baselineRequest);
-
-    // current requests
-    builder = ThirdEyeRequest.newBuilder();
-    builder.setMetricFunctions(metricFunctions);
-    builder.setGroupBy(groupBy);
-    builder.setStartTimeInclusive(currentStartInclusive);
-    builder.setEndTimeExclusive(currentEndExclusive);
-    builder.setDataSource(ThirdEyeUtils.getDataSourceFromMetricFunctions(metricFunctions));
-    builder.setFilterSet(filterSets);
-    ThirdEyeRequest currentRequest = builder.build("current");
-    requests.add(currentRequest);
-
-    return requests;
-  }
-
-  /**
-   * @throws Exception Throws exceptions when no useful data is retrieved, i.e., time out, failed to connect
-   * to the backend database, no non-zero data returned from the database, etc.
-   */
-  private List<List<Row>> constructAggregatedValues(Dimensions dimensions, List<ThirdEyeRequest> bulkRequests)
-      throws Exception {
-    Map<ThirdEyeRequest, Future<ThirdEyeResponse>> queryResponses = queryCache.getQueryResultsAsync(bulkRequests);
-
-    List<List<Row>> res = new ArrayList<>();
-    for (int i = 0; i < bulkRequests.size(); ) {
-      ThirdEyeRequest baselineRequest = bulkRequests.get(i++);
-      ThirdEyeRequest currentRequest = bulkRequests.get(i++);
-      ThirdEyeResponse baselineResponses = queryResponses.get(baselineRequest).get(TIME_OUT_VALUE, TIME_OUT_UNIT);
-      ThirdEyeResponse currentResponses = queryResponses.get(currentRequest).get(TIME_OUT_VALUE, TIME_OUT_UNIT);
-      if (baselineResponses.getNumRows() == 0) {
-        LOG.warn("Get 0 rows from the request(s): {}", baselineRequest);
-      }
-      if (currentResponses.getNumRows() == 0) {
-        LOG.warn("Get 0 rows from the request(s): {}", currentRequest);
-      }
-
-      Map<List<String>, Row> rowTable = new HashMap<>();
-      buildMetricFunctionOrExpressionsRows(dimensions, baselineResponses, rowTable, true);
-      buildMetricFunctionOrExpressionsRows(dimensions, currentResponses, rowTable, false);
-      if (rowTable.size() == 0) {
-        LOG.warn("Failed to retrieve non-zero results with these requests: " + baselineRequest + ", " + currentRequest);
-      }
-      List<Row> rows = new ArrayList<>(rowTable.values());
-      res.add(rows);
-    }
-
-    return res;
-  }
-
-  /**
-   * Returns a list of rows. The value of each row is evaluated and no further processing is needed.
-   * @param dimensions dimensions of the response
-   * @param response the response from backend database
-   * @param rowTable the storage for rows
-   * @param isBaseline true if the response is for baseline values
-   */
-  private void buildMetricFunctionOrExpressionsRows(Dimensions dimensions, ThirdEyeResponse response,
-      Map<List<String>, Row> rowTable, boolean isBaseline) {
-    for (int rowIdx = 0; rowIdx < response.getNumRows(); ++rowIdx) {
-      double value = 0d;
-      // If the metric expression is a single metric function, then we get the value immediately
-      if (metricFunctions.size() <= 1) {
-        value = response.getRow(rowIdx).getMetrics().get(0);
-      } else { // Otherwise, we need to evaluate the expression
-        context.reset();
-        for (int metricFuncIdx = 0; metricFuncIdx < metricFunctions.size(); ++metricFuncIdx) {
-          double contextValue = response.getRow(rowIdx).getMetrics().get(metricFuncIdx);
-          context.set(metricFunctions.get(metricFuncIdx).getMetricName(), contextValue);
-        }
-        try {
-          value = MetricExpression.evaluateExpression(metricExpression, context.getContext());
-        } catch (Exception e) {
-          Log.warn(e);
-        }
-      }
-      if (Double.compare(0d, value) < 0 && !Double.isInfinite(value)) {
-        List<String> dimensionValues = response.getRow(rowIdx).getDimensions();
-        Row row = rowTable.get(dimensionValues);
-        if (row == null) {
-          row = new Row(dimensions, new DimensionValues(dimensionValues));
-          rowTable.put(dimensionValues, row);
-        }
-        if (isBaseline) {
-          row.setBaselineValue(value);
-        } else {
-          row.setCurrentValue(value);
-        }
-      }
-    }
-  }
-
-  private class MetricExpressionsContext {
-    private Map<String, Double> context;
-    public MetricExpressionsContext () {
-      context = new HashMap<>();
-      for (MetricFunction metricFunction : metricFunctions) {
-        context.put(metricFunction.getMetricName(), 0d);
-      }
-    }
-    public void set(String metricName, double value) {
-      context.put(metricName, value);
-    }
-    public Map<String, Double> getContext() {
-      return context;
-    }
-    public void reset() {
-      for (Map.Entry<String, Double> entry : context.entrySet()) {
-        entry.setValue(0d);
-      }
-    }
-  }
-}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/AdditiveCubeNode.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/AdditiveCubeNode.java
new file mode 100644
index 0000000..1b9f85f
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/AdditiveCubeNode.java
@@ -0,0 +1,183 @@
+/*
+ * 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.cube.additive;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.google.common.base.MoreObjects;
+import java.util.Objects;
+import org.apache.pinot.thirdeye.cube.data.node.BaseCubeNode;
+
+
+/**
+ * A CubeNode for additive metrics such as page view count.
+ */
+public class AdditiveCubeNode extends BaseCubeNode<AdditiveCubeNode, AdditiveRow> {
+  private double baselineValue;
+  private double currentValue;
+
+  /**
+   * Constructs a root CubeNode whose level and index is 0 and parent pointer is null.
+   *
+   * @param data the data of this root node.
+   */
+  public AdditiveCubeNode(AdditiveRow data) {
+    super(data);
+    this.baselineValue = data.getBaselineValue();
+    this.currentValue = data.getCurrentValue();
+  }
+
+  /**
+   * Constructs a CubeNode which is specified information.
+   *
+   * @param level the level of this node.
+   * @param index the index of this node that is located in its parent's children list.
+   * @param data the data of this node.
+   * @param parent the parent of this node.
+   */
+  public AdditiveCubeNode(int level, int index, AdditiveRow data, AdditiveCubeNode parent) {
+    super(level, index, data, parent);
+    this.baselineValue = data.getBaselineValue();
+    this.currentValue = data.getCurrentValue();
+  }
+
+  @Override
+  public void resetValues() {
+    this.baselineValue = this.data.getBaselineValue();
+    this.currentValue = this.data.getCurrentValue();
+  }
+
+  @Override
+  public void removeNodeValues(AdditiveCubeNode node) {
+    this.baselineValue -= node.baselineValue;
+    this.currentValue -= node.currentValue;
+  }
+
+  @Override
+  public void addNodeValues(AdditiveCubeNode node) {
+    this.baselineValue += node.baselineValue;
+    this.currentValue += node.currentValue;
+  }
+
+  @Override
+  public double getBaselineSize() {
+    return baselineValue;
+  }
+
+  @Override
+  public double getCurrentSize() {
+    return currentValue;
+  }
+
+  @Override
+  public double getOriginalBaselineSize() {
+    return data.getBaselineValue();
+  }
+
+  @Override
+  public double getOriginalCurrentSize() {
+    return data.getCurrentValue();
+  }
+
+  @Override
+  public double getBaselineValue() {
+    return baselineValue;
+  }
+
+  @Override
+  public double getCurrentValue() {
+    return currentValue;
+  }
+
+  @Override
+  @JsonIgnore
+  public double getOriginalBaselineValue() {
+    return data.getBaselineValue();
+  }
+
+  @Override
+  @JsonIgnore
+  public double getOriginalCurrentValue() {
+    return data.getCurrentValue();
+  }
+
+  @Override
+  public double originalChangeRatio() {
+    return data.currentValue / data.baselineValue;
+  }
+
+  @Override
+  public double changeRatio() {
+    return currentValue / baselineValue;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    AdditiveCubeNode that = (AdditiveCubeNode) o;
+    return getLevel() == that.getLevel() && index == that.index
+        && Double.compare(that.getBaselineValue(), getBaselineValue()) == 0
+        && Double.compare(that.getCurrentValue(), getCurrentValue()) == 0
+        && Double.compare(that.getCost(), getCost()) == 0 && Objects.equals(data, that.data);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects
+        .hash(getLevel(), index, getBaselineValue(), getCurrentValue(), getCost(), data);
+  }
+
+  /**
+   * The toString method for parent node. We don't invoke parent's toString() to prevent multiple calls of toString to
+   * their parents.
+   *
+   * @return a simple string representation of a parent cube node, which does not toString its parent node recursively.
+   */
+  private String toStringAsParent() {
+    return MoreObjects.toStringHelper(this).add("level", level).add("index", index).add("baselineValue", baselineValue)
+        .add("currentValue", currentValue).add("cost", cost).add("data", data).toString();
+  }
+
+  /**
+   * ToString that handles if the given cube node is null, i.e., a root cube node.
+   *
+   * @param node the node to be converted to string.
+   *
+   * @return a string representation of this node.
+   */
+  private static String toStringAsParent(AdditiveCubeNode node) {
+    if (node == null) {
+      return "null";
+    } else {
+      return node.toStringAsParent();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this).add("level", level).add("index", index).add("baselineValue", baselineValue)
+        .add("currentValue", currentValue).add("cost", cost).add("data", data).add("parent", toStringAsParent(parent))
+        .toString();
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/AdditiveDBClient.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/AdditiveDBClient.java
new file mode 100644
index 0000000..2047801
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/AdditiveDBClient.java
@@ -0,0 +1,98 @@
+/*
+ * 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.cube.additive;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.pinot.thirdeye.cube.data.dbrow.DimensionValues;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.data.dbclient.CubeTag;
+import org.apache.pinot.thirdeye.cube.data.dbclient.BaseCubePinotClient;
+import org.apache.pinot.thirdeye.cube.data.dbclient.CubeSpec;
+import org.apache.pinot.thirdeye.datasource.cache.QueryCache;
+
+/**
+ * This class generates query requests to the backend database and retrieve the data for summary algorithm.
+ *
+ * The generated requests are organized the following tree structure:
+ *   Root level by GroupBy dimensions.
+ *   Mid  level by "baseline" or "current"; The "baseline" request is ordered before the "current" request.
+ *   Leaf level by metric functions; This level is handled by the request itself, i.e., a request can gather multiple
+ *   metric functions at the same time.
+ * The generated requests are store in a List. Because of the tree structure, the requests belong to the same
+ * timeline (baseline or current) are located together. Then, the requests belong to the same GroupBy dimension are
+ * located together.
+ */
+public class AdditiveDBClient extends BaseCubePinotClient<AdditiveRow> {
+  private String metric;
+
+  /**
+   * Constructs a DB client to an additive metric.
+   *
+   * @param queryCache the query cache to Pinot DB.
+   */
+  public AdditiveDBClient(QueryCache queryCache) {
+    super(queryCache);
+  }
+
+  /**
+   * Sets the additive metric name.
+   *
+   * @param metric the additive metric name.
+   */
+  public void setMetric(String metric) {
+    this.metric = Preconditions.checkNotNull(metric);
+  }
+
+  @Override
+  protected List<CubeSpec> getCubeSpecs() {
+    List<CubeSpec> cubeSpecs = new ArrayList<>();
+
+    cubeSpecs.add(new CubeSpec(CubeTag.Baseline, metric, baselineStartInclusive, baselineEndExclusive));
+    cubeSpecs.add(new CubeSpec(CubeTag.Current, metric, currentStartInclusive, currentEndExclusive));
+
+    return cubeSpecs;
+  }
+
+  @Override
+  protected void fillValueToRowTable(Map<List<String>, AdditiveRow> rowTable, Dimensions dimensions,
+      List<String> dimensionValues, double value, CubeTag tag) {
+
+    if (Double.compare(0d, value) < 0 && !Double.isInfinite(value)) {
+      AdditiveRow row = rowTable.get(dimensionValues);
+      if (row == null) {
+        row = new AdditiveRow(dimensions, new DimensionValues(dimensionValues));
+        rowTable.put(dimensionValues, row);
+      }
+      switch (tag) {
+        case Baseline:
+          row.setBaselineValue(value);
+          break;
+        case Current:
+          row.setCurrentValue(value);
+          break;
+        default:
+          throw new IllegalArgumentException("Unsupported CubeTag: " + tag.name());
+      }
+    }
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/Row.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/AdditiveRow.java
similarity index 50%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/Row.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/AdditiveRow.java
index c5300c8..0172d61 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/Row.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/AdditiveRow.java
@@ -17,66 +17,93 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
+package org.apache.pinot.thirdeye.cube.additive;
 
+import com.google.common.base.MoreObjects;
 import com.google.common.base.Preconditions;
 import java.util.Objects;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.pinot.thirdeye.cube.data.dbrow.DimensionValues;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.data.node.CubeNode;
+import org.apache.pinot.thirdeye.cube.data.dbrow.BaseRow;
 
 
-class Row {
-  private Dimensions dimensions;
-  private DimensionValues dimensionValues;
-  private double baselineValue;
-  private double currentValue;
-
-  public Row(Dimensions dimensions, DimensionValues dimensionValues) {
+/**
+ * Stores the additive metric that is returned from DB.
+ */
+public class AdditiveRow extends BaseRow {
+  protected double baselineValue;
+  protected double currentValue;
+
+  /**
+   * Constructs an additive row.
+   *
+   * @param dimensions the dimension names of this row.
+   * @param dimensionValues the dimension values of this row.
+   */
+  public AdditiveRow(Dimensions dimensions, DimensionValues dimensionValues) {
     this.dimensions = Preconditions.checkNotNull(dimensions);
     this.dimensionValues = Preconditions.checkNotNull(dimensionValues);
   }
 
-  public Row(Dimensions dimensions, DimensionValues dimensionValues, double baselineValue, double currentValue) {
+  /**
+   * Constructs an additive row.
+   *
+   * @param dimensions the dimension names of this row.
+   * @param dimensionValues the dimension values of this row.
+   * @param baselineValue the baseline value of this additive metric.
+   * @param currentValue the current value of this additive metric.
+   */
+  public AdditiveRow(Dimensions dimensions, DimensionValues dimensionValues, double baselineValue, double currentValue) {
     this.dimensions = Preconditions.checkNotNull(dimensions);
     this.dimensionValues = Preconditions.checkNotNull(dimensionValues);
     this.baselineValue = baselineValue;
     this.currentValue = currentValue;
   }
 
-  public Dimensions getDimensions() {
-    return dimensions;
-  }
-
-  public void setDimensions(Dimensions dimensions) {
-    this.dimensions = Preconditions.checkNotNull(dimensions);
-  }
-
-  public DimensionValues getDimensionValues() {
-    return dimensionValues;
-  }
-
-  public void setDimensionValues(DimensionValues dimensionValues) {
-    this.dimensionValues = Preconditions.checkNotNull(dimensionValues);
-  }
-
+  /**
+   * Returns the baseline value of this additive row.
+   *
+   * @return the baseline value of this additive row.
+   */
   public double getBaselineValue() {
     return baselineValue;
   }
 
+  /**
+   * Sets the baseline value of this additive row.
+   *
+   * @param baselineValue the baseline value of this additive row.
+   */
   public void setBaselineValue(double baselineValue) {
     this.baselineValue = baselineValue;
   }
 
+  /**
+   * Returns the current value of this additive row.
+   *
+   * @return the current value of this additive row.
+   */
   public double getCurrentValue() {
     return currentValue;
   }
 
+  /**
+   * Sets the current value of this additive row.
+   * @param currentValue the current value of this additive row.
+   */
   public void setCurrentValue(double currentValue) {
     this.currentValue = currentValue;
   }
 
-  public double ratio() {
-    return currentValue / baselineValue;
+  @Override
+  public CubeNode toNode() {
+    return new AdditiveCubeNode(this);
+  }
+
+  @Override
+  public CubeNode toNode(int level, int index, CubeNode parent) {
+    return new AdditiveCubeNode(level, index, this, (AdditiveCubeNode) parent);
   }
 
   @Override
@@ -87,7 +114,7 @@ class Row {
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    Row row = (Row) o;
+    AdditiveRow row = (AdditiveRow) o;
     return Double.compare(row.getBaselineValue(), getBaselineValue()) == 0
         && Double.compare(row.getCurrentValue(), getCurrentValue()) == 0 && Objects
         .equals(getDimensions(), row.getDimensions()) && Objects.equals(getDimensionValues(), row.getDimensionValues());
@@ -100,6 +127,11 @@ class Row {
 
   @Override
   public String toString() {
-    return ToStringBuilder.reflectionToString(this, ToStringStyle.SIMPLE_STYLE);
+    return MoreObjects.toStringHelper(this)
+        .add("baselineValue", baselineValue)
+        .add("currentValue", currentValue)
+        .add("dimensions", dimensions)
+        .add("dimensionValues", dimensionValues)
+        .toString();
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/MultiDimensionalSummary.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/MultiDimensionalSummary.java
similarity index 76%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/MultiDimensionalSummary.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/MultiDimensionalSummary.java
index 19e9c01..3591c83 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/MultiDimensionalSummary.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/MultiDimensionalSummary.java
@@ -17,18 +17,18 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
+package org.apache.pinot.thirdeye.cube.additive;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.Multimap;
-import org.apache.pinot.thirdeye.client.diffsummary.costfunctions.CostFunction;
-import org.apache.pinot.thirdeye.constant.MetricAggFunction;
-import org.apache.pinot.thirdeye.dashboard.Utils;
-import org.apache.pinot.thirdeye.dashboard.views.diffsummary.Summary;
-import org.apache.pinot.thirdeye.dashboard.views.diffsummary.SummaryResponse;
-import org.apache.pinot.thirdeye.datasource.MetricExpression;
 import java.util.List;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.cost.CostFunction;
+import org.apache.pinot.thirdeye.cube.data.cube.Cube;
+import org.apache.pinot.thirdeye.cube.data.dbclient.CubePinotClient;
+import org.apache.pinot.thirdeye.cube.summary.Summary;
+import org.apache.pinot.thirdeye.cube.summary.SummaryResponse;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 
@@ -37,16 +37,16 @@ import org.joda.time.DateTimeZone;
  * A portal class that is used to trigger the multi-dimensional summary algorithm and to get the summary response.
  */
 public class MultiDimensionalSummary {
-  private OLAPDataBaseClient olapClient;
+  private CubePinotClient dbClient;
   private CostFunction costFunction;
   private DateTimeZone dateTimeZone;
 
-  public MultiDimensionalSummary(OLAPDataBaseClient olapClient, CostFunction costFunction,
+  public MultiDimensionalSummary(CubePinotClient olapClient, CostFunction costFunction,
       DateTimeZone dateTimeZone) {
     Preconditions.checkNotNull(olapClient);
     Preconditions.checkNotNull(dateTimeZone);
     Preconditions.checkNotNull(costFunction);
-    this.olapClient = olapClient;
+    this.dbClient = olapClient;
     this.costFunction = costFunction;
     this.dateTimeZone = dateTimeZone;
   }
@@ -88,22 +88,21 @@ public class MultiDimensionalSummary {
     Preconditions.checkNotNull(hierarchies);
     Preconditions.checkArgument(depth >= 0);
 
-    olapClient.setCollection(dataset);
-    List<MetricExpression> metricExpressions = Utils.convertToMetricExpressions(metric, MetricAggFunction.SUM, dataset);
-    olapClient.setMetricExpression(metricExpressions.get(0));
-    olapClient.setCurrentStartInclusive(new DateTime(currentStartInclusive, dateTimeZone));
-    olapClient.setCurrentEndExclusive(new DateTime(currentEndExclusive, dateTimeZone));
-    olapClient.setBaselineStartInclusive(new DateTime(baselineStartInclusive, dateTimeZone));
-    olapClient.setBaselineEndExclusive(new DateTime(baselineEndExclusive, dateTimeZone));
+    dbClient.setDataset(dataset);
+    ((AdditiveDBClient) dbClient).setMetric(metric);
+    dbClient.setCurrentStartInclusive(new DateTime(currentStartInclusive, dateTimeZone));
+    dbClient.setCurrentEndExclusive(new DateTime(currentEndExclusive, dateTimeZone));
+    dbClient.setBaselineStartInclusive(new DateTime(baselineStartInclusive, dateTimeZone));
+    dbClient.setBaselineEndExclusive(new DateTime(baselineEndExclusive, dateTimeZone));
 
     Cube cube = new Cube(costFunction);
     SummaryResponse response;
     if (depth > 0) { // depth != 0 means manual dimension order
-      cube.buildWithAutoDimensionOrder(olapClient, dimensions, dataFilters, depth, hierarchies);
+      cube.buildWithAutoDimensionOrder(dbClient, dimensions, dataFilters, depth, hierarchies);
       Summary summary = new Summary(cube, costFunction);
       response = summary.computeSummary(summarySize, doOneSideError, depth);
     } else { // manual dimension order
-      cube.buildWithManualDimensionOrder(olapClient, dimensions, dataFilters);
+      cube.buildWithManualDimensionOrder(dbClient, dimensions, dataFilters);
       Summary summary = new Summary(cube, costFunction);
       response = summary.computeSummary(summarySize, doOneSideError);
     }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/MultiDimensionalSummaryCLITool.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/MultiDimensionalSummaryCLITool.java
similarity index 95%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/MultiDimensionalSummaryCLITool.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/MultiDimensionalSummaryCLITool.java
index d1a986d..963e265 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/MultiDimensionalSummaryCLITool.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/additive/MultiDimensionalSummaryCLITool.java
@@ -17,17 +17,19 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
+package org.apache.pinot.thirdeye.cube.additive;
 
 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.Multimap;
-import org.apache.pinot.thirdeye.client.diffsummary.costfunctions.BalancedCostFunction;
-import org.apache.pinot.thirdeye.client.diffsummary.costfunctions.CostFunction;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.cost.BalancedCostFunction;
+import org.apache.pinot.thirdeye.cube.cost.CostFunction;
+import org.apache.pinot.thirdeye.cube.data.dbclient.CubePinotClient;
 import org.apache.pinot.thirdeye.dashboard.Utils;
-import org.apache.pinot.thirdeye.dashboard.views.diffsummary.SummaryResponse;
+import org.apache.pinot.thirdeye.cube.summary.SummaryResponse;
 import org.apache.pinot.thirdeye.datasource.ThirdEyeCacheRegistry;
 import org.apache.pinot.thirdeye.util.ThirdEyeUtils;
 import java.io.IOException;
@@ -223,7 +225,7 @@ public class MultiDimensionalSummaryCLITool {
 
       // Initialize ThirdEye's environment
       ThirdEyeUtils.initLightWeightThirdEyeEnvironment(argList.get(0));
-      OLAPDataBaseClient olapClient = new ThirdEyeSummaryClient(CACHE_REGISTRY_INSTANCE.getQueryCache());
+      CubePinotClient<AdditiveRow> cubeDbClient = new AdditiveDBClient(CACHE_REGISTRY_INSTANCE.getQueryCache());
 
       // Convert JSON string to Objects
       Dimensions dimensions;
@@ -242,7 +244,7 @@ public class MultiDimensionalSummaryCLITool {
       });
 
       // Trigger summary algorithm
-      MultiDimensionalSummary mdSummary = new MultiDimensionalSummary(olapClient, costFunction, timeZone);
+      MultiDimensionalSummary mdSummary = new MultiDimensionalSummary(cubeDbClient, costFunction, timeZone);
       SummaryResponse summaryResponse = mdSummary
           .buildSummary(dataset, metricName, currentStart, currentEnd, baselineStart, baselineEnd, dimensions,
               dataFilter, summarySize, depth, hierarchies, oneSideError);
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/BalancedCostFunction.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/cost/BalancedCostFunction.java
similarity index 75%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/BalancedCostFunction.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/cost/BalancedCostFunction.java
index c02e22e..e4c1cb1 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/BalancedCostFunction.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/cost/BalancedCostFunction.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary.costfunctions;
+package org.apache.pinot.thirdeye.cube.cost;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
@@ -25,13 +25,14 @@ import java.util.Map;
 
 
 /**
- * This cost function consider change difference, change ratio, and node size (contribution percentage of a node).
- * More for details: {@link #computeCost(double, double, double, double, double)}.
+ * This cost function consider change difference, change changeRatio, and node size (contribution percentage of a node).
+ * More for details: {@link CostFunction#computeCost(double, double, double, double, double, double, double, double, double)}.
  */
 public class BalancedCostFunction implements CostFunction {
   public static final String CHANGE_CONTRIBUTION_THRESHOLD_PARAM = "threshold";
   // The threshold to the contribution to overall changes in percentage
   private double changeContributionThreshold = 3d;
+  private double epsilon = 0.00001;
 
   public BalancedCostFunction() {
   }
@@ -53,38 +54,49 @@ public class BalancedCostFunction implements CostFunction {
   }
 
   /**
-   * Returns the cost that consider change difference, change ratio, and node size (contribution percentage of a node).
+   * Returns the cost that consider change difference, change changeRatio, and node size (contribution percentage of a node).
    *
    * In brief, this function uses this formula to compute the cost:
-   *   change difference * log(contribution percentage * change ratio)
+   *   change difference * log(contribution percentage * change changeRatio)
    *
    * In addition, if a node's contribution to overall changes is smaller than the threshold, which is defined when
    * constructing this class, then the cost is always zero.
    *
+   * @param parentChangeRatio the changeRatio between baseline and current value of parent node.
    * @param baselineValue the baseline value of the current node.
    * @param currentValue the current value of the current node.
-   * @param parentRatio the ratio between baseline and current value of parent node.
-   * @param globalBaselineValue the baseline value of the top node.
-   * @param globalCurrentValue the current value of the top node.
+   * @param baselineSize
+   * @param currentSize
+   * @param topBaselineValue the baseline value of the top node.
+   * @param topCurrentValue the current value of the top node.
    *
-   * @return the cost that consider change difference, change ratio, and node size.
+   * @param topBaselineSize
+   * @param topCurrentSize
+   * @return the cost that consider change difference, change changeRatio, and node size.
    */
   @Override
-  public double computeCost(double baselineValue, double currentValue, double parentRatio, double globalBaselineValue,
-      double globalCurrentValue) {
+  public double computeCost(double parentChangeRatio, double baselineValue, double currentValue, double baselineSize,
+      double currentSize, double topBaselineValue, double topCurrentValue, double topBaselineSize,
+      double topCurrentSize) {
 
     // Typically, users don't care nodes with small contribution to overall changes
     double contributionToOverallChange =
-        Math.abs((currentValue - baselineValue) / (globalCurrentValue - globalBaselineValue));
+        Math.abs((currentValue - baselineValue) / (topCurrentValue - topBaselineValue));
     if (Double.compare(contributionToOverallChange, changeContributionThreshold / 100d) < 0) {
       return 0d;
     }
     // Contribution is the size of the node
-    double contribution = (((baselineValue + currentValue) / (globalBaselineValue + globalCurrentValue)));
+    double contribution = (baselineSize + currentSize) / (topBaselineSize + topCurrentSize);
+    if (Math.abs(0d - contribution) < epsilon) {
+      contribution = 0d;
+    }
+    if (Double.compare(contribution, 1) > 0 || Double.compare(contribution, 0) < 0) {
+      System.out.println("Here");
+    }
     Preconditions.checkState(Double.compare(contribution, 0) >= 0, "Contribution {} is smaller than 0.", contribution);
     Preconditions.checkState(Double.compare(contribution, 1) <= 0, "Contribution {} is larger than 1", contribution);
-    // The cost function considers change difference, change ratio, and node size (i.e., contribution)
-    return fillEmptyValuesAndGetError(baselineValue, currentValue, parentRatio, contribution);
+    // The cost function considers change difference, change changeRatio, and node size (i.e., contribution)
+    return fillEmptyValuesAndGetError(baselineValue, currentValue, parentChangeRatio, contribution);
   }
 
   private static double error(double baselineValue, double currentValue, double parentRatio, double contribution) {
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/ChangeRatioCostFunction.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/cost/ChangeRatioCostFunction.java
similarity index 81%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/ChangeRatioCostFunction.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/cost/ChangeRatioCostFunction.java
index 2801b03..a6fdb1f 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/ChangeRatioCostFunction.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/cost/ChangeRatioCostFunction.java
@@ -17,18 +17,14 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary.costfunctions;
+package org.apache.pinot.thirdeye.cube.cost;
 
 public class ChangeRatioCostFunction implements CostFunction {
   @Override
-  public double computeCost(double baselineValue, double currentValue, double parentRatio, double globalBaselineValue,
-      double globalCurrentValue) {
-    return fillEmptyValuesAndGetError(baselineValue, currentValue, parentRatio);
-  }
-
-  private static double error(double baselineValue, double currentValue, double parentRatio) {
-    double expectedBaselineValue = parentRatio * baselineValue;
-    return (currentValue - expectedBaselineValue) * Math.log(currentValue / expectedBaselineValue);
+  public double computeCost(double parentChangeRatio, double baselineValue, double currentValue, double baselineSize,
+      double currentSize, double topBaselineValue, double topCurrentValue, double topBaselineSize,
+      double topCurrentSize) {
+    return fillEmptyValuesAndGetError(baselineValue, currentValue, parentChangeRatio);
   }
 
   /**
@@ -55,4 +51,18 @@ public class ChangeRatioCostFunction implements CostFunction {
       return 0.;
     }
   }
+
+  /**
+   * The error proposed by iDiff paper.
+   *
+   * @param baselineValue the baseline value of a node.
+   * @param currentValue the current value of a node.
+   * @param parentRatio the change ratio of a parent node.
+   *
+   * @return the error cost.
+   */
+  private static double error(double baselineValue, double currentValue, double parentRatio) {
+    double expectedBaselineValue = parentRatio * baselineValue;
+    return (currentValue - expectedBaselineValue) * Math.log(currentValue / expectedBaselineValue);
+  }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/ContributionToOverallChangeCostFunction.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/cost/ContributionToOverallChangeCostFunction.java
similarity index 83%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/ContributionToOverallChangeCostFunction.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/cost/ContributionToOverallChangeCostFunction.java
index 292afb1..4d80491 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/ContributionToOverallChangeCostFunction.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/cost/ContributionToOverallChangeCostFunction.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary.costfunctions;
+package org.apache.pinot.thirdeye.cube.cost;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
@@ -47,11 +47,12 @@ public class ContributionToOverallChangeCostFunction implements CostFunction {
   }
 
   @Override
-  public double computeCost(double baselineValue, double currentValue, double parentRatio, double globalBaselineValue,
-      double globalCurrentValue) {
+  public double computeCost(double parentChangeRatio, double baselineValue, double currentValue, double baselineSize,
+      double currentSize, double topBaselineValue, double topCurrentValue, double topBaselineSize,
+      double topCurrentSize) {
 
-    double contributionToOverallChange = (currentValue - baselineValue) / (globalCurrentValue - globalBaselineValue);
-    double percentageContribution = (((baselineValue) / globalBaselineValue) * 100);
+    double contributionToOverallChange = (currentValue - baselineValue) / (topCurrentValue - topBaselineValue);
+    double percentageContribution = (((baselineValue) / topBaselineValue) * 100);
     if (Double.compare(percentageContribution, contributionPercentageThreshold) < 0) {
       return 0d;
     } else {
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/cost/CostFunction.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/cost/CostFunction.java
new file mode 100644
index 0000000..75e6392
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/cost/CostFunction.java
@@ -0,0 +1,41 @@
+/*
+ * 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.cube.cost;
+
+public interface CostFunction {
+  /**
+   * Calculates the error cost of a node when it is inserted back to it's parent.
+   *
+   * @param parentChangeRatio change ratio of the parent node.
+   * @param baselineValue the baseline value of the node.
+   * @param currentValue the current value of the node.
+   * @param baselineSize the baseline node size.
+   * @param currentSize the current node size.
+   * @param topBaselineValue the baseline value of the root node.
+   * @param topCurrentValue the current value of the root node.
+   * @param topBaselineSize the baseline size of root node.
+   * @param topCurrentSize the current size of root node.
+   *
+   * @return the error cost of the current node.
+   */
+  double computeCost(double parentChangeRatio, double baselineValue, double currentValue, double baselineSize,
+      double currentSize, double topBaselineValue, double topCurrentValue, double topBaselineSize,
+      double topCurrentSize);
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/Cube.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/cube/Cube.java
similarity index 74%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/Cube.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/cube/Cube.java
index bec6481..dad61b8 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/Cube.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/cube/Cube.java
@@ -17,16 +17,16 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
+package org.apache.pinot.thirdeye.cube.data.cube;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Multimap;
 import org.apache.pinot.thirdeye.anomaly.utils.ThirdeyeMetricsUtil;
-import org.apache.pinot.thirdeye.client.diffsummary.costfunctions.CostFunction;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.cost.CostFunction;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -38,6 +38,9 @@ import java.util.Set;
 
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.pinot.thirdeye.cube.data.dbclient.CubeClient;
+import org.apache.pinot.thirdeye.cube.data.node.CubeNode;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Row;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -54,9 +57,13 @@ public class Cube { // the cube (Ca|Cb)
   private static final int DEFAULT_DEPTH = 3;
   private static final int TOP_COST_ENTRIES_TO_LOG = 20;
 
+  // Node value
   private double baselineTotal;
   private double currentTotal;
-  private double topRatio;
+  // Node size
+  private double baselineTotalSize;
+  private double currentTotalSize;
+
   private List<DimNameValueCostEntry> costSet = Collections.emptyList();
   private List<DimensionCost> sortedDimensionCosts = Collections.emptyList();
 
@@ -69,7 +76,7 @@ public class Cube { // the cube (Ca|Cb)
 
   // The logical nodes of the hierarchy among the actual data
   @JsonIgnore
-  private List<List<HierarchyNode>> hierarchicalNodes = new ArrayList<>();
+  private List<List<CubeNode>> hierarchicalNodes = new ArrayList<>();
 
   public double getBaselineTotal() {
     return baselineTotal;
@@ -79,8 +86,12 @@ public class Cube { // the cube (Ca|Cb)
     return currentTotal;
   }
 
-  public double getTopRatio() {
-    return topRatio;
+  public double getBaselineTotalSize() {
+    return baselineTotalSize;
+  }
+
+  public double getCurrentTotalSize() {
+    return currentTotalSize;
   }
 
   public Dimensions getDimensions() {
@@ -94,7 +105,7 @@ public class Cube { // the cube (Ca|Cb)
   }
 
   @JsonIgnore
-  public HierarchyNode getRoot() {
+  public CubeNode getRoot() {
     if (hierarchicalNodes.size() != 0 && hierarchicalNodes.get(0).size() != 0) {
       return hierarchicalNodes.get(0).get(0);
     } else {
@@ -120,7 +131,7 @@ public class Cube { // the cube (Ca|Cb)
    * @param depth the number of the top dimensions to be considered in the subcube.
    * @param hierarchy the hierarchy among the given dimensions, whose order will be honors before dimensions' cost.
    */
-  public void buildWithAutoDimensionOrder(OLAPDataBaseClient olapClient,
+  public void buildWithAutoDimensionOrder(CubeClient olapClient,
       Dimensions dimensions, Multimap<String, String> dataFilter, int depth, List<List<String>> hierarchy)
       throws Exception {
     long tStart = System.nanoTime();
@@ -129,8 +140,9 @@ public class Cube { // the cube (Ca|Cb)
       Preconditions.checkNotNull(hierarchy, "hierarchy cannot be null.");
 
       initializeBasicInfo(olapClient, dataFilter);
-      Dimensions shrankDimensions = shrinkDimensionsByFilterSets(dimensions, dataFilter);
-      costSet = computeOneDimensionCost(olapClient, baselineTotal, currentTotal, shrankDimensions, dataFilter);
+      Dimensions shrankDimensions = CubeUtils.shrinkDimensionsByFilterSets(dimensions, dataFilter);
+      costSet = computeOneDimensionCost(olapClient, baselineTotal, currentTotal, baselineTotalSize, currentTotalSize,
+          shrankDimensions, dataFilter);
       sortedDimensionCosts = calculateSortedDimensionCost(costSet);
       this.dimensions = sortDimensions(sortedDimensionCosts, depth, hierarchy);
 
@@ -155,17 +167,15 @@ public class Cube { // the cube (Ca|Cb)
    * @param dimensions the dimensions, whose order has been given, of the subcube.
    * @param dataFilter the filter to be applied on the incoming data.
    */
-  public void buildWithManualDimensionOrder(OLAPDataBaseClient olapClient, Dimensions dimensions,
+  public void buildWithManualDimensionOrder(CubeClient olapClient, Dimensions dimensions,
       Multimap<String, String> dataFilter)
       throws Exception {
     long tStart = System.nanoTime();
     try {
       buildSubCube(olapClient, dimensions, dataFilter);
-
     } catch (Exception e) {
       ThirdeyeMetricsUtil.cubeExceptionCounter.inc();
       throw e;
-
     } finally {
       ThirdeyeMetricsUtil.cubeCallCounter.inc();
       ThirdeyeMetricsUtil.cubeDurationCounter.inc(System.nanoTime() - tStart);
@@ -173,51 +183,20 @@ public class Cube { // the cube (Ca|Cb)
   }
 
   /**
-   * Removes dimensions from the given list of dimensions, which has single values in the filter set. Only dimensions
-   * with one value is removed from the given dimensions because setting a filter one dimension names with one dimension
-   * value (e.g., "country=US") implies that the final data cube does not contain other dimension values. Thus, the
-   * summary algorithm could simply ignore that dimension (because the cube does not have any other values to compare
-   * with in that dimension).
-   *
-   * @param dimensions the list of dimensions to be modified.
-   * @param filterSets the filter to be applied on the data cube.
-   *
-   * @return the list of dimensions that should be used for retrieving the data for summary algorithm.
-   */
-  private static Dimensions shrinkDimensionsByFilterSets(Dimensions dimensions, Multimap<String, String> filterSets) {
-    Set<String> dimensionsToRemove = new HashSet<>();
-    for (Map.Entry<String, Collection<String>> filterSetEntry : filterSets.asMap().entrySet()) {
-      if (filterSetEntry.getValue().size() == 1) {
-        dimensionsToRemove.add(filterSetEntry.getKey());
-      }
-    }
-    return removeDimensions(dimensions, dimensionsToRemove);
-  }
-
-  private static Dimensions removeDimensions(Dimensions dimensions, Collection<String> dimensionsToRemove) {
-    List<String> dimensionsToRetain = new ArrayList<>();
-    for (String dimensionName : dimensions.names()) {
-      if(!dimensionsToRemove.contains(dimensionName)){
-        dimensionsToRetain.add(dimensionName);
-      }
-    }
-    return new Dimensions(dimensionsToRetain);
-  }
-
-  /**
    * Builds the subcube according to the given dimension order.
    *
    * @param olapClient the data client to retrieve data.
    * @param dimensions the given dimension order.
    * @param dataFilter the data filter to applied on the data cube.
    */
-  private void buildSubCube(OLAPDataBaseClient olapClient, Dimensions dimensions, Multimap<String, String> dataFilter)
+  private void buildSubCube(CubeClient olapClient, Dimensions dimensions, Multimap<String, String> dataFilter)
       throws Exception {
     Preconditions.checkArgument((dimensions != null && dimensions.size() != 0), "Dimensions cannot be empty.");
     if (this.dimensions == null) { // which means buildWithAutoDimensionOrder is not triggered
       initializeBasicInfo(olapClient, dataFilter);
       this.dimensions = dimensions;
-      costSet = computeOneDimensionCost(olapClient, baselineTotal, currentTotal, dimensions, dataFilter);
+      costSet = computeOneDimensionCost(olapClient, baselineTotal, currentTotal, baselineTotalSize, currentTotalSize,
+          dimensions, dataFilter);
     }
 
     int size = 0;
@@ -233,25 +212,30 @@ public class Cube { // the cube (Ca|Cb)
     List<List<Row>> rowOfLevels = olapClient.getAggregatedValuesOfLevels(dimensions, dataFilter);
     for (int i = 0; i <= dimensions.size(); ++i) {
       List<Row> rowAtLevelI = rowOfLevels.get(i);
-      Collections.sort(rowAtLevelI, new RowDimensionValuesComparator());
+      rowAtLevelI.sort(new RowDimensionValuesComparator());
       hierarchicalRows.add(rowAtLevelI);
       size += rowAtLevelI.size();
     }
     LOG.info("Size of the cube for generating summary: " + size);
 
-    hierarchicalNodes = hierarchyRowToHierarchyNode(hierarchicalRows, dimensions);
+    hierarchicalNodes = dataRowToCubeNode(hierarchicalRows, dimensions);
   }
 
   /**
-   * Calculate the change ratio of the top aggregated values.
+   * Calculate the change changeRatio of the top aggregated values.
    * @throws Exception An exception is thrown if OLAP database cannot be connected.
    */
-  private void initializeBasicInfo(OLAPDataBaseClient olapClient, Multimap<String, String> filterSets)
+  private void initializeBasicInfo(CubeClient olapClient, Multimap<String, String> filterSets)
       throws Exception {
+
     Row topAggValues = olapClient.getTopAggregatedValues(filterSets);
-    baselineTotal = topAggValues.getBaselineValue(); // aggregated baseline values
-    currentTotal = topAggValues.getCurrentValue(); // aggregated current values
-    topRatio = currentTotal / baselineTotal; // change ratio
+    CubeNode node = topAggValues.toNode();
+
+    baselineTotal =  node.getBaselineValue();
+    currentTotal = node.getCurrentValue();
+
+    baselineTotalSize = node.getBaselineSize();
+    currentTotalSize = node.getCurrentSize();
   }
 
   /**
@@ -267,42 +251,41 @@ public class Cube { // the cube (Ca|Cb)
   /**
    * Establishes the hierarchical relationship between the aggregated data (parent) and detailed data (children).
    *
-   * @param hierarchicalRows the actual data.
+   * @param dataRows the actual data.
    * @param dimensions the dimension names of the actual data.
    *
-   * @return HierarchyNode that contains the hierarchical relationship.
+   * @return CubeNode that contains the hierarchical relationship.
    */
-  public static List<List<HierarchyNode>> hierarchyRowToHierarchyNode(List<List<Row>> hierarchicalRows,
-      Dimensions dimensions) {
+  public static List<List<CubeNode>> dataRowToCubeNode(List<List<Row>> dataRows, Dimensions dimensions) {
 
-    List<List<HierarchyNode>> hierarchicalNodes = new ArrayList<>();
-    HashMap<String, HierarchyNode> curParent = new HashMap<>();
-    HashMap<String, HierarchyNode> nextParent = new HashMap<>();
+    List<List<CubeNode>> hierarchicalNodes = new ArrayList<>();
+    HashMap<String, CubeNode> curParent = new HashMap<>();
+    HashMap<String, CubeNode> nextParent = new HashMap<>();
 
     for (int level = 0; level <= dimensions.size(); ++level) {
-      hierarchicalNodes.add(new ArrayList<HierarchyNode>(hierarchicalRows.get(level).size()));
+      hierarchicalNodes.add(new ArrayList<>(dataRows.get(level).size()));
 
       if (level != 0) {
-        for (int index = 0; index < hierarchicalRows.get(level).size(); ++index) {
-          Row row = hierarchicalRows.get(level).get(index);
+        for (int index = 0; index < dataRows.get(level).size(); ++index) {
+          Row row = dataRows.get(level).get(index);
           StringBuilder parentDimValues = new StringBuilder();
           for (int i = 0; i < level - 1; ++i) {
             parentDimValues.append(row.getDimensionValues().get(i));
           }
-          HierarchyNode parentNode = curParent.get(parentDimValues.toString());
+          CubeNode parentNode = curParent.get(parentDimValues.toString());
           // Sometimes Pinot returns a node without any matching parent; we discard those nodes.
           if (parentNode == null) {
             continue;
           }
-          HierarchyNode node = new HierarchyNode(level, index, row, parentNode);
+          CubeNode node = row.toNode(level, index, parentNode);
           hierarchicalNodes.get(level).add(node);
           // Add current node's dimension values to next parent lookup table for the next level of nodes
           parentDimValues.append(row.getDimensionValues().get(level - 1));
           nextParent.put(parentDimValues.toString(), node);
         }
       } else { // root
-        Row row = hierarchicalRows.get(0).get(0);
-        HierarchyNode node = new HierarchyNode(row);
+        Row row = dataRows.get(0).get(0);
+        CubeNode node = row.toNode();
         hierarchicalNodes.get(0).add(node);
         nextParent.put("", node);
       }
@@ -318,11 +301,12 @@ public class Cube { // the cube (Ca|Cb)
     return hierarchicalNodes;
   }
 
-  private List<DimNameValueCostEntry> computeOneDimensionCost(OLAPDataBaseClient olapClient, double baselineTotal,
-      double currentTotal, Dimensions dimensions, Multimap<String, String> filterSets) throws Exception {
+  private List<DimNameValueCostEntry> computeOneDimensionCost(CubeClient olapClient, double topBaselineValue,
+      double topCurrentValue, double topBaselineSize, double topCurrentSize, Dimensions dimensions,
+      Multimap<String, String> filterSets) throws Exception {
 
-    double topRatio = currentTotal / baselineTotal;
-    LOG.info("baselineTotal:{}, currentTotal:{}, ratio:{}", baselineTotal, currentTotal, topRatio);
+    double topRatio = topCurrentValue / topBaselineValue;
+    LOG.info("topBaselineValue:{}, topCurrentValue:{}, changeRatio:{}", topBaselineValue, topCurrentValue, topRatio);
 
     List<DimNameValueCostEntry> costSet = new ArrayList<>();
     List<List<Row>> wowValuesOfDimensions = olapClient.getAggregatedValuesOfDimension(dimensions, filterSets);
@@ -330,18 +314,20 @@ public class Cube { // the cube (Ca|Cb)
       String dimensionName = dimensions.get(i);
       List<Row> wowValuesOfOneDimension = wowValuesOfDimensions.get(i);
       for (Row wowValues : wowValuesOfOneDimension) {
-        String dimensionValue = wowValues.getDimensionValues().get(0);
+        CubeNode wowNode = wowValues.toNode();
+        String dimensionValue = wowNode.getDimensionValues().get(0);
         double contributionFactor =
-            (wowValues.getBaselineValue() + wowValues.getCurrentValue()) / (baselineTotal + currentTotal);
-        double cost = costFunction
-            .computeCost(wowValues.getBaselineValue(), wowValues.getCurrentValue(), topRatio, baselineTotal, currentTotal);
+            (wowNode.getBaselineSize() + wowNode.getCurrentSize()) / (topBaselineSize + topCurrentSize);
+        double cost = costFunction.computeCost(topRatio, wowNode.getBaselineValue(), wowNode.getCurrentValue(),
+            wowNode.getBaselineSize(), wowNode.getCurrentSize(), topBaselineValue, topCurrentValue, topBaselineSize,
+            topCurrentSize);
 
-        costSet.add(new DimNameValueCostEntry(dimensionName, dimensionValue, wowValues.getBaselineValue(),
-            wowValues.getCurrentValue(), contributionFactor, cost));
+        costSet.add(new DimNameValueCostEntry(dimensionName, dimensionValue, wowNode.getBaselineValue(),
+            wowNode.getCurrentValue(), wowNode.getBaselineSize(), wowNode.getCurrentSize(), contributionFactor, cost));
       }
     }
 
-    Collections.sort(costSet, Collections.reverseOrder());
+    costSet.sort(Collections.reverseOrder());
     LOG.info("Top {} nodes (depth=1):", TOP_COST_ENTRIES_TO_LOG);
     for (DimNameValueCostEntry entry : costSet.subList(0, Math.min(costSet.size(), TOP_COST_ENTRIES_TO_LOG))) {
       LOG.info("\t{}", entry);
@@ -386,11 +372,11 @@ public class Cube { // the cube (Ca|Cb)
   }
 
   private static class HierarchicalDimensionCost {
-    List<String> dimensinoNames;
+    List<String> dimensionNames;
     double cost;
 
-    public HierarchicalDimensionCost(List<String> dimensinoNames, double cost) {
-      this.dimensinoNames = dimensinoNames;
+    public HierarchicalDimensionCost(List<String> dimensionNames, double cost) {
+      this.dimensionNames = dimensionNames;
       this.cost = cost;
     }
   }
@@ -417,7 +403,7 @@ public class Cube { // the cube (Ca|Cb)
     for (Map.Entry<String, Double> dimNameToCostEntry : dimNameToCost.entrySet()) {
       dimensionCosts.add(new DimensionCost(dimNameToCostEntry.getKey(), dimNameToCostEntry.getValue()));
     }
-    Collections.sort(dimensionCosts, new Comparator<DimensionCost>() {
+    dimensionCosts.sort(new Comparator<DimensionCost>() {
       @Override
       public int compare(DimensionCost d1, DimensionCost d2) {
         return Double.compare(d2.cost, d1.cost);
@@ -438,22 +424,22 @@ public class Cube { // the cube (Ca|Cb)
       List<List<String>> suggestedHierarchies) {
 
     // Trim the list of dimension cost to the max depth that is specified by users
-    List<DimensionCost> trimedSortedDimensionCosts =
+    List<DimensionCost> trimmedSortedDimensionCosts =
         sortedDimensionCosts.subList(0, Math.min(sortedDimensionCosts.size(), Math.max(1, depth)));
 
     // Reorder the dimensions based on the given hierarchy
-    List<String> dimensionsToBeOrdered = new ArrayList<>(trimedSortedDimensionCosts.size());
-    for (DimensionCost dimensionCost : trimedSortedDimensionCosts) {
+    List<String> dimensionsToBeOrdered = new ArrayList<>(trimmedSortedDimensionCosts.size());
+    for (DimensionCost dimensionCost : trimmedSortedDimensionCosts) {
       dimensionsToBeOrdered.add(dimensionCost.dimensinoName);
     }
     List<HierarchicalDimensionCost> hierarchicalDimensionCosts =
         getInitialHierarchicalDimensionList(dimensionsToBeOrdered, suggestedHierarchies);
-    sortHierarchicalDimensions(hierarchicalDimensionCosts, trimedSortedDimensionCosts);
+    sortHierarchicalDimensions(hierarchicalDimensionCosts, trimmedSortedDimensionCosts);
 
     // The ordered dimension names
     List<String> dimensionNames = new ArrayList<>();
     for (HierarchicalDimensionCost value : hierarchicalDimensionCosts) {
-      dimensionNames.addAll(value.dimensinoNames);
+      dimensionNames.addAll(value.dimensionNames);
     }
     return new Dimensions(dimensionNames);
   }
@@ -483,15 +469,15 @@ public class Cube { // the cube (Ca|Cb)
         continue;
       }
 
-      List<String> sanatizedHierarchy = new ArrayList<>();
+      List<String> sanitizedHierarchy = new ArrayList<>();
       for (String dimension : suggestedHierarchy) {
         if (availableDimensionKeySet.contains(dimension)) {
-          sanatizedHierarchy.add(dimension);
+          sanitizedHierarchy.add(dimension);
           availableDimensionKeySet.remove(dimension);
         }
       }
 
-      hierarchicalDimensionCosts.add(new HierarchicalDimensionCost(sanatizedHierarchy, 0));
+      hierarchicalDimensionCosts.add(new HierarchicalDimensionCost(sanitizedHierarchy, 0));
     }
 
     for (String remainDimension : availableDimensionKeySet) {
@@ -512,7 +498,7 @@ public class Cube { // the cube (Ca|Cb)
       List<DimensionCost> sortedDimensionCosts) {
     Map<String, HierarchicalDimensionCost> hierarchicalDimensionCostTable = new HashMap<>();
     for (HierarchicalDimensionCost hierarchicalDimensionCost : hierarchicalDimensionCosts) {
-      List<String> dimensions = hierarchicalDimensionCost.dimensinoNames;
+      List<String> dimensions = hierarchicalDimensionCost.dimensionNames;
       for (String dimension : dimensions) {
         hierarchicalDimensionCostTable.put(dimension, hierarchicalDimensionCost);
       }
@@ -525,11 +511,11 @@ public class Cube { // the cube (Ca|Cb)
       hierarchicalDimensionCost.cost += dimensionCost.cost;
     }
     for (HierarchicalDimensionCost hierarchicalDimensionCost : hierarchicalDimensionCosts) {
-      hierarchicalDimensionCost.cost /= hierarchicalDimensionCost.dimensinoNames.size();
+      hierarchicalDimensionCost.cost /= hierarchicalDimensionCost.dimensionNames.size();
     }
 
     // Sort the groups of hierarchical dimensions by their average cost
-    Collections.sort(hierarchicalDimensionCosts, new Comparator<HierarchicalDimensionCost>() {
+    hierarchicalDimensionCosts.sort(new Comparator<HierarchicalDimensionCost>() {
       @Override
       public int compare(HierarchicalDimensionCost o1, HierarchicalDimensionCost o2) {
         return Double.compare(o2.cost, o1.cost);
@@ -553,7 +539,7 @@ public class Cube { // the cube (Ca|Cb)
    * Reestablishes the hierarchy between aggregated and detailed rows after the cube is read from a Json string.
    */
   private void rebuildHierarchy() {
-    this.hierarchicalNodes = hierarchyRowToHierarchyNode(hierarchicalRows, dimensions);
+    this.hierarchicalNodes = dataRowToCubeNode(hierarchicalRows, dimensions);
   }
 
   @Override
@@ -561,8 +547,8 @@ public class Cube { // the cube (Ca|Cb)
     ToStringBuilder tsb = new ToStringBuilder(this, ToStringStyle.MULTI_LINE_STYLE);
     tsb.append("Baseline Value", baselineTotal)
         .append("Current Value", currentTotal)
-        .append("Ratio", topRatio)
-        .append("Dimentions", this.dimensions)
+        .append("Change Ratio", currentTotal / baselineTotal)
+        .append("Dimensions", this.dimensions)
         .append("#Detailed Rows", hierarchicalRows.get(hierarchicalRows.size() - 1).size());
     return tsb.toString();
   }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/cube/CubeUtils.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/cube/CubeUtils.java
new file mode 100644
index 0000000..99be81a
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/cube/CubeUtils.java
@@ -0,0 +1,85 @@
+/*
+ * 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.cube.data.cube;
+
+import com.google.common.collect.Multimap;
+import com.google.common.math.DoubleMath;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+
+
+public class CubeUtils {
+  private static double epsilon = 0.0001;
+
+  /**
+   * Removes dimensions from the given list of dimensions, which has single values in the filter set. Only dimensions
+   * with one value is removed from the given dimensions because setting a filter one dimension names with one dimension
+   * value (e.g., "country=US") implies that the final data cube does not contain other dimension values. Thus, the
+   * summary algorithm could simply ignore that dimension (because the cube does not have any other values to compare
+   * with in that dimension).
+   *
+   * @param dimensions the list of dimensions to be modified.
+   * @param filterSets the filter to be applied on the data cube.
+   *
+   * @return the list of dimensions that should be used for retrieving the data for summary algorithm.
+   */
+  public static Dimensions shrinkDimensionsByFilterSets(Dimensions dimensions, Multimap<String, String> filterSets) {
+    Set<String> dimensionsToRemove = new HashSet<>();
+    for (Map.Entry<String, Collection<String>> filterSetEntry : filterSets.asMap().entrySet()) {
+      if (filterSetEntry.getValue().size() == 1) {
+        dimensionsToRemove.add(filterSetEntry.getKey());
+      }
+    }
+    return removeDimensions(dimensions, dimensionsToRemove);
+  }
+
+  private static Dimensions removeDimensions(Dimensions dimensions, Collection<String> dimensionsToRemove) {
+    List<String> dimensionsToRetain = new ArrayList<>();
+    for (String dimensionName : dimensions.names()) {
+      if(!dimensionsToRemove.contains(dimensionName)){
+        dimensionsToRetain.add(dimensionName);
+      }
+    }
+    return new Dimensions(dimensionsToRetain);
+  }
+
+  /**
+   * Return the results of a minus b. If the result is very close to zero, then zero is returned.
+   * This method is use to prevent the precision issue of double from inducing -0.00000000000000001, which is
+   * actually zero.
+   *
+   * @param a a double value.
+   * @param b the other double value.
+   * @return the results of a minus b.
+   */
+  public static double doubleMinus(double a, double b) {
+    double ret = a - b;
+    if (DoubleMath.fuzzyEquals(ret, 0, epsilon)) {
+      return 0.0;
+    } else {
+      return ret;
+    }
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/DimNameValueCostEntry.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/cube/DimNameValueCostEntry.java
similarity index 72%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/DimNameValueCostEntry.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/cube/DimNameValueCostEntry.java
index 3f10f47..3cbd24e 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/DimNameValueCostEntry.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/cube/DimNameValueCostEntry.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
+package org.apache.pinot.thirdeye.cube.data.cube;
 
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Preconditions;
@@ -29,9 +29,11 @@ public class DimNameValueCostEntry implements Comparable<DimNameValueCostEntry>{
   private double contributionFactor;
   private double currentValue;
   private double baselineValue;
+  private double baselineSize;
+  private double currentSize;
 
   public DimNameValueCostEntry(String dimensionName, String dimensionValue, double baselineValue, double currentValue,
-      double contributionFactor, double cost) {
+      double baselineSize, double currentSize, double contributionFactor, double cost) {
     Preconditions.checkNotNull(dimensionName, "dimension name cannot be null.");
     Preconditions.checkNotNull(dimensionValue, "dimension value cannot be null.");
 
@@ -39,6 +41,8 @@ public class DimNameValueCostEntry implements Comparable<DimNameValueCostEntry>{
     this.dimValue = dimensionValue;
     this.baselineValue = baselineValue;
     this.currentValue = currentValue;
+    this.baselineSize = baselineSize;
+    this.currentSize = currentSize;
     this.contributionFactor = contributionFactor;
     this.cost = cost;
   }
@@ -91,6 +95,22 @@ public class DimNameValueCostEntry implements Comparable<DimNameValueCostEntry>{
     this.baselineValue = baselineValue;
   }
 
+  public double getBaselineSize() {
+    return baselineSize;
+  }
+
+  public void setBaselineSize(double baselineSize) {
+    this.baselineSize = baselineSize;
+  }
+
+  public double getCurrentSize() {
+    return currentSize;
+  }
+
+  public void setCurrentSize(double currentSize) {
+    this.currentSize = currentSize;
+  }
+
   @Override
   public int compareTo(DimNameValueCostEntry that) {
     return Double.compare(this.cost, that.cost);
@@ -98,10 +118,16 @@ public class DimNameValueCostEntry implements Comparable<DimNameValueCostEntry>{
 
   @Override
   public String toString() {
-    return MoreObjects.toStringHelper("Entry").add("dim", String.format("%s:%s", dimName, dimValue))
-        .add("baselineVal", baselineValue).add("currentVal", currentValue).add("delta", currentValue - baselineValue)
-        .add("ratio", String.format("%.2f", currentValue / baselineValue))
-        .add("sizeFactor", String.format("%.2f", contributionFactor)).add("cost", String.format("%.2f", cost))
+    return MoreObjects.toStringHelper("Entry")
+        .add("dim", String.format("%s:%s", dimName, dimValue))
+        .add("baselineVal", baselineValue)
+        .add("currentVal", currentValue)
+        .add("delta", currentValue - baselineValue)
+        .add("changeRatio", String.format("%.2f", currentValue / baselineValue))
+        .add("baselineSize", baselineSize)
+        .add("currentSize", currentSize)
+        .add("sizeFactor", String.format("%.2f", contributionFactor))
+        .add("cost", String.format("%.4f", cost))
         .toString();
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/BaseCubePinotClient.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/BaseCubePinotClient.java
new file mode 100644
index 0000000..ae5327d
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/BaseCubePinotClient.java
@@ -0,0 +1,261 @@
+/*
+ * 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.cube.data.dbclient;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.constant.MetricAggFunction;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Row;
+import org.apache.pinot.thirdeye.dashboard.Utils;
+import org.apache.pinot.thirdeye.datasource.MetricExpression;
+import org.apache.pinot.thirdeye.datasource.MetricFunction;
+import org.apache.pinot.thirdeye.datasource.ThirdEyeRequest;
+import org.apache.pinot.thirdeye.datasource.ThirdEyeResponse;
+import org.apache.pinot.thirdeye.datasource.cache.QueryCache;
+import org.apache.pinot.thirdeye.util.ThirdEyeUtils;
+import org.joda.time.DateTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BaseCubePinotClient<R extends Row> implements CubePinotClient<R> {
+  protected static final Logger LOG = LoggerFactory.getLogger(BaseCubePinotClient.class);
+
+  protected final static DateTime NULL_DATETIME = new DateTime();
+  protected final static int TIME_OUT_VALUE = 1200;
+  protected final static TimeUnit TIME_OUT_UNIT = TimeUnit.SECONDS;
+
+  protected QueryCache queryCache;
+  protected String dataset = "";
+  protected DateTime baselineStartInclusive = NULL_DATETIME;
+  protected DateTime baselineEndExclusive = NULL_DATETIME;
+  protected DateTime currentStartInclusive = NULL_DATETIME;
+  protected DateTime currentEndExclusive = NULL_DATETIME;
+
+  /**
+   * Constructs a Pinot client.
+   *
+   * @param queryCache the query cached to Pinot.
+   */
+  public BaseCubePinotClient(QueryCache queryCache) {
+    this.queryCache = Preconditions.checkNotNull(queryCache);
+  }
+
+  public void setDataset(String dataset) {
+    this.dataset = Preconditions.checkNotNull(dataset);
+  }
+
+  @Override
+  public void setBaselineStartInclusive(DateTime dateTime) {
+    baselineStartInclusive = Preconditions.checkNotNull(dateTime);
+  }
+
+  @Override
+  public void setBaselineEndExclusive(DateTime dateTime) {
+    baselineEndExclusive = Preconditions.checkNotNull(dateTime);
+  }
+
+  @Override
+  public void setCurrentStartInclusive(DateTime dateTime) {
+    currentStartInclusive = Preconditions.checkNotNull(dateTime);
+  }
+
+  @Override
+  public void setCurrentEndExclusive(DateTime dateTime) {
+    currentEndExclusive = Preconditions.checkNotNull(dateTime);
+  }
+
+  /**
+   * Construct bulks ThirdEye requests.
+   *
+   * @param dataset the data set to be queries.
+   * @param cubeSpecs the spec to retrieve the metrics.
+   * @param groupBy groupBy for database.
+   * @param filterSets the data filter.
+   * @return a list of ThirdEye requests.
+   */
+  protected static Map<CubeTag, ThirdEyeRequestMetricExpressions> constructBulkRequests(String dataset,
+      List<CubeSpec> cubeSpecs, List<String> groupBy, Multimap<String, String> filterSets) throws ExecutionException {
+
+    Map<CubeTag, ThirdEyeRequestMetricExpressions> requests = new HashMap<>();
+
+    for (CubeSpec cubeSpec : cubeSpecs) {
+      // Set dataset and metric
+      List<MetricExpression> metricExpressions =
+          Utils.convertToMetricExpressions(cubeSpec.getMetric(), MetricAggFunction.SUM, dataset);
+      List<MetricFunction> metricFunctions = metricExpressions.get(0).computeMetricFunctions();
+
+      ThirdEyeRequest.ThirdEyeRequestBuilder builder = ThirdEyeRequest.newBuilder();
+
+      builder.setMetricFunctions(metricFunctions);
+      builder.setDataSource(ThirdEyeUtils.getDataSourceFromMetricFunctions(metricFunctions));
+
+      // Set start and end time
+      builder.setStartTimeInclusive(cubeSpec.getStartInclusive());
+      builder.setEndTimeExclusive(cubeSpec.getEndExclusive());
+
+      // Set groupBy and filter
+      builder.setGroupBy(groupBy);
+      builder.setFilterSet(filterSets);
+
+      requests.put(cubeSpec.getTag(),
+          new ThirdEyeRequestMetricExpressions(builder.build(cubeSpec.getTag().toString()), metricExpressions));
+    }
+
+    return requests;
+  }
+
+  /**
+   * The cube specs that specified which metric and dataset to be queried.
+   *
+   * @return a list of cube spec.
+   */
+  protected abstract List<CubeSpec> getCubeSpecs();
+
+  /**
+   * Fills in multiple Pinot results to one Cube row.
+   *
+   * @param rowTable the table from dimension values to cube row; the return of this method.
+   * @param dimensions the dimension names of the row.
+   * @param dimensionValues the dimension values of the row.
+   * @param value the value to be filled in to the row.
+   * @param tag The field of the row where the value is filled in.
+   */
+  protected abstract void fillValueToRowTable(Map<List<String>, R> rowTable, Dimensions dimensions,
+      List<String> dimensionValues, double value, CubeTag tag);
+
+  /**
+   * Returns a list of rows. The value of each row is evaluated and no further processing is needed.
+   * @param dimensions dimensions of the response
+   * @param response the response from backend database
+   * @param rowTable the storage for rows
+   * @param tag true if the response is for baseline values
+   */
+  protected void buildMetricFunctionOrExpressionsRows(Dimensions dimensions, List<MetricExpression> metricExpressions,
+      List<MetricFunction> metricFunctions, ThirdEyeResponse response, Map<List<String>, R> rowTable, CubeTag tag) {
+    Map<String, Double> context = new HashMap<>();
+    for (int rowIdx = 0; rowIdx < response.getNumRows(); ++rowIdx) {
+      double value = 0d;
+      // If the metric expression is a single metric function, then we get the value immediately
+      if (metricFunctions.size() <= 1) {
+        value = response.getRow(rowIdx).getMetrics().get(0);
+      } else { // Otherwise, we need to evaluate the expression
+        for (int metricFuncIdx = 0; metricFuncIdx < metricFunctions.size(); ++metricFuncIdx) {
+          double contextValue = response.getRow(rowIdx).getMetrics().get(metricFuncIdx);
+          context.put(metricFunctions.get(metricFuncIdx).getMetricName(), contextValue);
+        }
+        try {
+          value = MetricExpression.evaluateExpression(metricExpressions.get(0), context);
+        } catch (Exception e) {
+          LOG.warn(e.getMessage());
+        }
+      }
+      List<String> dimensionValues = response.getRow(rowIdx).getDimensions();
+      fillValueToRowTable(rowTable, dimensions, dimensionValues, value, tag);
+    }
+  }
+
+  /**
+   * Converts Pinot results to Cube Rows.
+   *
+   * @param dimensions the dimension of the Pinot results.
+   * @param bulkRequests the original requests of those results.
+   * @return Cube rows.
+   */
+  protected List<List<R>> constructAggregatedValues(Dimensions dimensions,
+      List<Map<CubeTag, ThirdEyeRequestMetricExpressions>> bulkRequests) throws Exception {
+
+    List<ThirdEyeRequest> allRequests = new ArrayList<>();
+    for (Map<CubeTag, ThirdEyeRequestMetricExpressions> bulkRequest : bulkRequests) {
+      for (Map.Entry<CubeTag, ThirdEyeRequestMetricExpressions> entry : bulkRequest.entrySet()) {
+        ThirdEyeRequest thirdEyeRequest = entry.getValue().getThirdEyeRequest();
+        allRequests.add(thirdEyeRequest);
+      }
+    }
+
+    Map<ThirdEyeRequest, Future<ThirdEyeResponse>> queryResponses = queryCache.getQueryResultsAsync(allRequests);
+
+    List<List<R>> res = new ArrayList<>();
+    int level = 0;
+    for (Map<CubeTag, ThirdEyeRequestMetricExpressions> bulkRequest : bulkRequests) {
+      Map<List<String>, R> rowOfSameLevel = new HashMap<>();
+
+      for (Map.Entry<CubeTag, ThirdEyeRequestMetricExpressions> entry : bulkRequest.entrySet()) {
+        CubeTag tag = entry.getKey();
+        ThirdEyeRequest thirdEyeRequest = entry.getValue().getThirdEyeRequest();
+        ThirdEyeResponse thirdEyeResponse = queryResponses.get(thirdEyeRequest).get(TIME_OUT_VALUE, TIME_OUT_UNIT);
+        if (thirdEyeResponse.getNumRows() == 0) {
+          LOG.warn("Get 0 rows from the request(s): {}", thirdEyeRequest);
+        }
+        List<MetricExpression> metricExpressions = entry.getValue().getMetricExpressions();
+        buildMetricFunctionOrExpressionsRows(dimensions, metricExpressions, thirdEyeRequest.getMetricFunctions(),
+            thirdEyeResponse, rowOfSameLevel, tag);
+      }
+      if (rowOfSameLevel.size() == 0) {
+        LOG.warn("Failed to retrieve non-zero results for requests of level {}.", level);
+      }
+      List<R> rows = new ArrayList<>(rowOfSameLevel.values());
+      res.add(rows);
+      ++level;
+    }
+
+    return res;
+  }
+
+  @Override
+  public R getTopAggregatedValues(Multimap<String, String> filterSets) throws Exception {
+    List<String> groupBy = Collections.emptyList();
+    List<Map<CubeTag, ThirdEyeRequestMetricExpressions>> bulkRequests = Collections.singletonList(
+        BaseCubePinotClient.constructBulkRequests(dataset, getCubeSpecs(), groupBy, filterSets));
+    return constructAggregatedValues(new Dimensions(), bulkRequests).get(0).get(0);
+  }
+
+  @Override
+  public List<List<R>> getAggregatedValuesOfDimension(Dimensions dimensions, Multimap<String, String> filterSets)
+      throws Exception {
+    List<Map<CubeTag, ThirdEyeRequestMetricExpressions>> bulkRequests = new ArrayList<>();
+    for (int level = 0; level < dimensions.size(); ++level) {
+      List<String> groupBy = Lists.newArrayList(dimensions.get(level));
+      bulkRequests.add(BaseCubePinotClient.constructBulkRequests(dataset, getCubeSpecs(), groupBy, filterSets));
+    }
+    return constructAggregatedValues(dimensions, bulkRequests);
+  }
+
+  @Override
+  public List<List<R>> getAggregatedValuesOfLevels(Dimensions dimensions, Multimap<String, String> filterSets)
+      throws Exception {
+    List<Map<CubeTag, ThirdEyeRequestMetricExpressions>> bulkRequests = new ArrayList<>();
+    for (int level = 0; level < dimensions.size() + 1; ++level) {
+      List<String> groupBy = Lists.newArrayList(dimensions.namesToDepth(level));
+      bulkRequests.add(BaseCubePinotClient.constructBulkRequests(dataset, getCubeSpecs(), groupBy, filterSets));
+    }
+    return constructAggregatedValues(dimensions, bulkRequests);
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/CubeClient.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/CubeClient.java
new file mode 100644
index 0000000..e586aab
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/CubeClient.java
@@ -0,0 +1,70 @@
+/*
+ * 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.cube.data.dbclient;
+
+import com.google.common.collect.Multimap;
+import java.util.List;
+
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Row;
+
+
+/**
+ * The database client that provides the function of data retrieval for the cube algorithm.
+ * @param <R>
+ */
+public interface CubeClient<R extends Row> {
+
+  /**
+   * Returns the baseline and current value for the root node.
+   *
+   * @param filterSets the data filter.
+   * @return a row of data that contains the baseline and current value for the root node.
+   */
+  R getTopAggregatedValues(Multimap<String, String> filterSets) throws Exception;
+
+  /**
+   * Returns the baseline and current value for nodes at each dimension from the given list.
+   * For instance, if the list has ["country", "page name"], then it returns nodes of ["US", "IN", "JP", ...,
+   * "linkedin.com", "google.com", ...]
+   *
+   * @param dimensions the list of dimensions.
+   * @param filterSets the data filter.
+   *
+   * @return the baseline and current value for nodes at each dimension from the given list.
+   */
+  List<List<R>> getAggregatedValuesOfDimension(Dimensions dimensions, Multimap<String, String> filterSets)
+      throws Exception;
+
+  /**
+   * Returns the baseline and current value for nodes for each dimension combination.
+   * For instance, if the list has ["country", "page name"], then it returns nodes of
+   * [
+   *   ["US", "IN", "JP", ...,],
+   *   ["US, linkedin.com", "US, google.com", "IN, linkedin.com", "IN, google.com", "JP, linkedin.com", "JP, google.com", ...]
+   * ]
+   * @param dimensions the dimensions to be drilled down.
+   * @param filterSets the data filter.
+   *
+   * @return the baseline and current value for nodes for each dimension combination.
+   */
+  List<List<R>> getAggregatedValuesOfLevels(Dimensions dimensions, Multimap<String, String> filterSets)
+      throws Exception;
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/OLAPDataBaseClient.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/CubePinotClient.java
similarity index 54%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/OLAPDataBaseClient.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/CubePinotClient.java
index 84b5fda..939f03d 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/OLAPDataBaseClient.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/CubePinotClient.java
@@ -17,34 +17,51 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
-
-import com.google.common.collect.Multimap;
-import java.util.List;
+package org.apache.pinot.thirdeye.cube.data.dbclient;
 
+import org.apache.pinot.thirdeye.cube.data.dbrow.Row;
 import org.joda.time.DateTime;
 
-import org.apache.pinot.thirdeye.datasource.MetricExpression;
-
-public interface OLAPDataBaseClient {
 
-  void setCollection(String collection);
+/**
+ * The CubeClient that is based on Pinot.
+ * @param <R>
+ */
+public interface CubePinotClient<R extends Row> extends CubeClient<R> {
 
-  void setMetricExpression(MetricExpression metricExpressions);
+  /**
+   * Sets Pinot dataset.
+   *
+   * @param dataset the dataset.
+   */
+  void setDataset(String dataset);
 
+  /**
+   * Sets the start date time of baseline (inclusive).
+   *
+   * @param dateTime the start date time of baseline (inclusive).
+   */
   void setBaselineStartInclusive(DateTime dateTime);
 
+  /**
+   * Sets the end data time of baseline (exclusive).
+   *
+   * @param dateTime the end data time of baseline (exclusive).
+   */
   void setBaselineEndExclusive(DateTime dateTime);
 
+  /**
+   * Sets the start date time of current (inclusive).
+   *
+   * @param dateTime the start date time of current (inclusive).
+   */
   void setCurrentStartInclusive(DateTime dateTime);
 
+  /**
+   * Sets the end date time of current (exclusive).
+   *
+   * @param dateTime the end date time of current (exclusive).
+   */
   void setCurrentEndExclusive(DateTime dateTime);
 
-  Row getTopAggregatedValues(Multimap<String, String> filterSets) throws Exception;
-
-  List<List<Row>> getAggregatedValuesOfDimension(Dimensions dimensions, Multimap<String, String> filterSets)
-      throws Exception;
-
-  List<List<Row>> getAggregatedValuesOfLevels(Dimensions dimensions, Multimap<String, String> filterSets)
-      throws Exception;
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/CubeSpec.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/CubeSpec.java
new file mode 100644
index 0000000..d4eb3f7
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/CubeSpec.java
@@ -0,0 +1,123 @@
+/*
+ * 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.cube.data.dbclient;
+
+import com.google.common.base.Preconditions;
+import org.joda.time.DateTime;
+
+
+/**
+ * The spec that specifies the metric and its time range to be retrieved from the data base.
+ */
+public class CubeSpec {
+  private CubeTag tag;
+  private String metric;
+  private DateTime startInclusive;
+  private DateTime endExclusive;
+
+  /**
+   * Constructs a cube spec.
+   *
+   * @param tag the field name corresponds to the retrieved metric.
+   * @param metric the name of the metric.
+   * @param startInclusive start time of the metric, inclusive.
+   * @param endExclusive the time of the metric, exclusive.
+   */
+  public CubeSpec(CubeTag tag, String metric, DateTime startInclusive, DateTime endExclusive) {
+    setTag(tag);
+    setMetric(metric);
+    setStartInclusive(startInclusive);
+    setEndExclusive(endExclusive);
+  }
+
+  /**
+   * Returns the field name corresponds to the retrieved metric.
+   *
+   * @return the field name corresponds to the retrieved metric.
+   */
+  public CubeTag getTag() {
+    return tag;
+  }
+
+  /**
+   * Sets the field name corresponds to the retrieved metric.
+   *
+   * @param tag the field name corresponds to the retrieved metric.
+   */
+  public void setTag(CubeTag tag) {
+    Preconditions.checkNotNull(tag);
+    this.tag = tag;
+  }
+
+  /**
+   * Returns the metric name.
+   *
+   * @return the metric name.
+   */
+  public String getMetric() {
+    return metric;
+  }
+
+  /**
+   * Sets the metric name.
+   * @param metric the metric name.
+   */
+  public void setMetric(String metric) {
+    Preconditions.checkNotNull(metric);
+    this.metric = metric;
+  }
+
+  /**
+   * Returns start time of the metric, inclusive.
+   *
+   * @return start time of the metric, inclusive.
+   */
+  public DateTime getStartInclusive() {
+    return startInclusive;
+  }
+
+  /**
+   * Sets start time of the metric, inclusive.
+   * @param startInclusive start time of the metric, inclusive.
+   */
+  public void setStartInclusive(DateTime startInclusive) {
+    Preconditions.checkNotNull(startInclusive);
+    this.startInclusive = startInclusive;
+  }
+
+  /**
+   * Returns end time of the metric, exclusive.
+   *
+   * @return end time of the metric, exclusive.
+   */
+  public DateTime getEndExclusive() {
+    return endExclusive;
+  }
+
+  /**
+   * Sets end time of the metric, exclusive.
+   *
+   * @param endExclusive end time of the metric, exclusive.
+   */
+  public void setEndExclusive(DateTime endExclusive) {
+    Preconditions.checkNotNull(endExclusive);
+    this.endExclusive = endExclusive;
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/CostFunction.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/CubeTag.java
similarity index 76%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/CostFunction.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/CubeTag.java
index 44dbb85..c5bc2de 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/CostFunction.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/CubeTag.java
@@ -17,9 +17,11 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary.costfunctions;
+package org.apache.pinot.thirdeye.cube.data.dbclient;
 
-public interface CostFunction {
-  double computeCost(double baselineValue, double currentValue, double parentRatio, double globalBaselineValue,
-      double globalCurrentValue);
+/**
+ * The field names where the metric value should be filled in.
+ */
+public enum CubeTag {
+  Baseline, Current, BaselineNumerator, CurrentNumerator, BaselineDenominator, CurrentDenominator
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/ThirdEyeRequestMetricExpressions.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/ThirdEyeRequestMetricExpressions.java
new file mode 100644
index 0000000..8bdd2ed
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbclient/ThirdEyeRequestMetricExpressions.java
@@ -0,0 +1,63 @@
+/*
+ * 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.cube.data.dbclient;
+
+import java.util.List;
+import org.apache.pinot.thirdeye.datasource.MetricExpression;
+import org.apache.pinot.thirdeye.datasource.ThirdEyeRequest;
+
+
+/**
+ * Classes to keep the information that can be used to construct the metric value from Pinot results.
+ */
+public class ThirdEyeRequestMetricExpressions {
+  private ThirdEyeRequest thirdEyeRequest;
+  private List<MetricExpression> metricExpressions;
+
+  /**
+   * Construct a pair of ThirdEye request and metric expression, which can be used to construct the metric value from
+   * Pinot results.
+   *
+   * @param thirdEyeRequest the ThirdEye request.
+   * @param metricExpressions the metric expression of the ThirdEye request.
+   */
+  public ThirdEyeRequestMetricExpressions(ThirdEyeRequest thirdEyeRequest, List<MetricExpression> metricExpressions) {
+    this.thirdEyeRequest = thirdEyeRequest;
+    this.metricExpressions = metricExpressions;
+  }
+
+  /**
+   * Returns the ThirdEye request.
+   *
+   * @return the ThirdEye request.
+   */
+  public ThirdEyeRequest getThirdEyeRequest() {
+    return thirdEyeRequest;
+  }
+
+  /**
+   * Returns the metric expression of the ThirdEye request.
+   *
+   * @return the metric expression of the ThirdEye request.
+   */
+  public List<MetricExpression> getMetricExpressions() {
+    return metricExpressions;
+  }
+}
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/BaseResponseRow.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbrow/BaseRow.java
similarity index 54%
copy from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/BaseResponseRow.java
copy to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbrow/BaseRow.java
index 76fa644..bf02f69 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/BaseResponseRow.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbrow/BaseRow.java
@@ -17,12 +17,32 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.dashboard.views.diffsummary;
-
-public class BaseResponseRow {
-  public double baselineValue;
-  public double currentValue;
-  public String percentageChange;
-  public String contributionChange;
-  public String contributionToOverallChange;
+package org.apache.pinot.thirdeye.cube.data.dbrow;
+
+import com.google.common.base.Preconditions;
+
+
+public abstract class BaseRow implements Row {
+  protected Dimensions dimensions;
+  protected DimensionValues dimensionValues;
+
+  @Override
+  public Dimensions getDimensions() {
+    return dimensions;
+  }
+
+  @Override
+  public void setDimensions(Dimensions dimensions) {
+    this.dimensions = Preconditions.checkNotNull(dimensions);
+  }
+
+  @Override
+  public DimensionValues getDimensionValues() {
+    return dimensionValues;
+  }
+
+  @Override
+  public void setDimensionValues(DimensionValues dimensionValues) {
+    this.dimensionValues = Preconditions.checkNotNull(dimensionValues);
+  }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/DimensionValues.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbrow/DimensionValues.java
similarity index 98%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/DimensionValues.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbrow/DimensionValues.java
index b8de356..b94f7ad 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/DimensionValues.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbrow/DimensionValues.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
+package org.apache.pinot.thirdeye.cube.data.dbrow;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.collect.ImmutableList;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/Dimensions.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbrow/Dimensions.java
similarity index 96%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/Dimensions.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbrow/Dimensions.java
index 879b267..3654146 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/client/diffsummary/Dimensions.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbrow/Dimensions.java
@@ -17,9 +17,8 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
+package org.apache.pinot.thirdeye.cube.data.dbrow;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import java.util.HashSet;
 import java.util.List;
@@ -36,7 +35,7 @@ public class Dimensions {
   @JsonProperty("names")
   private ImmutableList<String> names;
 
-  Dimensions() {
+  public Dimensions() {
     names = ImmutableList.of();
   }
 
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbrow/Row.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbrow/Row.java
new file mode 100644
index 0000000..81fc071
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/dbrow/Row.java
@@ -0,0 +1,78 @@
+/*
+ * 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.cube.data.dbrow;
+
+import org.apache.pinot.thirdeye.cube.data.node.CubeNode;
+
+
+public interface Row {
+
+  /**
+   * Returns the dimension names of this row, such as ["country", "page key"].
+   *
+   * @return the dimension names of this row.
+   */
+  Dimensions getDimensions();
+
+  /**
+   * Sets dimension names of this row, such as ["country", "page key"].
+   *
+   * @param dimensions the dimension names for this row.
+   */
+  void setDimensions(Dimensions dimensions);
+
+  /**
+   * Returns dimension values of this row, such as ["US", "linkedin.com"]
+   *
+   * @return dimension values of this row, such as ["US", "linkedin.com"]
+   */
+  DimensionValues getDimensionValues();
+
+  /**
+   * Sets dimension values of this row, such as ["US", "linkedin.com"]
+   *
+   * @param dimensionValues the dimension values for this row.
+   */
+  void setDimensionValues(DimensionValues dimensionValues);
+
+  /**
+   * Converts current row to a CubeNode.
+   *
+   * @return a CubeNode of this row.
+   */
+  CubeNode toNode();
+
+  /**
+   * Converts current row to a CubeNode.
+   *
+   * @param level the level of this node; 0 is the top level.
+   * @param index the index of this node, which is used for speeding up algorithm speed.
+   * @param parent the parent of this node.
+   *
+   * @return a CubeNode of this row.
+   */
+  CubeNode toNode(int level, int index, CubeNode parent);
+
+  @Override
+  boolean equals(Object o);
+
+  @Override
+  int hashCode();
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/node/BaseCubeNode.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/node/BaseCubeNode.java
new file mode 100644
index 0000000..9a3558e
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/node/BaseCubeNode.java
@@ -0,0 +1,144 @@
+/*
+ * 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.cube.data.node;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.pinot.thirdeye.cube.data.dbrow.DimensionValues;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Row;
+
+
+public abstract class BaseCubeNode<N extends BaseCubeNode, R extends Row> implements CubeNode<N> {
+  protected int level;
+  protected int index;
+  protected double cost;
+  protected R data;
+  protected N parent;
+  protected List<N> children = new ArrayList<>();
+
+  /**
+   * Constructs a CubeNode with the given data, i.e., a Row. The level, index, cost will be set to 0; parent will
+   * be set to null.
+   *
+   * @param data the data of this cube node.
+   */
+  public BaseCubeNode(R data) {
+    this.data = Preconditions.checkNotNull(data);
+  }
+
+  /**
+   * Constructs a CubeNode with the given data and specified information.
+   *
+   * @param level the level of this node; level 0 the topmost level.
+   * @param index the index of this node, which is used to speed up algorithm.
+   * @param data the data of this cube node.
+   * @param parent the parent of this cube node.
+   */
+  public BaseCubeNode(int level, int index, R data, N parent) {
+    this(data);
+    this.level = level;
+    this.index = index;
+    Preconditions.checkArgument((level != 0 && parent != null) || (level == 0 && parent == null));
+    this.parent = parent;
+    if (parent != null) { // non root node
+      Dimensions parentDimension = new Dimensions(parent.getDimensions().namesToDepth(parent.getLevel()));
+      Dimensions childDimension = new Dimensions(data.getDimensions().namesToDepth(level));
+      Preconditions.checkState(parentDimension.isParentOf(childDimension),
+          "Current node is not a child node of the given parent node. Current and parent dimensions: ",
+          data.getDimensions(), parent.getDimensions());
+      parent.children.add(this);
+    }
+  }
+
+  @Override
+  public int getLevel() {
+    return level;
+  }
+
+  @Override
+  public double getCost() {
+    return cost;
+  }
+
+  @Override
+  public void setCost(double cost) {
+    this.cost = cost;
+  }
+
+  @Override
+  @JsonIgnore
+  public Dimensions getDimensions() {
+    return data.getDimensions();
+  }
+
+  @Override
+  @JsonIgnore
+  public DimensionValues getDimensionValues() {
+    return data.getDimensionValues();
+  }
+
+  @Override
+  public N getParent() {
+    return parent;
+  }
+
+  @Override
+  public int childrenSize() {
+    return children.size();
+  }
+
+  @Override
+  public List<N> getChildren() {
+    return Collections.unmodifiableList(children);
+  }
+
+  @Override
+  public double targetChangeRatio() {
+    double ratio = changeRatio();
+    if (!Double.isInfinite(ratio) && Double.compare(ratio, 0d) != 0) {
+      return ratio;
+    } else {
+      ratio = originalChangeRatio();
+      if (!Double.isInfinite(ratio) && Double.compare(ratio, 0d) != 0) {
+        return ratio;
+      } else {
+        if (parent != null) {
+          return parent.targetChangeRatio();
+        } else {
+          return 1.;
+        }
+      }
+    }
+  }
+
+  @Override
+  public boolean side() {
+    double ratio = changeRatio();
+    if (!Double.isNaN(ratio)) {
+      return Double.compare(1., changeRatio()) <= 0;
+    } else {
+      return Double.compare(1., originalChangeRatio()) <= 0;
+    }
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/node/CubeNode.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/node/CubeNode.java
new file mode 100644
index 0000000..ff37eb7
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/node/CubeNode.java
@@ -0,0 +1,211 @@
+/*
+ * 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.cube.data.node;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import java.util.List;
+import org.apache.pinot.thirdeye.cube.data.dbrow.DimensionValues;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+
+
+public interface CubeNode<N extends CubeNode> {
+
+  /**
+   * Returns the level of this node in the cube; level 0 the topmost level.
+   *
+   * @return the level of this node in the cube.
+   */
+  int getLevel();
+
+  /**
+   * Resets all values (e.g., baseline and current value) of this node.
+   */
+  void resetValues();
+
+  /**
+   * Updates all values when the child node is extracted from this node.
+   *
+   * @param node the child node to be extracted.
+   */
+  void removeNodeValues(N node);
+
+  /**
+   * Updates all values when an extracted child node is added back to this node.
+   *
+   * @param node the child node to be added back.
+   */
+  void addNodeValues(N node);
+
+  /**
+   * Returns the latest node size of baseline time period.
+   *
+   * @return the latest node size of baseline time period.
+   */
+  double getBaselineSize();
+
+  /**
+   * Returns the latest node size of current time period.
+   *
+   * @return the latest node size of current time period.
+   */
+  double getCurrentSize();
+
+  /**
+   * Returns the original node size of baseline time period.
+   *
+   * @return the original node size of baseline time period.
+   */
+  double getOriginalBaselineSize();
+
+  /**
+   * Returns the original node size of current time period.
+   *
+   * @return the original node size of current time period.
+   */
+  double getOriginalCurrentSize();
+
+  /**
+   * Returns the latest baseline value.
+   *
+   * @return the latest baseline value.
+   */
+  double getBaselineValue();
+
+  /**
+   * Returns the latest current value.
+   *
+   * @return the latest current value.
+   */
+  double getCurrentValue();
+
+  /**
+   * Returns the original baseline value.
+   *
+   * @return the original baseline value.
+   */
+  @JsonIgnore
+  double getOriginalBaselineValue();
+
+  /**
+   * Return the original current value.
+   *
+   * @return the original current value.
+   */
+  @JsonIgnore
+  double getOriginalCurrentValue();
+
+  /**
+   * Returns the latest cost of this node.
+   *
+   * @return the latest cost of this node.
+   */
+  double getCost();
+
+  /**
+   * Sets the latest cost of this node.
+   *
+   * @param cost the latest cost of this node.
+   */
+  void setCost(double cost);
+
+  /**
+   * Returns the dimension names of this node, e.g., ["country", "page key"]
+   *
+   * @return the dimension names of this node.
+   */
+  @JsonIgnore
+  Dimensions getDimensions();
+
+  /**
+   * Returns the dimension values of this node, e.g., ["US", "linkedin.com"]
+   *
+   * @return the dimension values of this node.
+   */
+  @JsonIgnore
+  DimensionValues getDimensionValues();
+
+  /**
+   * Returns the parent of this node.
+   *
+   * @return the parent of this node.
+   */
+  N getParent();
+
+  /**
+   * Returns the number of children of this node.
+   *
+   * @return the number of children of this node.
+   */
+  int childrenSize();
+
+  /**
+   * Returns the children list of this node.
+   *
+   * @return the children list of this node.
+   */
+  List<N> getChildren();
+
+  /**
+   * Returns the changeRatio that is calculated by the aggregate current and aggregate baseline values of all children node.
+   *
+   * @return aggregated current value of all children / aggregated baseline value of all children;
+   */
+  double originalChangeRatio();
+
+  /**
+   * Returns the changeRatio that is calculated by currentValue and baselineValue.
+   * @return currentValue / baselineValue;
+   */
+  double changeRatio();
+
+  /**
+   * Return the changeRatio of the node. If the changeRatio is not a finite number, then it returns the originalChangeRatio.
+   * If the originalChangeRatio is not a finite number, then it bootstraps to the parents until it finds a finite
+   * changeRatio. If no finite changeRatio available, then it returns 1.
+   */
+  double targetChangeRatio();
+
+  /**
+   * Returns the current changeRatio of this node is increased or decreased, i.e., returns true if changeRatio of the node >= 1.0.
+   * If the current changeRatio is NAN, then the changeRatio of the aggregated values is used.
+   *
+   * Precondition: the aggregated baseline and current values cannot both be zero.
+   */
+  boolean side();
+
+  /**
+   * Returns if the data of current node equals to the data of other node. The parent and children nodes are not
+   * compared due to the cyclic references between parent and children nodes.
+   *
+   * @param o the other node.
+   *
+   * @return true if the data of current node equals to the data of other node.
+   */
+  @Override
+  boolean equals(Object o);
+
+  /**
+   * Returns the hash code that is generated base on the data of this node.
+   *
+   * @return the hash code that is generated base on the data of this node.
+   */
+  @Override
+  int hashCode();
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/node/CubeNodeUtils.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/node/CubeNodeUtils.java
new file mode 100644
index 0000000..3c37797
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/data/node/CubeNodeUtils.java
@@ -0,0 +1,72 @@
+/*
+ * 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.cube.data.node;
+
+import java.util.List;
+import org.apache.commons.lang.ObjectUtils;
+
+
+public class CubeNodeUtils {
+
+  /**
+   * Check if the hierarchical tree of the given two root nodes are the same.
+   *
+   * @param node1 the root node of the first hierarchical tree.
+   * @param node2 the root node of the second hierarchical tree.
+   *
+   * @return true if both hierarchical tree are the same.
+   */
+  public static boolean equalHierarchy(CubeNode node1, CubeNode node2) {
+    return equalHierarchy(node1, null, node2, null);
+  }
+
+  public static boolean equalHierarchy(CubeNode node1, CubeNode node1Parent, CubeNode node2, CubeNode node2Parent) {
+    boolean sameData = ObjectUtils.equals(node1, node2);
+    if (sameData) {
+      // Check parent reference
+      if (node1Parent != null && node1.getParent() != node1Parent) {
+        return false;
+      }
+      if (node2Parent != null && node2.getParent() != node2Parent) {
+        return false;
+      }
+
+      // Check children reference
+      if (node1.childrenSize() != node2.childrenSize()) {
+        return false;
+      }
+      List<CubeNode> children1 = node1.getChildren();
+      List<CubeNode> children2 = node2.getChildren();
+      int size = children1.size();
+      for (int i = 0; i < size; i++) {
+        CubeNode child1 = children1.get(i);
+        CubeNode child2 = children2.get(i);
+        boolean sameChild = equalHierarchy(child1, node1, child2, node2);
+        if (!sameChild) {
+          return false;
+        }
+      }
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/BaseResponseRow.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/BaseResponseRow.java
similarity index 90%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/BaseResponseRow.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/BaseResponseRow.java
index 76fa644..0c7ef3b 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/BaseResponseRow.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/BaseResponseRow.java
@@ -17,11 +17,13 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.dashboard.views.diffsummary;
+package org.apache.pinot.thirdeye.cube.summary;
 
 public class BaseResponseRow {
   public double baselineValue;
   public double currentValue;
+  public double baselineSize;
+  public double currentSize;
   public String percentageChange;
   public String contributionChange;
   public String contributionToOverallChange;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/DPArray.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/DPArray.java
similarity index 91%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/DPArray.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/DPArray.java
index ab299ed..25e1639 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/DPArray.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/DPArray.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.dashboard.views.diffsummary;
+package org.apache.pinot.thirdeye.cube.summary;
 
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -27,7 +27,7 @@ import java.util.Set;
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
 
-import org.apache.pinot.thirdeye.client.diffsummary.HierarchyNode;
+import org.apache.pinot.thirdeye.cube.data.node.CubeNode;
 
 
 public class DPArray {
@@ -61,7 +61,7 @@ public class DPArray {
     this.shrankSize = size;
   }
 
-  public Set<HierarchyNode> getAnswer() {
+  public Set<CubeNode> getAnswer() {
     return slots.get(this.shrankSize - 1).ans;
   }
 
@@ -92,7 +92,7 @@ public class DPArray {
 
   public static class DPSlot {
     double cost;
-    Set<HierarchyNode> ans = new HashSet<>();
+    Set<CubeNode> ans = new HashSet<>();
 
     public String toString() {
       return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/Summary.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/Summary.java
similarity index 73%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/Summary.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/Summary.java
index aaddaaa..f523fca 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/Summary.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/Summary.java
@@ -17,10 +17,10 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.dashboard.views.diffsummary;
+package org.apache.pinot.thirdeye.cube.summary;
 
-import org.apache.pinot.thirdeye.client.diffsummary.DimNameValueCostEntry;
-import org.apache.pinot.thirdeye.client.diffsummary.costfunctions.BalancedCostFunction;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -28,15 +28,13 @@ import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-
+import org.apache.pinot.thirdeye.cube.data.cube.Cube;
+import org.apache.pinot.thirdeye.cube.data.cube.DimNameValueCostEntry;
+import org.apache.pinot.thirdeye.cube.cost.BalancedCostFunction;
+import org.apache.pinot.thirdeye.cube.cost.CostFunction;
+import org.apache.pinot.thirdeye.cube.data.node.CubeNode;
 import org.jfree.util.Log;
 
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.pinot.thirdeye.client.diffsummary.costfunctions.CostFunction;
-import org.apache.pinot.thirdeye.client.diffsummary.Cube;
-import org.apache.pinot.thirdeye.client.diffsummary.HierarchyNode;
-
 
 public class Summary {
   static final NodeDimensionValuesComparator NODE_COMPARATOR = new NodeDimensionValuesComparator();
@@ -48,6 +46,8 @@ public class Summary {
 
   private double globalBaselineValue;
   private double globalCurrentValue;
+  private double globalBaselineSize;
+  private double globalCurrentSize;
 
   private CostFunction costFunction;
   private RowInserter basicRowInserter;
@@ -61,6 +61,8 @@ public class Summary {
     this.maxLevelCount = cube.getDimensions().size();
     this.globalBaselineValue = cube.getBaselineTotal();
     this.globalCurrentValue = cube.getCurrentTotal();
+    this.globalBaselineSize = cube.getBaselineTotalSize();
+    this.globalCurrentSize = cube.getCurrentTotalSize();
     this.levelCount = this.maxLevelCount;
     this.costSet = cube.getCostSet();
     this.sortedDimensionCosts = cube.getSortedDimensionCosts();
@@ -97,16 +99,16 @@ public class Summary {
     for (int i = 0; i < this.levelCount; ++i) {
       dpArrays.add(new DPArray(answerSize));
     }
-    HierarchyNode root = cube.getRoot();
+    CubeNode root = cube.getRoot();
     if (doOneSideError) {
       oneSideErrorRowInserter =
-          new OneSideErrorRowInserter(basicRowInserter, Double.compare(1., root.targetRatio()) <= 0);
+          new OneSideErrorRowInserter(basicRowInserter, Double.compare(1., root.targetChangeRatio()) <= 0);
       // If this cube contains only one dimension, one side error is calculated starting at leaf (detailed) level;
       // otherwise, a row at different side is removed through internal nodes.
       if (this.levelCount == 1) leafRowInserter = oneSideErrorRowInserter;
     }
     computeChildDPArray(root);
-    List<HierarchyNode> answer = new ArrayList<>(dpArrays.get(0).getAnswer());
+    List<CubeNode> answer = new ArrayList<>(dpArrays.get(0).getAnswer());
     SummaryResponse response = new SummaryResponse();
     response.buildDiffSummary(answer, this.levelCount, costFunction);
     response.buildGainerLoserGroup(costSet);
@@ -120,13 +122,13 @@ public class Summary {
    * SummaryResponse is generated.
    */
   public void testCorrectnessOfWowValues() {
-    List<HierarchyNode> nodeList = new ArrayList<>(dpArrays.get(0).getAnswer());
-    Collections.sort(nodeList, NODE_COMPARATOR); // Process lower level nodes first
-    for (HierarchyNode node : nodeList) {
-      HierarchyNode parent = findAncestor(node, null, dpArrays.get(0).getAnswer());
+    List<CubeNode> nodeList = new ArrayList<>(dpArrays.get(0).getAnswer());
+    nodeList.sort(NODE_COMPARATOR); // Process lower level nodes first
+    for (CubeNode node : nodeList) {
+      CubeNode parent = findAncestor(node, null, dpArrays.get(0).getAnswer());
       if (parent != null) parent.addNodeValues(node);
     }
-    for (HierarchyNode node : nodeList) {
+    for (CubeNode node : nodeList) {
       if (Double.compare(node.getBaselineValue(), node.getOriginalBaselineValue()) != 0
           || Double.compare(node.getCurrentValue(), node.getOriginalCurrentValue()) != 0) {
         Log.warn("Wrong Wow values at node: " + node.getDimensionValues() + ". Expected: "
@@ -136,9 +138,9 @@ public class Summary {
     }
   }
 
-  static class NodeDimensionValuesComparator implements Comparator<HierarchyNode> {
+  static class NodeDimensionValuesComparator implements Comparator<CubeNode> {
     @Override
-    public int compare(HierarchyNode n1, HierarchyNode n2) {
+    public int compare(CubeNode n1, CubeNode n2) {
       return n1.getDimensionValues().compareTo(n2.getDimensionValues());
     }
   }
@@ -148,11 +150,11 @@ public class Summary {
    * The calculated answer for each invocation is put at dpArrays[node.level].
    * So, the final answer is located at dpArray[0].
    */
-  private void computeChildDPArray(HierarchyNode node) {
-    HierarchyNode parent = node.getParent();
+  private void computeChildDPArray(CubeNode node) {
+    CubeNode parent = node.getParent();
     DPArray dpArray = dpArrays.get(node.getLevel());
     dpArray.fullReset();
-    dpArray.targetRatio = node.targetRatio();
+    dpArray.targetRatio = node.targetChangeRatio();
 
     // Compute DPArray if the current node is the lowest internal node.
     // Otherwise, merge DPArrays from its children.
@@ -162,28 +164,28 @@ public class Summary {
 //      if (node.childrenSize() < dpArray.size()) {
 //        dpArray.setShrinkSize(Math.max(2, (node.childrenSize()+1)/2));
 //      }
-      for (HierarchyNode child : node.getChildren()) {
-        leafRowInserter.insertRowToDPArray(dpArray, child, node.targetRatio());
+      for (CubeNode child : (List<CubeNode>) node.getChildren()) {
+        leafRowInserter.insertRowToDPArray(dpArray, child, node.targetChangeRatio());
         updateWowValues(node, dpArray.getAnswer());
-        dpArray.targetRatio = node.targetRatio(); // get updated ratio
+        dpArray.targetRatio = node.targetChangeRatio(); // get updated changeRatio
       }
     } else {
-      for (HierarchyNode child : node.getChildren()) {
+      for (CubeNode child : (List<CubeNode>) node.getChildren()) {
         computeChildDPArray(child);
         mergeDPArray(node, dpArray, dpArrays.get(node.getLevel() + 1));
         updateWowValues(node, dpArray.getAnswer());
-        dpArray.targetRatio = node.targetRatio(); // get updated ratio
+        dpArray.targetRatio = node.targetChangeRatio(); // get updated changeRatio
       }
       // Use the following block to replace the above one to roll-up rows aggressively
-//      List<HierarchyNode> removedNodes = new ArrayList<>();
+//      List<CubeNode> removedNodes = new ArrayList<>();
 //      boolean doRollback = false;
 //      do {
 //        doRollback = false;
-//        for (HierarchyNode child : node.getChildren()) {
+//        for (CubeNode child : node.getChildren()) {
 //          computeChildDPArray(child);
 //          removedNodes.addAll(mergeDPArray(node, dpArray, dpArrays.get(node.getLevel() + 1)));
 //          updateWowValues(node, dpArray.getAnswer());
-//          dpArray.targetRatio = node.targetRatio(); // get updated ratio
+//          dpArray.targetChangeRatio = node.targetChangeRatio(); // get updated changeRatio
 //        }
 //        // Aggregate current node's answer if it is thinned out due to the user's answer size is too huge.
 //        // If the current node is kept being thinned out, it eventually aggregates all its children.
@@ -193,7 +195,7 @@ public class Summary {
 //          removedNodes.clear();
 //          dpArray.setShrinkSize(Math.max(1, (dpArray.getAnswer().size()*2)/3));
 //          dpArray.reset();
-//          dpArray.targetRatio = node.targetRatio();
+//          dpArray.targetChangeRatio = node.targetChangeRatio();
 //        }
 //      } while (doRollback);
     }
@@ -203,16 +205,20 @@ public class Summary {
     // Moreover, if a node is thinned out by its children, it won't be inserted to the answer.
     if (node.getLevel() != 0) {
       updateWowValues(parent, dpArray.getAnswer());
-      double targetRatio = parent.targetRatio();
+      double targetRatio = parent.targetChangeRatio();
       recomputeCostAndRemoveSmallNodes(node, dpArray, targetRatio);
       dpArray.targetRatio = targetRatio;
       if ( !nodeIsThinnedOut(node) ) {
         // dpArray actually takes (dpArray.size-1) nodes as the answer, so we set its size to 2
         // in order to insert the aggregated node to the answer.
         if (dpArray.size() == 1) dpArray.setShrinkSize(2);
-        Set<HierarchyNode> removedNode = new HashSet<>(dpArray.getAnswer());
+        Set<CubeNode> removedNode = new HashSet<>(dpArray.getAnswer());
         basicRowInserter.insertRowToDPArray(dpArray, node, targetRatio);
-        removedNode.removeAll(dpArray.getAnswer());
+        // The following block is trying to achieve removedNode.removeAll(dpArray.getAnswer());
+        // However, removeAll uses equalsTo() instead of equals() to determine if two objects are equal.
+        for (CubeNode cubeNode : dpArray.getAnswer()) {
+          removedNode.remove(cubeNode);
+        }
         if (removedNode.size() != 0) {
           updateWowValuesDueToRemoval(node, dpArray.getAnswer(), removedNode);
           updateWowValues(node, dpArray.getAnswer());
@@ -225,17 +231,17 @@ public class Summary {
 
   // TODO: Need a better definition for "a node is thinned out by its children."
   // We also need to look into the case where parent node is much smaller than its children.
-  private static boolean nodeIsThinnedOut(HierarchyNode node) {
-    return Double.compare(0., node.getBaselineValue()) == 0 && Double.compare(0., node.getCurrentValue()) == 0;
+  private static boolean nodeIsThinnedOut(CubeNode node) {
+    return Double.compare(0., node.getBaselineSize()) == 0 && Double.compare(0., node.getCurrentSize()) == 0;
   }
 
-  private static void rollbackInsertions(HierarchyNode node, Set<HierarchyNode> answer, List<HierarchyNode> removedNodes) {
-    Collections.sort(removedNodes, NODE_COMPARATOR); // Rollback from top to bottom nodes
+  private static void rollbackInsertions(CubeNode node, Set<CubeNode> answer, List<CubeNode> removedNodes) {
+    removedNodes.sort(NODE_COMPARATOR); // Rollback from top to bottom nodes
     Collections.reverse(removedNodes);
-    Set<HierarchyNode> targetSet = new HashSet<>(answer);
+    Set<CubeNode> targetSet = new HashSet<>(answer);
     targetSet.addAll(removedNodes);
-    for (HierarchyNode removedNode : removedNodes) {
-      HierarchyNode parents = findAncestor(removedNode, node, targetSet);
+    for (CubeNode removedNode : removedNodes) {
+      CubeNode parents = findAncestor(removedNode, node, targetSet);
       if (parents != null) parents.removeNodeValues(removedNode);
     }
     node.resetValues();
@@ -246,15 +252,15 @@ public class Summary {
    * After merging, the baseline and current values of the removed nodes (rows) will be add back to those of their
    * parent node.
    */
-  private Set<HierarchyNode> mergeDPArray(HierarchyNode parentNode, DPArray parentArray, DPArray childArray) {
-    Set<HierarchyNode> removedNodes = new HashSet<>(parentArray.getAnswer());
+  private Set<CubeNode> mergeDPArray(CubeNode parentNode, DPArray parentArray, DPArray childArray) {
+    Set<CubeNode> removedNodes = new HashSet<>(parentArray.getAnswer());
     removedNodes.addAll(childArray.getAnswer());
     // Compute the merged answer
     double targetRatio = (parentArray.targetRatio + childArray.targetRatio) / 2.;
     recomputeCostAndRemoveSmallNodes(parentNode, parentArray, targetRatio);
-    List<HierarchyNode> childNodeList = new ArrayList<>(childArray.getAnswer());
-    Collections.sort(childNodeList, NODE_COMPARATOR);
-    for (HierarchyNode childNode : childNodeList) {
+    List<CubeNode> childNodeList = new ArrayList<>(childArray.getAnswer());
+    childNodeList.sort(NODE_COMPARATOR);
+    for (CubeNode childNode : childNodeList) {
       insertRowWithAdaptiveRatio(parentArray, childNode, targetRatio);
     }
     // Update an internal node's baseline and current value if any of its child is removed due to the merge
@@ -267,9 +273,9 @@ public class Summary {
    * Recompute the baseline value and current value the node. The change is induced by the chosen nodes in
    * the answer. Note that the current node may be in the answer.
    */
-  private static void updateWowValues(HierarchyNode node, Set<HierarchyNode> answer) {
+  private static void updateWowValues(CubeNode node, Set<CubeNode> answer) {
     node.resetValues();
-    for (HierarchyNode child : answer) {
+    for (CubeNode child : answer) {
       if (child == node) continue;
       node.removeNodeValues(child);
     }
@@ -281,21 +287,21 @@ public class Summary {
    * @param answer The new answer.
    * @param removedNodes The nodes removed from the subtree of node.
    */
-  private static void updateWowValuesDueToRemoval(HierarchyNode node, Set<HierarchyNode> answer,
-      Set<HierarchyNode> removedNodes) {
-    List<HierarchyNode> removedNodesList = new ArrayList<>(removedNodes);
-    Collections.sort(removedNodesList, NODE_COMPARATOR); // Process lower level nodes first
-    for (HierarchyNode removedNode : removedNodesList) {
-      HierarchyNode parents = findAncestor(removedNode, node, answer);
+  private static void updateWowValuesDueToRemoval(CubeNode node, Set<CubeNode> answer,
+      Set<CubeNode> removedNodes) {
+    List<CubeNode> removedNodesList = new ArrayList<>(removedNodes);
+    removedNodesList.sort(NODE_COMPARATOR); // Process lower level nodes first
+    for (CubeNode removedNode : removedNodesList) {
+      CubeNode parents = findAncestor(removedNode, node, answer);
       if (parents != null) parents.addNodeValues(removedNode);
     }
   }
 
   /**
-   * Find a node's ancestor between the given node and ceiling that is contained in the target set of HierarchyNode.
+   * Find a node's ancestor between the given node and ceiling that is contained in the target set of CubeNode.
    * Returns null if no ancestor exists in the target set.
    */
-  private static HierarchyNode findAncestor(HierarchyNode node, HierarchyNode ceiling, Set<HierarchyNode> targets) {
+  private static CubeNode findAncestor(CubeNode node, CubeNode ceiling, Set<CubeNode> targets) {
     while ((node = node.getParent()) != ceiling) {
       if (targets.contains(node)) {
         return node;
@@ -305,14 +311,14 @@ public class Summary {
   }
 
   /**
-   * Recompute costs of the nodes in a DPArray using targetRatio for calculating the cost.
+   * Recompute costs of the nodes in a DPArray using targetChangeRatio for calculating the cost.
    */
-  private void recomputeCostAndRemoveSmallNodes(HierarchyNode parentNode, DPArray dp, double targetRatio) {
-    Set<HierarchyNode> removedNodes = new HashSet<>(dp.getAnswer());
-    List<HierarchyNode> ans = new ArrayList<>(dp.getAnswer());
-    Collections.sort(ans, NODE_COMPARATOR);
+  private void recomputeCostAndRemoveSmallNodes(CubeNode parentNode, DPArray dp, double targetRatio) {
+    Set<CubeNode> removedNodes = new HashSet<>(dp.getAnswer());
+    List<CubeNode> ans = new ArrayList<>(dp.getAnswer());
+    ans.sort(NODE_COMPARATOR);
     dp.reset();
-    for (HierarchyNode node : ans) {
+    for (CubeNode node : ans) {
       insertRowWithAdaptiveRatioNoOneSideError(dp, node, targetRatio);
     }
     removedNodes.removeAll(dp.getAnswer());
@@ -326,33 +332,33 @@ public class Summary {
   }
 
   /**
-   * If the node's parent is also in the DPArray, then it's parent's current ratio is used as the target ratio for
-   * calculating the cost of the node; otherwise, targetRatio is used.
+   * If the node's parent is also in the DPArray, then it's parent's current changeRatio is used as the target changeRatio for
+   * calculating the cost of the node; otherwise, targetChangeRatio is used.
    */
-  private void insertRowWithAdaptiveRatioNoOneSideError(DPArray dp, HierarchyNode node, double targetRatio) {
+  private void insertRowWithAdaptiveRatioNoOneSideError(DPArray dp, CubeNode node, double targetRatio) {
     if (dp.getAnswer().contains(node.getParent())) {
       // For one side error if node's parent is included in the solution, then its cost will be calculated normally.
-      basicRowInserter.insertRowToDPArray(dp, node, node.getParent().targetRatio());
+      basicRowInserter.insertRowToDPArray(dp, node, node.getParent().targetChangeRatio());
     } else {
       basicRowInserter.insertRowToDPArray(dp, node, targetRatio);
     }
   }
 
   /**
-   * If the node's parent is also in the DPArray, then it's parent's current ratio is used as the target ratio for
-   * calculating the cost of the node; otherwise, targetRatio is used.
+   * If the node's parent is also in the DPArray, then it's parent's current changeRatio is used as the target changeRatio for
+   * calculating the cost of the node; otherwise, targetChangeRatio is used.
    */
-  private void insertRowWithAdaptiveRatio(DPArray dp, HierarchyNode node, double targetRatio) {
+  private void insertRowWithAdaptiveRatio(DPArray dp, CubeNode node, double targetRatio) {
     if (dp.getAnswer().contains(node.getParent())) {
       // For one side error if node's parent is included in the solution, then its cost will be calculated normally.
-      basicRowInserter.insertRowToDPArray(dp, node, node.getParent().targetRatio());
+      basicRowInserter.insertRowToDPArray(dp, node, node.getParent().targetChangeRatio());
     } else {
       oneSideErrorRowInserter.insertRowToDPArray(dp, node, targetRatio);
     }
   }
 
   private interface RowInserter {
-    void insertRowToDPArray(DPArray dp, HierarchyNode node, double targetRatio);
+    void insertRowToDPArray(DPArray dp, CubeNode node, double targetRatio);
   }
 
   private class BasicRowInserter implements RowInserter {
@@ -363,11 +369,13 @@ public class Summary {
     }
 
     @Override
-    public void insertRowToDPArray(DPArray dp, HierarchyNode node, double targetRatio) {
+    public void insertRowToDPArray(DPArray dp, CubeNode node, double targetRatio) {
       double baselineValue = node.getBaselineValue();
       double currentValue = node.getCurrentValue();
-      double cost =
-          costFunction.computeCost(baselineValue, currentValue, targetRatio, globalBaselineValue, globalCurrentValue);
+      double baselineSize = node.getBaselineSize();
+      double currentSize = node.getCurrentSize();
+      double cost = costFunction.computeCost(targetRatio, baselineValue, currentValue, baselineSize, currentSize,
+          globalBaselineValue, globalCurrentValue, globalBaselineSize, globalCurrentSize);
 
       for (int n = dp.size() - 1; n > 0; --n) {
         double val1 = dp.slotAt(n - 1).cost;
@@ -397,16 +405,16 @@ public class Summary {
     }
 
     @Override
-    public void insertRowToDPArray(DPArray dp, HierarchyNode node, double targetRatio)  {
+    public void insertRowToDPArray(DPArray dp, CubeNode node, double targetRatio)  {
       // If the row has the same change trend with the top row, then it is inserted.
       if ( side == node.side() ) {
-        // When do oneSide, we try to make the root's ratio close to 1 in order to see the major root causes.
+        // When do oneSide, we try to make the root's changeRatio close to 1 in order to see the major root causes.
         if ( (side && Double.compare(targetRatio, 1d) > 0) || (!side && Double.compare(targetRatio, 1d) < 0)) {
           targetRatio = 1d;
         }
         basicRowInserter.insertRowToDPArray(dp, node, targetRatio);
       } else { // Otherwise, it is inserted only there exists an intermediate parent besides root node
-        HierarchyNode parent = findAncestor(node, null, dp.getAnswer());
+        CubeNode parent = findAncestor(node, null, dp.getAnswer());
         if (parent != null && parent.side() == side) basicRowInserter.insertRowToDPArray(dp, node, targetRatio);
       }
     }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/SummaryGainerLoserResponseRow.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/SummaryGainerLoserResponseRow.java
similarity index 95%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/SummaryGainerLoserResponseRow.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/SummaryGainerLoserResponseRow.java
index 6c40434..4ad79dd 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/SummaryGainerLoserResponseRow.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/SummaryGainerLoserResponseRow.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.dashboard.views.diffsummary;
+package org.apache.pinot.thirdeye.cube.summary;
 
 
 public class SummaryGainerLoserResponseRow extends BaseResponseRow {
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/SummaryResponse.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/SummaryResponse.java
similarity index 88%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/SummaryResponse.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/SummaryResponse.java
index ce6b3fe..70011d3 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/SummaryResponse.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/SummaryResponse.java
@@ -17,11 +17,11 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.dashboard.views.diffsummary;
+package org.apache.pinot.thirdeye.cube.summary;
 
-import org.apache.pinot.thirdeye.client.diffsummary.Cube;
-import org.apache.pinot.thirdeye.client.diffsummary.costfunctions.CostFunction;
-import org.apache.pinot.thirdeye.client.diffsummary.DimNameValueCostEntry;
+import org.apache.pinot.thirdeye.cube.data.cube.Cube;
+import org.apache.pinot.thirdeye.cube.cost.CostFunction;
+import org.apache.pinot.thirdeye.cube.data.cube.DimNameValueCostEntry;
 import java.text.DecimalFormat;
 import java.text.NumberFormat;
 import java.util.ArrayList;
@@ -30,9 +30,10 @@ import java.util.List;
 import java.util.Map;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.pinot.thirdeye.client.diffsummary.DimensionValues;
-import org.apache.pinot.thirdeye.client.diffsummary.Dimensions;
-import org.apache.pinot.thirdeye.client.diffsummary.HierarchyNode;
+import org.apache.pinot.thirdeye.cube.data.dbrow.DimensionValues;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.data.node.CubeNode;
+
 
 public class SummaryResponse {
   private final static int MAX_GAINER_LOSER_COUNT = 5;
@@ -55,6 +56,12 @@ public class SummaryResponse {
   @JsonProperty("currentTotal")
   private double currentTotal = 0d;
 
+  @JsonProperty("baselineTotalSize")
+  private double baselineTotalSize = 0d;
+
+  @JsonProperty("currentTotalSize")
+  private double currentTotalSize = 0d;
+
   @JsonProperty("globalRatio")
   private double globalRatio = 1d;
 
@@ -139,6 +146,8 @@ public class SummaryResponse {
     SummaryGainerLoserResponseRow row = new SummaryGainerLoserResponseRow();
     row.baselineValue = costEntry.getBaselineValue();
     row.currentValue = costEntry.getCurrentValue();
+    row.baselineSize = costEntry.getBaselineSize();
+    row.currentSize = costEntry.getCurrentSize();
     row.dimensionName = costEntry.getDimName();
     row.dimensionValue = costEntry.getDimValue();
     row.percentageChange = computePercentageChange(row.baselineValue, row.currentValue);
@@ -150,11 +159,14 @@ public class SummaryResponse {
     return row;
   }
 
-  public void buildDiffSummary(List<HierarchyNode> nodes, int targetLevelCount, CostFunction costFunction) {
+  public void buildDiffSummary(List<CubeNode> nodes, int targetLevelCount, CostFunction costFunction) {
     // Compute the total baseline and current value
-    for(HierarchyNode node : nodes) {
+
+    for(CubeNode node : nodes) {
       baselineTotal += node.getBaselineValue();
+      baselineTotalSize += node.getBaselineValue();
       currentTotal += node.getCurrentValue();
+      currentTotalSize += node.getCurrentValue();
     }
     if (Double.compare(baselineTotal, 0d) != 0) {
       globalRatio = roundUp(currentTotal / baselineTotal);
@@ -163,7 +175,7 @@ public class SummaryResponse {
     // If all nodes have a lower level count than targetLevelCount, then it is not necessary to print the summary with
     // height higher than the available level.
     int maxNodeLevelCount = 0;
-    for (HierarchyNode node : nodes) {
+    for (CubeNode node : nodes) {
       maxNodeLevelCount = Math.max(maxNodeLevelCount, node.getLevel());
     }
     targetLevelCount = Math.min(maxNodeLevelCount, targetLevelCount);
@@ -177,17 +189,17 @@ public class SummaryResponse {
     // Build the response
     nodes = SummaryResponseTree.sortResponseTree(nodes, targetLevelCount, costFunction);
     //   Build name tag for each row of responses
-    Map<HierarchyNode, NameTag> nameTags = new HashMap<>();
-    Map<HierarchyNode, List<String>> otherDimensionValues = new HashMap<>();
-    for (HierarchyNode node : nodes) {
+    Map<CubeNode, NameTag> nameTags = new HashMap<>();
+    Map<CubeNode, List<String>> otherDimensionValues = new HashMap<>();
+    for (CubeNode node : nodes) {
       NameTag tag = new NameTag(targetLevelCount);
       nameTags.put(node, tag);
       tag.copyNames(node.getDimensionValues());
       otherDimensionValues.put(node, new ArrayList<String>());
     }
     //   pre-condition: parent node is processed before its children nodes
-    for (HierarchyNode node : nodes) {
-      HierarchyNode parent = node;
+    for (CubeNode node : nodes) {
+      CubeNode parent = node;
       int levelDiff = 1;
       while ((parent = parent.getParent()) != null) {
         NameTag parentNameTag = nameTags.get(parent);
@@ -209,12 +221,14 @@ public class SummaryResponse {
       }
     }
     //    Fill in the information of each response row
-    for (HierarchyNode node : nodes) {
+    for (CubeNode node : nodes) {
       SummaryResponseRow row = new SummaryResponseRow();
       row.names = nameTags.get(node).names;
       row.baselineValue = node.getBaselineValue();
       row.currentValue = node.getCurrentValue();
       row.percentageChange = computePercentageChange(row.baselineValue, row.currentValue);
+      row.baselineSize = node.getBaselineSize();
+      row.currentSize = node.getCurrentSize();
       row.contributionChange =
           computeContributionChange(row.baselineValue, row.currentValue, baselineTotal, currentTotal);
       row.contributionToOverallChange =
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/SummaryResponseRow.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/SummaryResponseRow.java
similarity index 96%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/SummaryResponseRow.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/SummaryResponseRow.java
index 0355c55..b6e4343 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/SummaryResponseRow.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/SummaryResponseRow.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.dashboard.views.diffsummary;
+package org.apache.pinot.thirdeye.cube.summary;
 
 import java.util.ArrayList;
 import java.util.List;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/SummaryResponseTree.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/SummaryResponseTree.java
similarity index 66%
rename from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/SummaryResponseTree.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/SummaryResponseTree.java
index 685c023..0208ff9 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/views/diffsummary/SummaryResponseTree.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/summary/SummaryResponseTree.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.pinot.thirdeye.dashboard.views.diffsummary;
+package org.apache.pinot.thirdeye.cube.summary;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -27,31 +27,35 @@ import java.util.List;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.pinot.thirdeye.client.diffsummary.costfunctions.CostFunction;
-import org.apache.pinot.thirdeye.client.diffsummary.Dimensions;
-import org.apache.pinot.thirdeye.client.diffsummary.HierarchyNode;
+import org.apache.pinot.thirdeye.cube.cost.CostFunction;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.data.node.CubeNode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class SummaryResponseTree {
   private static final Logger LOG = LoggerFactory.getLogger(SummaryResponseTree.class);
-  @JsonProperty("dimensions")
-  List<String> dimensions = new ArrayList<>();
 
-  List<HierarchyNode> hierarchicalNodes = new ArrayList<>();
+  @JsonProperty("dimensions")
+  private List<String> dimensions = new ArrayList<>();
+  private List<CubeNode> hierarchicalNodes = new ArrayList<>();
 
 
-  public static List<HierarchyNode> sortResponseTree(List<HierarchyNode> nodes, int levelCount, CostFunction costFunction) {
+  public static List<CubeNode> sortResponseTree(List<CubeNode> nodes, int levelCount, CostFunction costFunction) {
     SummaryResponseTree responseTree = new SummaryResponseTree();
 
     // Build the header
     Dimensions dimensions = nodes.get(0).getDimensions();
-    double globalBaselineValue = nodes.get(0).getOriginalCurrentValue();
-    double globalCurrentValue = nodes.get(0).getOriginalBaselineValue();
-    for (HierarchyNode node : nodes) {
+    double topBaselineValue = nodes.get(0).getOriginalBaselineValue();
+    double topCurrentValue = nodes.get(0).getOriginalCurrentValue();
+    double topBaselineSize = nodes.get(0).getOriginalBaselineSize();
+    double topCurrentSize = nodes.get(0).getOriginalCurrentSize();
+    for (CubeNode node : nodes) {
       if (node.getLevel() == 0) {
-        globalBaselineValue = node.getOriginalBaselineValue();
-        globalCurrentValue = node.getOriginalCurrentValue();
+        topBaselineValue = node.getOriginalBaselineValue();
+        topCurrentValue = node.getOriginalCurrentValue();
+        topBaselineSize = node.getOriginalBaselineSize();
+        topCurrentSize = node.getOriginalCurrentSize();
         break;
       }
     }
@@ -61,10 +65,10 @@ public class SummaryResponseTree {
 
     List<SummaryResponseTreeNode> treeNodes = new ArrayList<>();
     // Build the response tree
-    Collections.sort(nodes, Collections.reverseOrder(Summary.NODE_COMPARATOR)); // pre-order traversal
-    for (HierarchyNode node : nodes) {
+    nodes.sort(Collections.reverseOrder(Summary.NODE_COMPARATOR)); // pre-order traversal
+    for (CubeNode node : nodes) {
       SummaryResponseTreeNode treeNode = new SummaryResponseTreeNode();
-      treeNode.hierarchyNode = node;
+      treeNode.cubeNode = node;
       treeNode.level = node.getLevel();
       treeNodes.add(treeNode);
     }
@@ -74,7 +78,7 @@ public class SummaryResponseTree {
     //    In that case, we have to bootstrap the search until a higher level parent, which also exists in the response
     //    tree, is found.
     //    Pre-condition: treeNodes are sorted in the pre-order fashion when projecting the nodes back to the tree of
-    //                   HierarchyNode.
+    //                   CubeNode.
     SummaryResponseTreeNode preTreeNode = null;
     for (SummaryResponseTreeNode treeNode : treeNodes) {
       if (preTreeNode != null) {
@@ -86,7 +90,7 @@ public class SummaryResponseTree {
     }
 
     // Sort the children of each node by their cost
-    sortChildNodes(treeNodes.get(0), globalBaselineValue, globalCurrentValue, costFunction);
+    sortChildNodes(treeNodes.get(0), topBaselineValue, topCurrentValue, topBaselineSize, topCurrentSize, costFunction);
 
     // Put the nodes to a flattened array
     insertChildNodes(treeNodes.get(0), responseTree.hierarchicalNodes);
@@ -94,9 +98,9 @@ public class SummaryResponseTree {
     return responseTree.hierarchicalNodes;
   }
 
-  private static void insertChildNodes(SummaryResponseTreeNode node, List<HierarchyNode> hierarchicalNodes) {
-    if (node.hierarchyNode != null) {
-      hierarchicalNodes.add(node.hierarchyNode);
+  private static void insertChildNodes(SummaryResponseTreeNode node, List<CubeNode> hierarchicalNodes) {
+    if (node.cubeNode != null) {
+      hierarchicalNodes.add(node.cubeNode);
     }
     for (SummaryResponseTreeNode child : node.children) {
       insertChildNodes(child, hierarchicalNodes);
@@ -106,30 +110,30 @@ public class SummaryResponseTree {
   /**
    * A recursive function to sort response tree.
    */
-  private static void sortChildNodes(SummaryResponseTreeNode node, double globalBaselineValue,
-      double globalCurrentValue, CostFunction costFunction) {
+  private static void sortChildNodes(SummaryResponseTreeNode node, double topBaselineValue,
+      double topCurrentValue, double topBaselineSize, double topCurrentSize, CostFunction costFunction) {
     if (node.children.size() == 0) return;
     for (SummaryResponseTreeNode child : node.children) {
-      sortChildNodes(child, globalBaselineValue, globalCurrentValue, costFunction);
+      sortChildNodes(child, topBaselineValue, topCurrentValue, topBaselineSize, topCurrentSize, costFunction);
     }
-    double ratio = node.currentRatio();
+    double ratio = node.currentChangeRatio();
     for (SummaryResponseTreeNode child : node.children) {
-      computeCost(child, ratio, globalBaselineValue, globalCurrentValue, costFunction);
+      computeCost(child, ratio, topBaselineValue, topCurrentValue, topBaselineSize, topCurrentSize, costFunction);
     }
-    Collections.sort(node.children, Collections.reverseOrder(new SummaryResponseTreeNodeCostComparator()));
+    node.children.sort(Collections.reverseOrder(new SummaryResponseTreeNodeCostComparator()));
   }
 
-  private static void computeCost(SummaryResponseTreeNode node, double targetRatio, double globalBaselineValue,
-      double globalCurrentValue, CostFunction costFunction) {
-    if (node.hierarchyNode != null) {
-      double nodeCost = costFunction
-          .computeCost(node.getBaselineValue(), node.getCurrentValue(), targetRatio, globalBaselineValue,
-              globalCurrentValue);
-      node.hierarchyNode.setCost(nodeCost);
+  private static void computeCost(SummaryResponseTreeNode node, double targetChangeRatio, double topBaselineValue,
+      double topCurrentValue, double topBaselineSize, double topCurrentSize, CostFunction costFunction) {
+    if (node.cubeNode != null) {
+      double nodeCost = costFunction.computeCost(targetChangeRatio, node.getBaselineValue(), node.getCurrentValue(),
+          node.getBaselineSize(), node.getCurrentSize(), topBaselineValue, topCurrentValue, topBaselineSize,
+          topCurrentSize);
+      node.cubeNode.setCost(nodeCost);
       node.subTreeCost = nodeCost;
     }
     for (SummaryResponseTreeNode child : node.children) {
-      computeCost(child, targetRatio, globalBaselineValue, globalCurrentValue, costFunction);
+      computeCost(child, targetChangeRatio, topBaselineValue, topCurrentValue, topBaselineSize, topCurrentSize, costFunction);
       node.subTreeCost += child.subTreeCost;
     }
   }
@@ -142,7 +146,7 @@ public class SummaryResponseTree {
   }
 
   public static class SummaryResponseTreeNode {
-    HierarchyNode hierarchyNode; // If it is null, this node is a dummy node.
+    CubeNode cubeNode; // If it is null, this node is a dummy node.
     double subTreeCost;
     int level;
 
@@ -152,7 +156,7 @@ public class SummaryResponseTree {
     List<SummaryResponseTreeNode> children = new ArrayList<>();
 
     public List<String> getDimensionValues() {
-      return hierarchyNode.getDimensionValues().values();
+      return cubeNode.getDimensionValues().values();
     }
 
     public void setLevel(int level) {
@@ -164,26 +168,34 @@ public class SummaryResponseTree {
     }
 
     public double getBaselineValue() {
-      return hierarchyNode.getBaselineValue();
+      return cubeNode.getBaselineValue();
     }
 
     public double getCurrentValue() {
-      return hierarchyNode.getCurrentValue();
+      return cubeNode.getCurrentValue();
+    }
+
+    public double getBaselineSize() {
+      return cubeNode.getBaselineSize();
+    }
+
+    public double getCurrentSize() {
+      return cubeNode.getCurrentSize();
     }
 
-    public double currentRatio() {
-      if (hierarchyNode != null) {
-        return hierarchyNode.currentRatio();
+    public double currentChangeRatio() {
+      if (cubeNode != null) {
+        return cubeNode.changeRatio();
       } else {
         SummaryResponseTreeNode parent = this;
         do {
-          if (parent.hierarchyNode == null) {
+          if (parent.cubeNode == null) {
             parent = parent.parent;
           } else {
             break;
           }
         } while (true);
-        return parent.currentRatio();
+        return parent.currentChangeRatio();
       }
     }
 
@@ -229,9 +241,9 @@ public class SummaryResponseTree {
     }
 
     private void swapContent(SummaryResponseTreeNode A, SummaryResponseTreeNode B) {
-      HierarchyNode tmpNode = A.hierarchyNode;
-      A.hierarchyNode = B.hierarchyNode;
-      B.hierarchyNode = tmpNode;
+      CubeNode tmpNode = A.cubeNode;
+      A.cubeNode = B.cubeNode;
+      B.cubeNode = tmpNode;
       List<SummaryResponseTreeNode> tmpChildren = A.children;
       A.children = B.children;
       B.children = tmpChildren;
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 0c1b675..0f590cc 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
@@ -24,17 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Strings;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
-import org.apache.pinot.thirdeye.client.diffsummary.Dimensions;
-import org.apache.pinot.thirdeye.client.diffsummary.MultiDimensionalSummary;
-import org.apache.pinot.thirdeye.client.diffsummary.MultiDimensionalSummaryCLITool;
-import org.apache.pinot.thirdeye.client.diffsummary.OLAPDataBaseClient;
-import org.apache.pinot.thirdeye.client.diffsummary.ThirdEyeSummaryClient;
-import org.apache.pinot.thirdeye.client.diffsummary.costfunctions.BalancedCostFunction;
-import org.apache.pinot.thirdeye.client.diffsummary.costfunctions.CostFunction;
-import org.apache.pinot.thirdeye.dashboard.Utils;
-import org.apache.pinot.thirdeye.dashboard.views.diffsummary.SummaryResponse;
-import org.apache.pinot.thirdeye.datasource.ThirdEyeCacheRegistry;
-import org.apache.pinot.thirdeye.util.ThirdEyeUtils;
 import java.net.URLDecoder;
 import java.util.Arrays;
 import java.util.Collections;
@@ -46,6 +35,18 @@ import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.MediaType;
 import org.apache.commons.lang.StringUtils;
+import org.apache.pinot.thirdeye.cube.additive.AdditiveDBClient;
+import org.apache.pinot.thirdeye.cube.additive.AdditiveRow;
+import org.apache.pinot.thirdeye.cube.additive.MultiDimensionalSummary;
+import org.apache.pinot.thirdeye.cube.additive.MultiDimensionalSummaryCLITool;
+import org.apache.pinot.thirdeye.cube.cost.BalancedCostFunction;
+import org.apache.pinot.thirdeye.cube.cost.CostFunction;
+import org.apache.pinot.thirdeye.cube.data.dbclient.CubePinotClient;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.summary.SummaryResponse;
+import org.apache.pinot.thirdeye.dashboard.Utils;
+import org.apache.pinot.thirdeye.datasource.ThirdEyeCacheRegistry;
+import org.apache.pinot.thirdeye.util.ThirdEyeUtils;
 import org.joda.time.DateTimeZone;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -113,8 +114,8 @@ public class SummaryResource {
 
       CostFunction costFunction = new BalancedCostFunction();
       DateTimeZone dateTimeZone = DateTimeZone.forID(timeZone);
-      OLAPDataBaseClient olapClient = new ThirdEyeSummaryClient(CACHE_REGISTRY_INSTANCE.getQueryCache());
-      MultiDimensionalSummary mdSummary = new MultiDimensionalSummary(olapClient, costFunction, dateTimeZone);
+      CubePinotClient<AdditiveRow> cubeDbClient = new AdditiveDBClient(CACHE_REGISTRY_INSTANCE.getQueryCache());
+      MultiDimensionalSummary mdSummary = new MultiDimensionalSummary(cubeDbClient, costFunction, dateTimeZone);
 
       response = mdSummary
           .buildSummary(dataset, metric, currentStartInclusive, currentEndExclusive, baselineStartInclusive,
@@ -166,8 +167,8 @@ public class SummaryResource {
 
       CostFunction costFunction = new BalancedCostFunction();
       DateTimeZone dateTimeZone = DateTimeZone.forID(timeZone);
-      OLAPDataBaseClient olapClient = new ThirdEyeSummaryClient(CACHE_REGISTRY_INSTANCE.getQueryCache());
-      MultiDimensionalSummary mdSummary = new MultiDimensionalSummary(olapClient, costFunction, dateTimeZone);
+      CubePinotClient<AdditiveRow> cubeDbClient = new AdditiveDBClient(CACHE_REGISTRY_INSTANCE.getQueryCache());
+      MultiDimensionalSummary mdSummary = new MultiDimensionalSummary(cubeDbClient, costFunction, dateTimeZone);
 
       response = mdSummary
           .buildSummary(dataset, metric, currentStartInclusive, currentEndExclusive, baselineStartInclusive,
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/rootcause/impl/MetricComponentAnalysisPipeline.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/rootcause/impl/MetricComponentAnalysisPipeline.java
index 5569cb6..951e312 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/rootcause/impl/MetricComponentAnalysisPipeline.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/rootcause/impl/MetricComponentAnalysisPipeline.java
@@ -21,6 +21,7 @@ package org.apache.pinot.thirdeye.rootcause.impl;
 
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
+import org.apache.pinot.thirdeye.cube.data.cube.Cube;
 import org.apache.pinot.thirdeye.dataframe.DataFrame;
 import org.apache.pinot.thirdeye.dataframe.DoubleSeries;
 import org.apache.pinot.thirdeye.dataframe.Series;
@@ -66,7 +67,7 @@ import org.slf4j.LoggerFactory;
  * slices from the dataset. The result is an ordered list of the top k slices with the biggest
  * relative change.
  *
- * @see org.apache.pinot.thirdeye.client.diffsummary.Cube
+ * @see Cube
  */
 public class MetricComponentAnalysisPipeline extends Pipeline {
   private static final Logger LOG = LoggerFactory.getLogger(MetricComponentAnalysisPipeline.class);
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/HierarchyNodeTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/HierarchyNodeTest.java
deleted file mode 100644
index 75e7b8c..0000000
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/HierarchyNodeTest.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/**
- * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
- *
- * Licensed 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.client.diffsummary;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-
-public class HierarchyNodeTest {
-
-  // Since HierarchyNode has cyclic reference between current node and parent node, the toString() will encounter
-  // overflowStack exception if it doesn't take care of the cyclic reference carefully.
-  @Test
-  public void testToString() {
-    Row root = new Row(new Dimensions(), new DimensionValues());
-    HierarchyNode rootNode = new HierarchyNode(root);
-
-    Row child = new Row(new Dimensions(Collections.singletonList("country")),
-        new DimensionValues(Collections.singletonList("US")), 20, 30);
-    HierarchyNode childNode = new HierarchyNode(1, 0, child, rootNode);
-
-    childNode.toString();
-  }
-
-  @Test
-  public void testSimpleEquals() throws Exception {
-    Row root1 = new Row(new Dimensions(), new DimensionValues());
-    HierarchyNode rootNode1 = new HierarchyNode(root1);
-
-    Row root2 = new Row(new Dimensions(), new DimensionValues());
-    HierarchyNode rootNode2 = new HierarchyNode(root2);
-
-    Assert.assertEquals(rootNode1, rootNode2);
-    Assert.assertTrue(HierarchyNode.equalHierarchy(rootNode1, rootNode2));
-
-    Row root3 = new Row(new Dimensions(Collections.singletonList("country")),
-        new DimensionValues(Collections.singletonList("US")));
-    HierarchyNode rootNode3 = new HierarchyNode(root3);
-    Assert.assertNotEquals(rootNode1, rootNode3);
-  }
-
-  @Test
-  public void testHierarchicalEquals() throws Exception {
-    HierarchyNode rootNode1 = buildHierarchicalNodes();
-    HierarchyNode rootNode2 = buildHierarchicalNodes();
-
-    Assert.assertTrue(HierarchyNode.equalHierarchy(rootNode1, rootNode2));
-  }
-
-  /**
-   * Hierarchy 1:
-   *      A
-   *     / \
-   *    B  C
-   *
-   * Hierarchy 2:
-   *      A
-   *
-   * Failed because structure difference.
-   */
-  @Test
-  public void testHierarchicalEqualsFail1() {
-    HierarchyNode rootNode1 = buildHierarchicalNodes();
-
-    Row rootRow = new Row(new Dimensions(), new DimensionValues(), 30, 45);
-    HierarchyNode rootNode2 = new HierarchyNode(rootRow);
-
-    Assert.assertEquals(rootNode1, rootNode2);
-    Assert.assertFalse(HierarchyNode.equalHierarchy(rootNode1, rootNode2));
-  }
-
-  /**
-   * Hierarchy 1:
-   *      A
-   *     / \
-   *    B  C
-   *
-   * Hierarchy 2:
-   *      A'
-   *
-   * Failed because data difference.
-   */
-  @Test
-  public void testHierarchicalEqualsFail2() throws Exception {
-    HierarchyNode rootNode1 = buildHierarchicalNodes();
-
-    Row rootRow = new Row(new Dimensions(), new DimensionValues(), 20, 15);
-    HierarchyNode rootNode2 = new HierarchyNode(rootRow);
-
-    Assert.assertNotEquals(rootNode1, rootNode2);
-    Assert.assertFalse(HierarchyNode.equalHierarchy(rootNode1, rootNode2));
-  }
-
-  /**
-   * Hierarchy 1:
-   *      A
-   *     ^ ^
-   *     / \
-   *    v  v
-   *    B  C
-   *
-   * Hierarchy 2:
-   *      A
-   *      ^
-   *       \
-   *       v
-   *       C
-   *
-   * Failed because Hierarchy 2's A doesn't have a reference to B.
-   */
-  @Test
-  public void testHierarchicalEqualsFail3() throws Exception {
-    HierarchyNode rootNode1 = buildHierarchicalNodes();
-
-    List<List<Row>> rows = buildHierarchicalRows();
-    // Root level
-    Row rootRow = rows.get(0).get(0);
-    HierarchyNode rootNode2 = new HierarchyNode(rootRow);
-
-    // Level 1
-    Row INRow = rows.get(1).get(1);
-    HierarchyNode INNode = new HierarchyNode(1, 1, INRow, rootNode2);
-
-    Assert.assertEquals(rootNode1, rootNode2);
-    Assert.assertFalse(HierarchyNode.equalHierarchy(rootNode1, rootNode2));
-  }
-
-  private List<List<Row>> buildHierarchicalRows() {
-    List<List<Row>> hierarchicalRows = new ArrayList<>();
-    // Root level
-    List<Row> rootLevel = new ArrayList<>();
-    rootLevel.add(new Row(new Dimensions(), new DimensionValues(), 30, 45));
-    hierarchicalRows.add(rootLevel);
-
-    // Level 1
-    List<Row> level1 = new ArrayList<>();
-    Row row1 = new Row(new Dimensions(Collections.singletonList("country")),
-        new DimensionValues(Collections.singletonList("US")), 20, 30);
-    level1.add(row1);
-
-    Row row2 = new Row(new Dimensions(Collections.singletonList("country")),
-        new DimensionValues(Collections.singletonList("IN")), 10, 15);
-    level1.add(row2);
-
-    hierarchicalRows.add(level1);
-
-    return hierarchicalRows;
-  }
-
-  private HierarchyNode buildHierarchicalNodes() {
-    List<List<Row>> rows = buildHierarchicalRows();
-    // Root level
-    Row rootRow = rows.get(0).get(0);
-    HierarchyNode rootNode = new HierarchyNode(rootRow);
-
-    // Level 1
-    Row USRow = rows.get(1).get(0);
-    HierarchyNode USNode = new HierarchyNode(1, 0, USRow, rootNode);
-
-    Row INRow = rows.get(1).get(1);
-    HierarchyNode INNode = new HierarchyNode(1, 1, INRow, rootNode);
-
-    return rootNode;
-  }
-}
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/MultiDimensionalSummaryCLIToolTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/additive/MultiDimensionalSummaryCLIToolTest.java
similarity index 92%
rename from thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/MultiDimensionalSummaryCLIToolTest.java
rename to thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/additive/MultiDimensionalSummaryCLIToolTest.java
index 502fc28..f6e5540 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/MultiDimensionalSummaryCLIToolTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/additive/MultiDimensionalSummaryCLIToolTest.java
@@ -14,9 +14,10 @@
  * limitations under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
+package org.apache.pinot.thirdeye.cube.additive;
 
-import org.apache.pinot.thirdeye.client.diffsummary.costfunctions.BalancedCostFunction;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.cost.BalancedCostFunction;
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.util.Arrays;
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/BalancedCostFunctionTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/cost/BalancedCostFunctionTest.java
similarity index 86%
rename from thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/BalancedCostFunctionTest.java
rename to thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/cost/BalancedCostFunctionTest.java
index 231d513..5fe0617 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/costfunctions/BalancedCostFunctionTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/cost/BalancedCostFunctionTest.java
@@ -14,14 +14,15 @@
  * limitations under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary.costfunctions;
+package org.apache.pinot.thirdeye.cube.cost;
 
 import java.util.HashMap;
 import java.util.Map;
+import org.apache.pinot.thirdeye.cube.cost.BalancedCostFunction;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-import static org.apache.pinot.thirdeye.client.diffsummary.costfunctions.BalancedCostFunction.*;
+import static org.apache.pinot.thirdeye.cube.cost.BalancedCostFunction.*;
 
 
 public class BalancedCostFunctionTest {
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/CubeTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/cube/CubeTest.java
similarity index 69%
rename from thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/CubeTest.java
rename to thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/cube/CubeTest.java
index 5e3a73c..c4ab8bb 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/CubeTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/cube/CubeTest.java
@@ -14,12 +14,19 @@
  * limitations under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
+package org.apache.pinot.thirdeye.cube.data.cube;
 
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import org.apache.pinot.thirdeye.cube.data.dbrow.DimensionValues;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.additive.AdditiveCubeNode;
+import org.apache.pinot.thirdeye.cube.additive.AdditiveRow;
+import org.apache.pinot.thirdeye.cube.data.node.CubeNode;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Row;
+import org.apache.pinot.thirdeye.cube.data.node.CubeNodeUtils;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -84,29 +91,29 @@ public class CubeTest {
 
   private List<DimNameValueCostEntry> getBasicCostSet() {
     List<DimNameValueCostEntry> costSet = new ArrayList<>();
-    costSet.add(new DimNameValueCostEntry("country", "US", 0, 0, 0, 7));
-    costSet.add(new DimNameValueCostEntry("country", "IN", 0, 0, 0, 3));
-    costSet.add(new DimNameValueCostEntry("continent", "N. America", 0, 0, 0, 4));
-    costSet.add(new DimNameValueCostEntry("continent", "S. America", 0, 0, 0, 1));
-    costSet.add(new DimNameValueCostEntry("page", "front_page", 0, 0, 0, 4));
-    costSet.add(new DimNameValueCostEntry("page", "page", 0, 0, 0, 3));
-    costSet.add(new DimNameValueCostEntry("page", "page2", 0, 0, 0, 1));
+    costSet.add(new DimNameValueCostEntry("country", "US", 0, 0, 0, 0, 0, 7));
+    costSet.add(new DimNameValueCostEntry("country", "IN", 0, 0, 0, 0,  0, 3));
+    costSet.add(new DimNameValueCostEntry("continent", "N. America", 0, 0, 0,  0, 0, 4));
+    costSet.add(new DimNameValueCostEntry("continent", "S. America", 0, 0, 0, 0, 0,  1));
+    costSet.add(new DimNameValueCostEntry("page", "front_page", 0, 0, 0, 0,  0, 4));
+    costSet.add(new DimNameValueCostEntry("page", "page", 0, 0, 0, 0,  0, 3));
+    costSet.add(new DimNameValueCostEntry("page", "page2", 0, 0, 0, 0,  0, 1));
     return costSet;
   }
 
   @Test
   public void testHierarchyRowToHierarchyNode() {
     List<List<Row>> rows = buildHierarchicalRows();
-    List<List<HierarchyNode>> actualNodes =
-        Cube.hierarchyRowToHierarchyNode(rows, new Dimensions(Arrays.asList("country", "page")));
+    List<List<CubeNode>> actualNodes =
+        Cube.dataRowToCubeNode(rows, new Dimensions(Arrays.asList("country", "page")));
 
-    List<List<HierarchyNode>> expectedNodes = expectedHierarchicalNodes();
+    List<List<CubeNode>> expectedNodes = expectedHierarchicalNodes();
 
     // Test if the data is current; the reference (i.e., tree structure is not tested)
     Assert.assertEquals(actualNodes, expectedNodes);
 
     // Test the structure of the hierarchy
-    Assert.assertTrue(HierarchyNode.equalHierarchy(actualNodes.get(0).get(0), expectedNodes.get(0).get(0)));
+    Assert.assertTrue(CubeNodeUtils.equalHierarchy(actualNodes.get(0).get(0), expectedNodes.get(0).get(0)));
   }
 
   private List<List<Row>> buildHierarchicalRows() {
@@ -114,17 +121,17 @@ public class CubeTest {
     // Root level
     {
       List<Row> rootLevel = new ArrayList<>();
-      rootLevel.add(new Row(new Dimensions(), new DimensionValues(), 30, 45));
+      rootLevel.add(new AdditiveRow(new Dimensions(), new DimensionValues(), 30, 45));
       hierarchicalRows.add(rootLevel);
     }
     // Level 1
     {
       List<Row> level1 = new ArrayList<>();
-      Row row1 = new Row(new Dimensions(Collections.singletonList("country")),
+      Row row1 = new AdditiveRow(new Dimensions(Collections.singletonList("country")),
           new DimensionValues(Collections.singletonList("US")), 20, 30);
       level1.add(row1);
 
-      Row row2 = new Row(new Dimensions(Collections.singletonList("country")),
+      Row row2 = new AdditiveRow(new Dimensions(Collections.singletonList("country")),
           new DimensionValues(Collections.singletonList("IN")), 10, 15);
       level1.add(row2);
 
@@ -134,17 +141,17 @@ public class CubeTest {
     {
       List<Row> level2 = new ArrayList<>();
       Row row1 =
-          new Row(new Dimensions(Arrays.asList("country", "page")), new DimensionValues(Arrays.asList("US", "page1")),
+          new AdditiveRow(new Dimensions(Arrays.asList("country", "page")), new DimensionValues(Arrays.asList("US", "page1")),
               8, 10);
       level2.add(row1);
 
       Row row2 =
-          new Row(new Dimensions(Arrays.asList("country", "page")), new DimensionValues(Arrays.asList("US", "page2")),
+          new AdditiveRow(new Dimensions(Arrays.asList("country", "page")), new DimensionValues(Arrays.asList("US", "page2")),
               12, 20);
       level2.add(row2);
 
       Row row3 =
-          new Row(new Dimensions(Arrays.asList("country", "page")), new DimensionValues(Arrays.asList("IN", "page1")),
+          new AdditiveRow(new Dimensions(Arrays.asList("country", "page")), new DimensionValues(Arrays.asList("IN", "page1")),
               10, 15);
       level2.add(row3);
 
@@ -153,43 +160,43 @@ public class CubeTest {
     return hierarchicalRows;
   }
 
-  private List<List<HierarchyNode>> expectedHierarchicalNodes() {
+  private List<List<CubeNode>> expectedHierarchicalNodes() {
     List<List<Row>> rows = buildHierarchicalRows();
-    List<List<HierarchyNode>> hierarchicalNodes = new ArrayList<>();
+    List<List<CubeNode>> hierarchicalNodes = new ArrayList<>();
     // Root level
-    List<HierarchyNode> rootLevel = new ArrayList<>();
+    List<CubeNode> rootLevel = new ArrayList<>();
     hierarchicalNodes.add(rootLevel);
 
     Row rootRow = rows.get(0).get(0);
-    HierarchyNode rootNode = new HierarchyNode(rootRow);
+    AdditiveCubeNode rootNode = new AdditiveCubeNode((AdditiveRow) rootRow);
     rootLevel.add(rootNode);
 
     // Level 1
-    List<HierarchyNode> level1 = new ArrayList<>();
+    List<CubeNode> level1 = new ArrayList<>();
     hierarchicalNodes.add(level1);
 
-    Row USRow = rows.get(1).get(0);
-    HierarchyNode USNode = new HierarchyNode(1, 0, USRow, rootNode);
+    AdditiveRow USRow = (AdditiveRow) rows.get(1).get(0);
+    AdditiveCubeNode USNode = new AdditiveCubeNode(1, 0, USRow, rootNode);
     level1.add(USNode);
 
-    Row INRow = rows.get(1).get(1);
-    HierarchyNode INNode = new HierarchyNode(1, 1, INRow, rootNode);
+    AdditiveRow INRow = (AdditiveRow) rows.get(1).get(1);
+    AdditiveCubeNode INNode = new AdditiveCubeNode(1, 1, INRow, rootNode);
     level1.add(INNode);
 
     // Level 2
-    List<HierarchyNode> level2 = new ArrayList<>();
+    List<CubeNode> level2 = new ArrayList<>();
     hierarchicalNodes.add(level2);
 
-    Row USPage1Row = rows.get(2).get(0);
-    HierarchyNode USPage1Node = new HierarchyNode(2, 0, USPage1Row, USNode);
+    AdditiveRow USPage1Row = (AdditiveRow) rows.get(2).get(0);
+    CubeNode USPage1Node = new AdditiveCubeNode(2, 0, USPage1Row, USNode);
     level2.add(USPage1Node);
 
-    Row USPage2Row = rows.get(2).get(1);
-    HierarchyNode USPage2Node = new HierarchyNode(2, 1, USPage2Row, USNode);
+    AdditiveRow USPage2Row = (AdditiveRow) rows.get(2).get(1);
+    CubeNode USPage2Node = new AdditiveCubeNode(2, 1, USPage2Row, USNode);
     level2.add(USPage2Node);
 
-    Row INPage1Row = rows.get(2).get(2);
-    HierarchyNode INPage1Node = new HierarchyNode(2, 2, INPage1Row, INNode);
+    AdditiveRow INPage1Row = (AdditiveRow) rows.get(2).get(2);
+    CubeNode INPage1Node = new AdditiveCubeNode(2, 2, INPage1Row, INNode);
     level2.add(INPage1Node);
 
     return hierarchicalNodes;
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/DimNameValueCostEntryTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/cube/DimNameValueCostEntryTest.java
similarity index 76%
rename from thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/DimNameValueCostEntryTest.java
rename to thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/cube/DimNameValueCostEntryTest.java
index 6be7f56..eac33ce 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/DimNameValueCostEntryTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/cube/DimNameValueCostEntryTest.java
@@ -14,24 +14,25 @@
  * limitations under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
+package org.apache.pinot.thirdeye.cube.data.cube;
 
+import org.apache.pinot.thirdeye.cube.data.cube.DimNameValueCostEntry;
 import org.testng.annotations.Test;
 
 public class DimNameValueCostEntryTest {
 
   @Test
   public void testCreation() {
-    new DimNameValueCostEntry("", "", 0, 0, 0, 0);
+    new DimNameValueCostEntry("", "", 0, 0, 0, 0, 0, 0);
   }
 
   @Test(expectedExceptions = NullPointerException.class)
   public void testNullDimensionNameCreation() {
-    new DimNameValueCostEntry(null, "", 0, 0, 0, 0);
+    new DimNameValueCostEntry(null, "", 0, 0, 0, 0, 0, 0);
   }
 
   @Test(expectedExceptions = NullPointerException.class)
   public void testNullDimensionValueCreation() {
-    new DimNameValueCostEntry("", null, 0, 0, 0, 0);
+    new DimNameValueCostEntry("", null, 0, 0, 0, 0, 0, 0);
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/DimensionValuesTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/dbrow/DimensionValuesTest.java
similarity index 96%
rename from thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/DimensionValuesTest.java
rename to thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/dbrow/DimensionValuesTest.java
index b0cce74..f3ace00 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/DimensionValuesTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/dbrow/DimensionValuesTest.java
@@ -14,11 +14,12 @@
  * limitations under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
+package org.apache.pinot.thirdeye.cube.data.dbrow;
 
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import org.apache.pinot.thirdeye.cube.data.dbrow.DimensionValues;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/DimensionsTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/dbrow/DimensionsTest.java
similarity index 97%
rename from thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/DimensionsTest.java
rename to thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/dbrow/DimensionsTest.java
index 718ee61..1b23e91 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/client/diffsummary/DimensionsTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/dbrow/DimensionsTest.java
@@ -14,13 +14,14 @@
  * limitations under the License.
  */
 
-package org.apache.pinot.thirdeye.client.diffsummary;
+package org.apache.pinot.thirdeye.cube.data.dbrow;
 
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 import org.apache.commons.collections.ListUtils;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/node/CubeNodeTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/node/CubeNodeTest.java
new file mode 100644
index 0000000..1286ca0
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/cube/data/node/CubeNodeTest.java
@@ -0,0 +1,187 @@
+/**
+ * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
+ *
+ * Licensed 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.cube.data.node;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.pinot.thirdeye.cube.data.dbrow.DimensionValues;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
+import org.apache.pinot.thirdeye.cube.additive.AdditiveCubeNode;
+import org.apache.pinot.thirdeye.cube.additive.AdditiveRow;
+import org.apache.pinot.thirdeye.cube.data.dbrow.Row;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class CubeNodeTest {
+
+  // Since CubeNode has cyclic reference between current node and parent node, the toString() will encounter
+  // overflowStack exception if it doesn't take care of the cyclic reference carefully.
+  @Test
+  public void testToString() {
+    AdditiveRow root = new AdditiveRow(new Dimensions(), new DimensionValues());
+    AdditiveCubeNode rootNode = new AdditiveCubeNode(root);
+
+    AdditiveRow child = new AdditiveRow(new Dimensions(Collections.singletonList("country")),
+        new DimensionValues(Collections.singletonList("US")), 20, 30);
+    AdditiveCubeNode childNode = new AdditiveCubeNode(1, 0, child, rootNode);
+
+    childNode.toString();
+  }
+
+  @Test
+  public void testSimpleEquals() {
+    AdditiveRow root1 = new AdditiveRow(new Dimensions(), new DimensionValues());
+    AdditiveCubeNode rootNode1 = new AdditiveCubeNode(root1);
+
+    AdditiveRow root2 = new AdditiveRow(new Dimensions(), new DimensionValues());
+    AdditiveCubeNode rootNode2 = new AdditiveCubeNode(root2);
+
+    Assert.assertEquals(rootNode1, rootNode2);
+    Assert.assertTrue(CubeNodeUtils.equalHierarchy(rootNode1, rootNode2));
+
+    AdditiveRow root3 = new AdditiveRow(new Dimensions(Collections.singletonList("country")),
+        new DimensionValues(Collections.singletonList("US")));
+    CubeNode rootNode3 = new AdditiveCubeNode(root3);
+    Assert.assertNotEquals(rootNode1, rootNode3);
+  }
+
+  @Test
+  public void testHierarchicalEquals() {
+    AdditiveCubeNode rootNode1 = buildHierarchicalNodes();
+    AdditiveCubeNode rootNode2 = buildHierarchicalNodes();
+
+    Assert.assertTrue(CubeNodeUtils.equalHierarchy(rootNode1, rootNode2));
+  }
+
+  /**
+   * Hierarchy 1:
+   *      A
+   *     / \
+   *    B  C
+   *
+   * Hierarchy 2:
+   *      A
+   *
+   * Failed because structure difference.
+   */
+  @Test
+  public void testHierarchicalEqualsFail1() {
+    AdditiveCubeNode rootNode1 = buildHierarchicalNodes();
+
+    AdditiveRow rootRow = new AdditiveRow(new Dimensions(), new DimensionValues(), 30, 45);
+    AdditiveCubeNode rootNode2 = new AdditiveCubeNode(rootRow);
+
+    Assert.assertEquals(rootNode1, rootNode2);
+    Assert.assertFalse(CubeNodeUtils.equalHierarchy(rootNode1, rootNode2));
+  }
+
+  /**
+   * Hierarchy 1:
+   *      A
+   *     / \
+   *    B  C
+   *
+   * Hierarchy 2:
+   *      A'
+   *
+   * Failed because data difference.
+   */
+  @Test
+  public void testHierarchicalEqualsFail2() throws Exception {
+    AdditiveCubeNode rootNode1 = buildHierarchicalNodes();
+
+    AdditiveRow rootRow = new AdditiveRow(new Dimensions(), new DimensionValues(), 20, 15);
+    AdditiveCubeNode rootNode2 = new AdditiveCubeNode(rootRow);
+
+    Assert.assertNotEquals(rootNode1, rootNode2);
+    Assert.assertFalse(CubeNodeUtils.equalHierarchy(rootNode1, rootNode2));
+  }
+
+  /**
+   * Hierarchy 1:
+   *      A
+   *     ^ ^
+   *     / \
+   *    v  v
+   *    B  C
+   *
+   * Hierarchy 2:
+   *      A
+   *      ^
+   *       \
+   *       v
+   *       C
+   *
+   * Failed because Hierarchy 2's A doesn't have a reference to B.
+   */
+  @Test
+  public void testHierarchicalEqualsFail3() throws Exception {
+    AdditiveCubeNode rootNode1 = buildHierarchicalNodes();
+
+    List<List<Row>> rows = buildHierarchicalRows();
+    // Root level
+    AdditiveRow rootRow = (AdditiveRow) rows.get(0).get(0);
+    AdditiveCubeNode rootNode2 = new AdditiveCubeNode(rootRow);
+
+    // Level 1
+    AdditiveRow INRow = (AdditiveRow) rows.get(1).get(1);
+    CubeNode INNode = new AdditiveCubeNode(1, 1, INRow, rootNode2);
+
+    Assert.assertEquals(rootNode1, rootNode2);
+    Assert.assertFalse(CubeNodeUtils.equalHierarchy(rootNode1, rootNode2));
+  }
+
+  private List<List<Row>> buildHierarchicalRows() {
+    List<List<Row>> hierarchicalRows = new ArrayList<>();
+    // Root level
+    List<Row> rootLevel = new ArrayList<>();
+    rootLevel.add(new AdditiveRow(new Dimensions(), new DimensionValues(), 30, 45));
+    hierarchicalRows.add(rootLevel);
+
+    // Level 1
+    List<Row> level1 = new ArrayList<>();
+    Row row1 = new AdditiveRow(new Dimensions(Collections.singletonList("country")),
+        new DimensionValues(Collections.singletonList("US")), 20, 30);
+    level1.add(row1);
+
+    Row row2 = new AdditiveRow(new Dimensions(Collections.singletonList("country")),
+        new DimensionValues(Collections.singletonList("IN")), 10, 15);
+    level1.add(row2);
+
+    hierarchicalRows.add(level1);
+
+    return hierarchicalRows;
+  }
+
+  private AdditiveCubeNode buildHierarchicalNodes() {
+    List<List<Row>> rows = buildHierarchicalRows();
+    // Root level
+    AdditiveRow rootRow = (AdditiveRow) rows.get(0).get(0);
+    AdditiveCubeNode rootNode = new AdditiveCubeNode(rootRow);
+
+    // Level 1
+    AdditiveRow USRow = (AdditiveRow) rows.get(1).get(0);
+    CubeNode USNode = new AdditiveCubeNode(1, 0, USRow, rootNode);
+
+    AdditiveRow INRow = (AdditiveRow) rows.get(1).get(1);
+    CubeNode INNode = new AdditiveCubeNode(1, 1, INRow, rootNode);
+
+    return rootNode;
+  }
+}


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