You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by xx...@apache.org on 2023/02/23 04:44:16 UTC

[kylin] branch kylin5 updated: KYLIN-5387 cost based index planner phase1 (#2089)

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

xxyu pushed a commit to branch kylin5
in repository https://gitbox.apache.org/repos/asf/kylin.git


The following commit(s) were added to refs/heads/kylin5 by this push:
     new b2b3a6a8d2 KYLIN-5387 cost based index planner phase1 (#2089)
b2b3a6a8d2 is described below

commit b2b3a6a8d290e05543da4a43225829a78e5b3bbb
Author: Kun Liu <li...@apache.org>
AuthorDate: Thu Feb 23 12:44:08 2023 +0800

    KYLIN-5387 cost based index planner phase1 (#2089)
    
    * cost based index planner phase1
    
    * use col order instead of layout index
    
    * [cost based index planner] fix comment: 1. remove useless TODO 2. remove useless code
    
    * [cost based index planner] add error code for cost based index planner job
    
    * change the cost based index planner it: change the metadata file to spark-it project; delete cube_planner test data
---
 .../src/components/monitor/batchJobs/handler.js    |    1 +
 .../src/components/monitor/batchJobs/locales.js    |    1 +
 .../org/apache/kylin/common/KylinConfigBase.java   |   49 +
 .../kylin/common/exception/JobErrorCode.java       |    5 +-
 .../resources/kylin_errorcode_conf_en.properties   |    1 +
 .../kylin/job/common/SegmentBuildJobUtil.java      |    2 +-
 .../kylin/job/constant/ExecutableConstants.java    |    1 +
 .../kylin/metadata/cube/model/IndexPlan.java       |   16 +-
 .../kylin/metadata/cube/model/LayoutEntity.java    |    5 +-
 .../kylin/metadata/cube/model/NBatchConstants.java |    1 +
 .../kylin/metadata/cube/model/NDataSegment.java    |    2 +-
 .../kylin/metadata/cube/model/RuleBasedIndex.java  |   71 +-
 .../cube/planner/CostBasePlannerUtils.java         |  197 +
 .../algorithm/AbstractRecommendAlgorithm.java      |   82 +
 .../cube/planner/algorithm/BPUSCalculator.java     |  175 +
 .../cube/planner/algorithm/BenefitPolicy.java      |   51 +
 .../cube/planner/algorithm/CuboidBenefitModel.java |   89 +
 .../algorithm/CuboidRecommendAlgorithm.java        |   50 +
 .../cube/planner/algorithm/CuboidStats.java        |  302 ++
 .../cube/planner/algorithm/CuboidStatsUtil.java    |  371 ++
 .../cube/planner/algorithm/PBPUSCalculator.java    |   55 +
 .../cube/planner/algorithm/SPBPUSCalculator.java   |   44 +
 .../planner/algorithm/genetic/BitsChromosome.java  |  117 +
 .../algorithm/genetic/BitsChromosomeHelper.java    |  122 +
 .../planner/algorithm/genetic/BitsMutation.java    |   59 +
 .../algorithm/genetic/BitsOnePointCrossover.java   |  131 +
 .../genetic/CombinedStoppingCondition.java         |   47 +
 .../algorithm/genetic/GeneticAlgorithm.java        |  137 +
 .../algorithm/genetic/RouletteWheelSelection.java  |   61 +
 .../planner/algorithm/greedy/GreedyAlgorithm.java  |  162 +
 .../cube/planner/CostBasePlannerUtilsTest.java     |   34 +
 .../cube/planner/algorithm/AlgorithmTestBase.java  |  567 +++
 .../algorithm/genetic/GeneticAlgorithmTest.java    |   99 +
 .../algorithm/greedy/GreedyAlgorithmTest.java      |  110 +
 .../src/test/resources/statistics.txt              | 4092 ++++++++++++++++++++
 .../kylin/rest/service/FusionIndexService.java     |   10 +-
 .../kylin/rest/service/IndexPlanService.java       |    8 +-
 .../apache/kylin/engine/spark/job/JobStepType.java |    6 +
 .../kylin/engine/spark/job/NSparkCubingJob.java    |  108 +-
 .../apache/kylin/engine/spark/job/StageType.java   |   16 +
 .../spark/merger/AfterBuildResourceMerger.java     |    1 -
 .../merger/AfterMergeOrRefreshResourceMerger.java  |    6 +-
 .../spark/merger/SparkJobMetadataMerger.java       |   16 +-
 .../kylin/engine/spark/job/PartitionExec.scala     |    2 +-
 .../kylin/engine/spark/job/RDSegmentBuildJob.java  |    2 +-
 .../kylin/engine/spark/job/SegmentBuildJob.java    |   19 +-
 .../kylin/engine/spark/job/SegmentExec.scala       |    1 +
 .../apache/kylin/engine/spark/job/SegmentJob.java  |   79 +-
 .../engine/spark/job/stage/build/BuildStage.scala  |   11 +-
 ...dFactTableView.scala => CostBasedPlanner.scala} |   40 +-
 .../job/stage/build/FlatTableAndDictBase.scala     |   45 +-
 .../stage/build/MaterializedFactTableView.scala    |    2 +-
 ...eView.scala => PartitionCostBasedPlanner.scala} |   54 +-
 .../PartitionMaterializedFactTableView.scala       |    2 +-
 .../spark/model/planner/CuboIdToLayoutUtils.java   |  140 +
 .../spark/model/planner/FlatTableToCostUtils.java  |  518 +++
 .../model/planner/CuboIdToLayoutUtilsTest.java     |  215 +
 .../d863b37c-e1a9-717f-7df7-74991815b1eb.json      |   13 +
 .../d863b37c-e1a9-717f-7df7-74991815b1eb.json      |   43 +
 .../d863b37c-e1a9-717f-7df7-74991815b1eb.json      |  128 +
 .../scala/org/apache/kylin/common/JobSupport.scala |   80 +-
 .../scala/org/apache/kylin/common/SSSource.scala   |   34 +-
 .../org/apache/kylin/it/TestCubePlanner.scala      |  157 +
 .../org/apache/kylin/it/TestModelViewQuery.scala   |   31 +-
 64 files changed, 8937 insertions(+), 159 deletions(-)

diff --git a/kystudio/src/components/monitor/batchJobs/handler.js b/kystudio/src/components/monitor/batchJobs/handler.js
index 0b27635e13..a1fe4fc0bf 100644
--- a/kystudio/src/components/monitor/batchJobs/handler.js
+++ b/kystudio/src/components/monitor/batchJobs/handler.js
@@ -23,6 +23,7 @@ export function getSubTasksName (that, name) {
     'Generate global dictionary': that.$t('generateGlobalDict'),
     'Generate flat table': that.$t('generateFlatTable'),
     'Save flat table': that.$t('saveFlatTable'),
+    'Cost based planner': that.$t('costBasedPlanner'),
     'Get flat table statistics': that.$t('getFlatTableStatistics'),
     'Generate global dictionary of computed columns': that.$t('generateDictOfCC'),
     'Merge flat table': that.$t('mergeFlatTable'),
diff --git a/kystudio/src/components/monitor/batchJobs/locales.js b/kystudio/src/components/monitor/batchJobs/locales.js
index d88c38ea8f..536ba33da8 100644
--- a/kystudio/src/components/monitor/batchJobs/locales.js
+++ b/kystudio/src/components/monitor/batchJobs/locales.js
@@ -110,6 +110,7 @@ export default {
     generateGlobalDict: 'Generate global dictionary',
     generateFlatTable: 'Generate flat table',
     saveFlatTable: 'Save flat table',
+    costBasedPlanner: 'Cost based planner',
     getFlatTableStatistics: 'Get flat table statistics',
     generateDictOfCC: 'Generate global dictionary of computed columns',
     mergeFlatTable: 'Merge flat table',
diff --git a/src/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/src/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index bbff8f22e8..7fd00664a1 100644
--- a/src/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/src/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -3749,4 +3749,53 @@ public abstract class KylinConfigBase implements Serializable {
     public boolean skipShardPruningForInExpr() {
         return Boolean.parseBoolean(getOptional("kylin.query.skip-shard-pruning-for-in", FALSE));
     }
+
+    // ============================================================================
+    // Cost based index Planner
+    // ============================================================================
+
+    public boolean enableCostBasedIndexPlanner() {
+        // If enable the cost based planner, will recommend subset of index layouts from the index rule.
+        return Boolean.parseBoolean(getOptional("kylin.index.costbased.enabled", FALSE));
+    }
+
+    public int getCostBasedPlannerGreedyAlgorithmAutoThreshold() {
+        return Integer.parseInt(getOptional("kylin.index.costbased.algorithm-threshold-greedy", "8"));
+    }
+
+    public int getCostBasedPlannerGeneticAlgorithmAutoThreshold() {
+        return Integer.parseInt(getOptional("kylin.index.costbased.algorithm-threshold-genetic", "23"));
+    }
+
+    public double getCostBasedPlannerExpansionRateThreshold() {
+        return Double.parseDouble(getOptional("kylin.index.costbased.expansion-threshold", "15.0"));
+    }
+
+    public double getCostBasedPlannerBPUSMinBenefitRatio() {
+        return Double.parseDouble(getOptional("kylin.index.costbased.bpus-min-benefit-ratio", "0.01"));
+    }
+
+    public int getStatsHLLPrecision() {
+        return Integer.parseInt(getOptional("kylin.index.costbased.sampling-hll-precision", "14"));
+    }
+
+    public double getJobCuboidSizeRatio() {
+        return Double.parseDouble(getOptional("kylin.index.costbased.model-size-estimate-ratio", "0.25"));
+    }
+
+    public double getJobCuboidSizeCountDistinctRatio() {
+        return Double.parseDouble(getOptional("kylin.index.costbased.model-size-estimate-countdistinct-ratio", "0.5"));
+    }
+
+    public double getJobCuboidSizeTopNRatio() {
+        return Double.parseDouble(getOptional("kylin.index.costbased.model-size-estimate-topn-ratio", "0.5"));
+    }
+
+    public int getJobPerReducerHLLCuboidNumber() {
+        return Integer.parseInt(getOptional("kylin.index.costbased.per-reducer-hll-cuboid-number", "100"));
+    }
+
+    public int getJobHLLMaxReducerNumber() {
+        return Integer.parseInt(getOptional("kylin.index.costbased.hll-max-reducer-number", "1"));
+    }
 }
diff --git a/src/core-common/src/main/java/org/apache/kylin/common/exception/JobErrorCode.java b/src/core-common/src/main/java/org/apache/kylin/common/exception/JobErrorCode.java
index 6154834770..f337611f47 100644
--- a/src/core-common/src/main/java/org/apache/kylin/common/exception/JobErrorCode.java
+++ b/src/core-common/src/main/java/org/apache/kylin/common/exception/JobErrorCode.java
@@ -35,7 +35,10 @@ public enum JobErrorCode implements ErrorCodeSupplier {
 
     SECOND_STORAGE_SEGMENTS_CONFLICTS("KE-030002001"),
     SECOND_STORAGE_JOB_EXISTS("KE-030002002"),
-    SECOND_STORAGE_PROJECT_JOB_EXISTS("KE-030002003");
+    SECOND_STORAGE_PROJECT_JOB_EXISTS("KE-030002003"),
+
+    // cost based planner error code
+    COST_BASED_PLANNER_ERROR("KE-030003001");
 
     private final ErrorCode errorCode;
 
diff --git a/src/core-common/src/main/resources/kylin_errorcode_conf_en.properties b/src/core-common/src/main/resources/kylin_errorcode_conf_en.properties
index ad67f4540f..40d5b154f5 100644
--- a/src/core-common/src/main/resources/kylin_errorcode_conf_en.properties
+++ b/src/core-common/src/main/resources/kylin_errorcode_conf_en.properties
@@ -204,6 +204,7 @@ KE-030001009=Spark master node is unreachable
 KE-030001010=Profiler not enabled
 KE-030001011=Profiler status error
 KE-030001012=Profiler collect timeout
+KE-030003001=Cost based index planner job error
 #system
 KE-040005001=Invalid Password Encoder
 KE-040005002=Failed Init Password Encoder
diff --git a/src/core-job/src/main/java/org/apache/kylin/job/common/SegmentBuildJobUtil.java b/src/core-job/src/main/java/org/apache/kylin/job/common/SegmentBuildJobUtil.java
index b5eaada3fe..42c1844d02 100644
--- a/src/core-job/src/main/java/org/apache/kylin/job/common/SegmentBuildJobUtil.java
+++ b/src/core-job/src/main/java/org/apache/kylin/job/common/SegmentBuildJobUtil.java
@@ -32,7 +32,7 @@ import lombok.val;
 import lombok.extern.slf4j.Slf4j;
 
 /**
- *
+ * This is used for computing and setting the parameters for spark building segment job.
  **/
 @Slf4j
 public class SegmentBuildJobUtil extends ExecutableUtil {
diff --git a/src/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java b/src/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
index 864f26d52a..93f2ced333 100644
--- a/src/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
+++ b/src/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
@@ -79,6 +79,7 @@ public final class ExecutableConstants {
     public static final String STAGE_NAME_MATERIALIZED_FACT_TABLE = "Materialize fact table view";
     public static final String STAGE_NAME_BUILD_DICT = "Generate global dictionary";
     public static final String STAGE_NAME_GENERATE_FLAT_TABLE = "Generate flat table";
+    public static final String STAGE_NAME_COST_BASED_PLANNER = "Cost based planner";
     public static final String STAGE_NAME_GATHER_FLAT_TABLE_STATS = "Get flat table statistics";
     public static final String STAGE_NAME_BUILD_LAYER = "Build indexes by layer";
     public static final String STAGE_NAME_REFRESH_COLUMN_BYTES = "Update flat table statistics";
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/IndexPlan.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/IndexPlan.java
index c004d53135..33c708b9e8 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/IndexPlan.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/IndexPlan.java
@@ -30,6 +30,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
+import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
@@ -239,7 +240,7 @@ public class IndexPlan extends RootPersistentEntity implements Serializable, IEn
         if (ruleBasedIndex.getBaseLayoutEnabled() == null) {
             ruleBasedIndex.setBaseLayoutEnabled(getConfig().isBaseCuboidAlwaysValid());
         }
-        ruleBasedLayouts.addAll(ruleBasedIndex.genCuboidLayouts());
+        ruleBasedLayouts.addAll(ruleBasedIndex.genCuboidLayouts(true));
         if (config.base().isSystemConfig() && isCachedAndShared) {
             ruleBasedIndex.getCuboidScheduler().validateOrder();
         }
@@ -431,7 +432,7 @@ public class IndexPlan extends RootPersistentEntity implements Serializable, IEn
             copy.getLayouts().forEach(layout -> layout.setInProposing(layouts.get(layout.getId()).isInProposing()));
             retSubscript++;
         }
-        for (LayoutEntity ruleBasedLayout : ruleBasedLayouts) {
+        for (LayoutEntity ruleBasedLayout : getRuleBaseLayouts()) {
             val ruleRelatedIndex = ruleBasedLayout.getIndex();
             if (!retSubscriptMap.containsKey(ruleRelatedIndex.getId())) {
                 val copy = JsonUtil.deepCopyQuietly(ruleRelatedIndex, IndexEntity.class);
@@ -480,7 +481,7 @@ public class IndexPlan extends RootPersistentEntity implements Serializable, IEn
         for (IndexEntity indexEntity : indexes) {
             indexEntity.getLayouts().forEach(layout -> classifyByIndexId(layout, resultMap, layout.isToBeDeleted()));
         }
-        for (LayoutEntity ruleBasedLayout : ruleBasedLayouts) {
+        for (LayoutEntity ruleBasedLayout : getRuleBaseLayouts()) {
             classifyByIndexId(ruleBasedLayout, resultMap, false);
         }
         for (IndexEntity indexEntity : toBeDeletedIndexes) {
@@ -594,8 +595,7 @@ public class IndexPlan extends RootPersistentEntity implements Serializable, IEn
         this.ruleBasedIndex = ruleBasedIndex;
 
         Set<LayoutEntity> targetSet = this.ruleBasedIndex.genCuboidLayouts();
-
-        this.ruleBasedLayouts = Lists.newArrayList(targetSet);
+        this.ruleBasedLayouts = Lists.newArrayList(this.ruleBasedIndex.genCuboidLayouts(true));
         if (markToBeDeleted && CollectionUtils.isNotEmpty(layoutsNotIn(targetSet, originSet))) {
             Set<LayoutEntity> toBeDeletedSet = layoutsNotIn(originSet, targetSet);
             if (CollectionUtils.isNotEmpty(toBeDeletedSet)) {
@@ -621,7 +621,7 @@ public class IndexPlan extends RootPersistentEntity implements Serializable, IEn
             ruleBasedIndex.setIndexStartId(nextAggregationIndexId);
             ruleBasedIndex.setLayoutIdMapping(Lists.newArrayList());
             ruleBasedIndex.genCuboidLayouts(ruleLayouts);
-            this.ruleBasedLayouts = Lists.newArrayList(ruleBasedIndex.genCuboidLayouts());
+            this.ruleBasedLayouts = Lists.newArrayList(ruleBasedIndex.genCuboidLayouts(true));
         }
         this.aggShardByColumns = aggShardByColumns;
         updateNextId();
@@ -634,7 +634,7 @@ public class IndexPlan extends RootPersistentEntity implements Serializable, IEn
             this.extendPartitionColumns = extendPartitionColumns;
             ruleBasedIndex.setLayoutIdMapping(Lists.newArrayList());
             ruleBasedIndex.genCuboidLayouts(ruleLayouts);
-            this.ruleBasedLayouts = Lists.newArrayList(ruleBasedIndex.genCuboidLayouts());
+            this.ruleBasedLayouts = Lists.newArrayList(ruleBasedIndex.genCuboidLayouts(true));
         }
         this.extendPartitionColumns = extendPartitionColumns;
         updateNextId();
@@ -741,7 +741,7 @@ public class IndexPlan extends RootPersistentEntity implements Serializable, IEn
             val newBlacklist = Sets.newHashSet(originBlacklist);
             newBlacklist.addAll(blacklist);
             ruleBasedIndex.setLayoutBlackList(newBlacklist);
-            this.ruleBasedLayouts = Lists.newArrayList(ruleBasedIndex.genCuboidLayouts());
+            this.ruleBasedLayouts = Lists.newArrayList(ruleBasedIndex.genCuboidLayouts(true));
         }
         updateNextId();
     }
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/LayoutEntity.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/LayoutEntity.java
index 0cd6c33326..d812214612 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/LayoutEntity.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/LayoutEntity.java
@@ -394,7 +394,10 @@ public class LayoutEntity implements IStorageAware, Serializable {
 
     @Override
     public String toString() {
-        return MoreObjects.toStringHelper(this).add("id", id).toString();
+        return MoreObjects.toStringHelper(this)
+                .add("id", id)
+                .add("colOrder", getColOrder().toString())
+                .toString();
     }
 
     public boolean isDraft() {
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NBatchConstants.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NBatchConstants.java
index a7b1260ce8..754769280b 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NBatchConstants.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NBatchConstants.java
@@ -23,6 +23,7 @@ public interface NBatchConstants {
     String P_SEGMENT_IDS = "segmentIds";
     String P_JOB_ID = "jobId";
     String P_JOB_TYPE = "jobType";
+    String P_JOB_ENABLE_PLANNER = "enablePlanner";
     String P_LAYOUT_IDS = "layoutIds";
     String P_TO_BE_DELETED_LAYOUT_IDS = "toBeDeletedLayoutIds";
     String P_CLASS_NAME = "className";
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataSegment.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataSegment.java
index eb7585347f..b0e4795c6d 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataSegment.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataSegment.java
@@ -723,7 +723,7 @@ public class NDataSegment implements ISegment, Serializable {
 
     @Override
     public String toString() {
-        return "NDataSegment [" + dataflow.getUuid() + "," + id + "," + segmentRange + "]";
+        return "NDataSegment [" + dataflow.getUuid() + "," + id + "," + segmentRange + "," + status + "]";
     }
 
     public String displayIdName() {
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/RuleBasedIndex.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/RuleBasedIndex.java
index 88fe1715a0..c59af3378b 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/RuleBasedIndex.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/RuleBasedIndex.java
@@ -20,6 +20,7 @@ package org.apache.kylin.metadata.cube.model;
 
 import java.io.Serializable;
 import java.math.BigInteger;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -29,13 +30,13 @@ import java.util.stream.Collectors;
 
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.metadata.model.IStorageAware;
-import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.cube.cuboid.CuboidScheduler;
 import org.apache.kylin.metadata.cube.cuboid.CuboidScheduler.ColOrder;
 import org.apache.kylin.metadata.cube.cuboid.NAggregationGroup;
 import org.apache.kylin.metadata.cube.model.IndexEntity.IndexIdentifier;
+import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.NDataModel;
+import org.apache.kylin.metadata.model.TblColRef;
 import org.springframework.beans.BeanUtils;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
@@ -101,6 +102,11 @@ public class RuleBasedIndex implements Serializable {
     @JsonProperty("layout_black_list")
     private Set<Long> layoutBlackList = new HashSet<>();
 
+    @Setter
+    @Getter
+    @JsonProperty("layout_cost_based_pruned_list")
+    private Set<Long> layoutsOfCostBasedList = null;
+
     @Setter
     @Getter
     @JsonProperty("scheduler_version")
@@ -191,7 +197,11 @@ public class RuleBasedIndex implements Serializable {
     }
 
     public Set<LayoutEntity> genCuboidLayouts() {
-        return genCuboidLayouts(Sets.newHashSet(), Sets.newHashSet(), true);
+        return genCuboidLayouts(Sets.newHashSet(), Sets.newHashSet(), true, false);
+    }
+
+    public Set<LayoutEntity> genCuboidLayouts(boolean useCostBasedList) {
+        return genCuboidLayouts(Sets.newHashSet(), Sets.newHashSet(), true, useCostBasedList);
     }
 
     public boolean getIndexUpdateEnabled() {
@@ -234,6 +244,30 @@ public class RuleBasedIndex implements Serializable {
         this.aggregationGroups = aggregationGroups;
     }
 
+    public Map<Integer, Integer> getColumnIdToRowKeyId() {
+        int rowKeyId = 0;
+        Map<Integer, Integer> result = new HashMap<>();
+        for (Integer columnId : dimensions) {
+            result.put(columnId, rowKeyId);
+            rowKeyId++;
+        }
+        return result;
+    }
+
+    public Map<Integer, Integer> getRowKeyIdToColumnId() {
+        int rowKeyId = 0;
+        Map<Integer, Integer> result = new HashMap<>();
+        for (Integer columnId : dimensions) {
+            result.put(rowKeyId, columnId);
+            rowKeyId++;
+        }
+        return result;
+    }
+
+    public int countOfIncludeDimension() {
+        return dimensions.size();
+    }
+
     public boolean isCachedAndShared() {
         return indexPlan != null && indexPlan.isCachedAndShared();
     }
@@ -256,15 +290,24 @@ public class RuleBasedIndex implements Serializable {
     }
 
     Set<LayoutEntity> genCuboidLayouts(Set<LayoutEntity> previousLayouts) {
-        return genCuboidLayouts(previousLayouts, Sets.newHashSet(), true);
+        return genCuboidLayouts(previousLayouts, Sets.newHashSet(), true, false);
     }
 
     Set<LayoutEntity> genCuboidLayouts(Set<LayoutEntity> previousLayouts, Set<LayoutEntity> needDelLayouts) {
-        return genCuboidLayouts(previousLayouts, needDelLayouts, true);
+        return genCuboidLayouts(previousLayouts, needDelLayouts, true, false);
     }
 
-    Set<LayoutEntity> genCuboidLayouts(Set<LayoutEntity> previousLayouts, Set<LayoutEntity> needDelLayouts,
-            boolean excludeDel) {
+    /**
+     *
+     * @param previousLayouts
+     * @param needDelLayouts In order to make no changes for the same layout,
+     *                       we can get the layout id from the previous `RuleBasedIndex` for `previousLayouts` and `needDelLayouts`
+     * @param excludeDel Whether to include the result of deleted layouts
+     * @param useCostBasedList Whether to use the the list of `layoutsOfCostBasedList`
+     * @return
+     */
+    private Set<LayoutEntity> genCuboidLayouts(Set<LayoutEntity> previousLayouts, Set<LayoutEntity> needDelLayouts,
+            boolean excludeDel, boolean useCostBasedList) {
 
         Set<LayoutEntity> genLayouts = Sets.newHashSet();
 
@@ -348,6 +391,18 @@ public class RuleBasedIndex implements Serializable {
         if (excludeDel) {
             genLayouts.removeIf(layout -> layoutBlackList.contains(layout.getId()));
         }
+
+        // If contains the `layout_cost_based_pruned_list`, will use layouts in the cost based planner list
+        if (useCostBasedList && layoutsOfCostBasedList != null) {
+            // use the recommend white list id
+            Set<LayoutEntity> result = Sets.newHashSet();
+            genLayouts.stream().forEach(layout -> {
+                if (layoutsOfCostBasedList.contains(layout.getId())) {
+                    result.add(layout);
+                }
+            });
+            genLayouts = result;
+        }
         return genLayouts;
     }
 
@@ -395,7 +450,7 @@ public class RuleBasedIndex implements Serializable {
     }
 
     public Set<LayoutEntity> getBlacklistLayouts() {
-        val allLayouts = genCuboidLayouts(Sets.newHashSet(), Sets.newHashSet(), false);
+        val allLayouts = genCuboidLayouts(Sets.newHashSet(), Sets.newHashSet(), false, false);
         val existLayouts = genCuboidLayouts();
         return allLayouts.stream().filter(layout -> !existLayouts.contains(layout)).collect(Collectors.toSet());
     }
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/CostBasePlannerUtils.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/CostBasePlannerUtils.java
new file mode 100644
index 0000000000..07139fb035
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/CostBasePlannerUtils.java
@@ -0,0 +1,197 @@
+/*
+ * 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.kylin.metadata.cube.planner;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.metadata.cube.cuboid.NAggregationGroup;
+import org.apache.kylin.metadata.cube.model.LayoutEntity;
+import org.apache.kylin.metadata.cube.model.RuleBasedIndex;
+import org.apache.kylin.metadata.cube.planner.algorithm.BenefitPolicy;
+import org.apache.kylin.metadata.cube.planner.algorithm.CuboidRecommendAlgorithm;
+import org.apache.kylin.metadata.cube.planner.algorithm.CuboidStats;
+import org.apache.kylin.metadata.cube.planner.algorithm.PBPUSCalculator;
+import org.apache.kylin.metadata.cube.planner.algorithm.genetic.GeneticAlgorithm;
+import org.apache.kylin.metadata.cube.planner.algorithm.greedy.GreedyAlgorithm;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+
+public class CostBasePlannerUtils {
+    private static final Logger logger = LoggerFactory.getLogger(CostBasePlannerUtils.class);
+
+    public static Map<BigInteger, Long> getRecommendCuboidList(RuleBasedIndex ruleBasedIndex, KylinConfig kylinConf,
+            String modelName, Map<BigInteger, Long> cuboidRowCountMap, Map<BigInteger, Double> cuboidSizeMap) {
+        BigInteger baseCuboid = generateBaseCuboId(ruleBasedIndex);
+        Set<BigInteger> mandatoryCuboids = generateMandatoryCuboIds(ruleBasedIndex);
+        logger.info("Build cuboid stats model name {}, baseCuboid {}, mandatory cuboid {}, statistic cuboid {}",
+                modelName, baseCuboid, mandatoryCuboids, cuboidRowCountMap.keySet());
+        CuboidStats cuboidStats = new CuboidStats.Builder(modelName, baseCuboid, baseCuboid, cuboidRowCountMap,
+                cuboidSizeMap).setMandatoryCuboids(mandatoryCuboids)
+                        .setBPUSMinBenefitRatio(kylinConf.getCostBasedPlannerBPUSMinBenefitRatio()).build();
+        Map<BigInteger, Long> result = getRecommendCuboidList(cuboidStats, kylinConf, !mandatoryCuboids.isEmpty());
+        // if not recommend any cuboid and just apply all layouts with the rule base index
+        if (result == null || result.isEmpty()) {
+            result = new HashMap<>();
+            Set<LayoutEntity> allLayouts = ruleBasedIndex.genCuboidLayouts();
+            for (LayoutEntity layoutEntity : allLayouts) {
+                BigInteger cuboid = convertDimensionsToCuboId(layoutEntity.getDimsIds(),
+                        ruleBasedIndex.countOfIncludeDimension(), ruleBasedIndex.getColumnIdToRowKeyId());
+                result.put(cuboid, 0L);
+            }
+            logger.info("Not recommend any cuboid with the cost based method, and use the rule based cuboid {}",
+                    result.keySet());
+        }
+        return result;
+    }
+
+    private static BigInteger generateBaseCuboId(RuleBasedIndex ruleBasedIndex) {
+        int dimensionCount = ruleBasedIndex.countOfIncludeDimension();
+        List<Integer> dimensionIds = new ArrayList<>(ruleBasedIndex.getDimensions());
+        BigInteger cuboid = convertDimensionsToCuboId(dimensionIds, dimensionCount,
+                ruleBasedIndex.getColumnIdToRowKeyId());
+        return cuboid;
+    }
+
+    private static Set<BigInteger> generateMandatoryCuboIds(RuleBasedIndex ruleBasedIndex) {
+        Set<BigInteger> result = new HashSet<>();
+        if (ruleBasedIndex != null) {
+            int dimensionCount = ruleBasedIndex.countOfIncludeDimension();
+            for (NAggregationGroup aggregationGroup : ruleBasedIndex.getAggregationGroups()) {
+                Integer[] mandatoryDimensionIds = aggregationGroup.getSelectRule().getMandatoryDims();
+                // If there is no mandatory for the agg group, should not add the cuboid
+                if (mandatoryDimensionIds != null && mandatoryDimensionIds.length != 0) {
+                    BigInteger cuboid = convertDimensionsToCuboId(mandatoryDimensionIds, dimensionCount,
+                            ruleBasedIndex.getColumnIdToRowKeyId());
+                    result.add(cuboid);
+                }
+            }
+        }
+        return result;
+    }
+
+    private static Map<BigInteger, Long> getRecommendCuboidList(CuboidStats cuboidStats, KylinConfig kylinConf,
+            boolean ifForceRecommend) {
+        // default is 1<<7 = 128
+        long threshold1 = 1L << kylinConf.getCostBasedPlannerGreedyAlgorithmAutoThreshold() - 1;
+        // default is 1<<22 = 1024*1024*4
+        long threshold2 = 1L << kylinConf.getCostBasedPlannerGeneticAlgorithmAutoThreshold() - 1;
+        if (threshold1 >= threshold2) {
+            logger.error("Invalid Cube Planner Algorithm configuration");
+            return null;
+        }
+        int allCuboidCount = cuboidStats.getAllCuboidsForMandatory().size()
+                + cuboidStats.getAllCuboidsForSelection().size();
+
+        if (!ifForceRecommend && allCuboidCount < threshold1) {
+            logger.info("Not recommend cuboid, the all cuboid count is {}, the threshold1 is {}, force recommend {}",
+                    allCuboidCount, threshold1, ifForceRecommend);
+            return null;
+        }
+
+        BenefitPolicy benefitPolicy = new PBPUSCalculator(cuboidStats);
+        CuboidRecommendAlgorithm algorithm = null;
+
+        if (allCuboidCount <= threshold2) {
+            algorithm = new GreedyAlgorithm(-1, benefitPolicy, cuboidStats);
+        } else {
+            algorithm = new GeneticAlgorithm(-1, benefitPolicy, cuboidStats);
+        }
+        long startTime = System.currentTimeMillis();
+        logger.info("Cube Planner Algorithm started at {}", startTime);
+        List<BigInteger> recommendCuboidList = algorithm
+                .recommend(kylinConf.getCostBasedPlannerExpansionRateThreshold());
+        logger.info("Cube Planner Algorithm ended at {}, cost time {}", System.currentTimeMillis(),
+                System.currentTimeMillis() - startTime);
+
+        if (recommendCuboidList.size() < allCuboidCount) {
+            logger.info("Cube Planner Algorithm chooses {} most effective cuboids to build among of all {} cuboids.",
+                    recommendCuboidList.size(), allCuboidCount);
+        }
+        Map<BigInteger, Long> recommendCuboidsWithStats = Maps.newLinkedHashMap();
+        for (BigInteger cuboid : recommendCuboidList) {
+            if (cuboid.equals(BigInteger.ZERO)) {
+                // for zero cuboid, just simply recommend the cheapest cuboid.
+                handleCuboidZeroRecommend(cuboidStats, recommendCuboidsWithStats);
+            } else {
+                recommendCuboidsWithStats.put(cuboid, cuboidStats.getCuboidCount(cuboid));
+            }
+        }
+        return recommendCuboidsWithStats;
+    }
+
+    private static void handleCuboidZeroRecommend(CuboidStats cuboidStats,
+            Map<BigInteger, Long> recommendCuboidsWithStats) {
+        Map<BigInteger, Long> statistics = cuboidStats.getStatistics();
+        BigInteger cheapestCuboid = null;
+        Long cheapestCuboidCount = Long.MAX_VALUE;
+        for (Map.Entry<BigInteger, Long> cuboidStatsEntry : statistics.entrySet()) {
+            if (cuboidStatsEntry.getValue() < cheapestCuboidCount) {
+                cheapestCuboid = cuboidStatsEntry.getKey();
+                cheapestCuboidCount = cuboidStatsEntry.getValue();
+            }
+        }
+        if (cheapestCuboid != null) {
+            logger.info("recommend cuboid:{} instead of cuboid zero", cheapestCuboid);
+            recommendCuboidsWithStats.put(cheapestCuboid, cheapestCuboidCount);
+        }
+    }
+
+    /**
+     * convert the dimensionid list to cuboid
+     *
+     * @param dimensionIds
+     * @param dimensionCount
+     * @return
+     */
+    private static BigInteger convertDimensionsToCuboId(Integer[] dimensionIds, int dimensionCount,
+            Map<Integer, Integer> columnIdToRowkeyId) {
+        Preconditions.checkNotNull(dimensionIds);
+        Preconditions.checkArgument(dimensionIds.length != 0, "The length of dimensionIds must be greater than 0");
+        BigInteger cuboid = BigInteger.ZERO;
+        // If the dimension count is 12, the ids is [4,8,11]
+        // and the result is 00000000,00000000,00000000,10001001
+        for (Integer dimensionId : dimensionIds) {
+            if (!columnIdToRowkeyId.containsKey(dimensionId)) {
+                throw new RuntimeException("Can't find the rowkey id for the dimension id");
+            }
+            int rowkeyId = columnIdToRowkeyId.get(dimensionId);
+            if (rowkeyId >= dimensionCount) {
+                throw new RuntimeException("The rowkey id must less than the count of dimension");
+            }
+            // set `dimensionCount - 1 - rowkeyId`th bit to 1
+            cuboid = cuboid.setBit(dimensionCount - 1 - rowkeyId);
+        }
+        return cuboid;
+    }
+
+    public static BigInteger convertDimensionsToCuboId(List<Integer> dimensionIds, int dimensionCount,
+            Map<Integer, Integer> columnIdToRowkeyId) {
+        return convertDimensionsToCuboId(dimensionIds.toArray(new Integer[0]), dimensionCount, columnIdToRowkeyId);
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/AbstractRecommendAlgorithm.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/AbstractRecommendAlgorithm.java
new file mode 100644
index 0000000000..21981db619
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/AbstractRecommendAlgorithm.java
@@ -0,0 +1,82 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm;
+
+import java.math.BigInteger;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractRecommendAlgorithm implements CuboidRecommendAlgorithm {
+    private static final Logger logger = LoggerFactory.getLogger(AbstractRecommendAlgorithm.class);
+
+    protected final CuboidStats cuboidStats;
+    protected final BenefitPolicy benefitPolicy;
+
+    private AtomicBoolean cancelRequested = new AtomicBoolean(false);
+    private AtomicBoolean canceled = new AtomicBoolean(false);
+
+    private long timeoutMillis;
+
+    public AbstractRecommendAlgorithm(final long timeout, BenefitPolicy benefitPolicy, CuboidStats cuboidStats) {
+        if (timeout <= 0) {
+            this.timeoutMillis = Long.MAX_VALUE;
+        } else {
+            this.timeoutMillis = timeout;
+        }
+        this.cuboidStats = cuboidStats;
+        this.benefitPolicy = benefitPolicy;
+    }
+
+    @Override
+    public List<BigInteger> recommend(double expansionRate) {
+        double spaceLimit = cuboidStats.getBaseCuboidSize() * expansionRate;
+        logger.info("space limit for the algorithm is {} with expansion rate {}", spaceLimit, expansionRate);
+        return start(spaceLimit);
+    }
+
+    @Override
+    public void cancel() {
+        cancelRequested.set(true);
+    }
+
+    /**
+     * Checks whether the algorithm has been canceled or timed out.
+     */
+    protected boolean shouldCancel() {
+        if (canceled.get()) {
+            return true;
+        }
+        if (cancelRequested.get()) {
+            canceled.set(true);
+            cancelRequested.set(false);
+            logger.warn("Algorithm is canceled.");
+            return true;
+        }
+        final long currentTimeMillis = System.currentTimeMillis();
+        if (currentTimeMillis > timeoutMillis) {
+            canceled.set(true);
+            logger.warn("Algorithm exceeds time limit.");
+            return true;
+        }
+        return false;
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/BPUSCalculator.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/BPUSCalculator.java
new file mode 100644
index 0000000000..689c1d682c
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/BPUSCalculator.java
@@ -0,0 +1,175 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm;
+
+import java.math.BigInteger;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+public class BPUSCalculator implements BenefitPolicy {
+    private static Logger logger = LoggerFactory.getLogger(BPUSCalculator.class);
+
+    protected final CuboidStats cuboidStats;
+    protected final ImmutableMap<BigInteger, Long> initCuboidAggCostMap;
+    protected final Map<BigInteger, Long> processCuboidAggCostMap;
+
+    public BPUSCalculator(CuboidStats cuboidStats) {
+        this.cuboidStats = cuboidStats;
+        this.initCuboidAggCostMap = ImmutableMap.copyOf(initCuboidAggCostMap());
+        this.processCuboidAggCostMap = Maps.newHashMap(initCuboidAggCostMap);
+    }
+
+    protected BPUSCalculator(CuboidStats cuboidStats, ImmutableMap<BigInteger, Long> initCuboidAggCostMap) {
+        this.cuboidStats = cuboidStats;
+        this.initCuboidAggCostMap = initCuboidAggCostMap;
+        this.processCuboidAggCostMap = Maps.newHashMap(initCuboidAggCostMap);
+    }
+
+    private Map<BigInteger, Long> initCuboidAggCostMap() {
+        Map<BigInteger, Long> cuboidAggCostMap = Maps.newHashMap();
+        //Initialize stats for mandatory cuboids
+        for (BigInteger cuboid : cuboidStats.getAllCuboidsForMandatory()) {
+            if (getCuboidCost(cuboid) != null) {
+                cuboidAggCostMap.put(cuboid, getCuboidCost(cuboid));
+            }
+        }
+
+        //Initialize stats for selection cuboids
+        long baseCuboidCost = getCuboidCost(cuboidStats.getBaseCuboid());
+        for (BigInteger cuboid : cuboidStats.getAllCuboidsForSelection()) {
+            long leastCost = baseCuboidCost;
+            for (Map.Entry<BigInteger, Long> cuboidTargetEntry : cuboidAggCostMap.entrySet()) {
+                // use the equal to check two value
+                if ((cuboid.or(cuboidTargetEntry.getKey())).equals(cuboidTargetEntry.getKey())) {
+                    if (leastCost > cuboidTargetEntry.getValue()) {
+                        leastCost = cuboidTargetEntry.getValue();
+                    }
+                }
+            }
+            cuboidAggCostMap.put(cuboid, leastCost);
+        }
+        return cuboidAggCostMap;
+    }
+
+    @Override
+    public CuboidBenefitModel.BenefitModel calculateBenefit(BigInteger cuboid, Set<BigInteger> selected) {
+        double totalCostSaving = 0;
+        int benefitCount = 0;
+        for (BigInteger descendant : cuboidStats.getAllDescendants(cuboid)) {
+            if (!selected.contains(descendant)) {
+                double costSaving = getCostSaving(descendant, cuboid);
+                if (costSaving > 0) {
+                    totalCostSaving += costSaving;
+                    benefitCount++;
+                }
+            }
+        }
+
+        double spaceCost = calculateSpaceCost(cuboid);
+        double benefitPerUnitSpace = totalCostSaving / spaceCost;
+        return new CuboidBenefitModel.BenefitModel(benefitPerUnitSpace, benefitCount);
+    }
+
+    @Override
+    public CuboidBenefitModel.BenefitModel calculateBenefitTotal(Set<BigInteger> cuboidsToAdd,
+            Set<BigInteger> selected) {
+        Set<BigInteger> selectedInner = Sets.newHashSet(selected);
+        Map<BigInteger, Long> cuboidAggCostMapCopy = Maps.newHashMap(processCuboidAggCostMap);
+        for (BigInteger cuboid : cuboidsToAdd) {
+            selectedInner.add(cuboid);
+            propagateAggregationCost(cuboid, selectedInner, cuboidAggCostMapCopy);
+        }
+        double totalCostSaving = 0;
+        int benefitCount = 0;
+        for (Map.Entry<BigInteger, Long> entry : cuboidAggCostMapCopy.entrySet()) {
+            if (entry.getValue() < processCuboidAggCostMap.get(entry.getKey())) {
+                totalCostSaving += processCuboidAggCostMap.get(entry.getKey()) - entry.getValue();
+                benefitCount++;
+            }
+        }
+
+        double benefitPerUnitSpace = totalCostSaving;
+        return new CuboidBenefitModel.BenefitModel(benefitPerUnitSpace, benefitCount);
+    }
+
+    protected double getCostSaving(BigInteger descendant, BigInteger cuboid) {
+        long cuboidCost = getCuboidCost(cuboid);
+        long descendantAggCost = getCuboidAggregationCost(descendant);
+        return (double) descendantAggCost - cuboidCost;
+    }
+
+    protected Long getCuboidCost(BigInteger cuboid) {
+        return cuboidStats.getCuboidCount(cuboid);
+    }
+
+    private long getCuboidAggregationCost(BigInteger cuboid) {
+        return processCuboidAggCostMap.get(cuboid);
+    }
+
+    @Override
+    public boolean ifEfficient(CuboidBenefitModel best) {
+        if (best.getBenefit() < getMinBenefitRatio()) {
+            logger.info(String.format(Locale.ROOT, "The recommended cuboid %s doesn't meet minimum benifit ratio %f",
+                    best, getMinBenefitRatio()));
+            return false;
+        }
+        return true;
+    }
+
+    public double getMinBenefitRatio() {
+        return cuboidStats.getBpusMinBenefitRatio();
+    }
+
+    @Override
+    public void propagateAggregationCost(BigInteger cuboid, Set<BigInteger> selected) {
+        propagateAggregationCost(cuboid, selected, processCuboidAggCostMap);
+    }
+
+    private void propagateAggregationCost(BigInteger cuboid, Set<BigInteger> selected,
+            Map<BigInteger, Long> processCuboidAggCostMap) {
+        long aggregationCost = getCuboidCost(cuboid);
+        Set<BigInteger> childrenCuboids = cuboidStats.getAllDescendants(cuboid);
+        for (BigInteger child : childrenCuboids) {
+            if (!selected.contains(child) && (aggregationCost < getCuboidAggregationCost(child))) {
+                processCuboidAggCostMap.put(child, aggregationCost);
+            }
+        }
+    }
+
+    /**
+     * Return the space cost of building a cuboid.
+     *
+     */
+    public double calculateSpaceCost(BigInteger cuboid) {
+        return cuboidStats.getCuboidCount(cuboid);
+    }
+
+    @Override
+    public BenefitPolicy getInstance() {
+        return new BPUSCalculator(this.cuboidStats, this.initCuboidAggCostMap);
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/BenefitPolicy.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/BenefitPolicy.java
new file mode 100644
index 0000000000..0909f71e06
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/BenefitPolicy.java
@@ -0,0 +1,51 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm;
+
+import java.math.BigInteger;
+import java.util.Set;
+
+public interface BenefitPolicy {
+
+    /**
+     * @return a cloned instance with initial status
+     */
+    public BenefitPolicy getInstance();
+
+    /**
+     * @param selected should not be changed
+     *                 Will not change the inner instance status
+     */
+    public CuboidBenefitModel.BenefitModel calculateBenefit(BigInteger cuboid, Set<BigInteger> selected);
+
+    /**
+     * @param cuboidsToAdd should not be changed
+     * @param selected     should not be changed
+     *                     Will not change the inner instance status
+     */
+    public CuboidBenefitModel.BenefitModel calculateBenefitTotal(Set<BigInteger> cuboidsToAdd, Set<BigInteger> selected);
+
+    public boolean ifEfficient(CuboidBenefitModel best);
+
+    /**
+     * @param selected should not be changed
+     * Will update the inner instance status
+     */
+    public void propagateAggregationCost(BigInteger cuboid, Set<BigInteger> selected);
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/CuboidBenefitModel.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/CuboidBenefitModel.java
new file mode 100644
index 0000000000..bb666f106b
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/CuboidBenefitModel.java
@@ -0,0 +1,89 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm;
+
+import java.math.BigInteger;
+
+public class CuboidBenefitModel {
+    private CuboidModel cuboidModel;
+    private BenefitModel benefitModel;
+
+    public CuboidBenefitModel(CuboidModel cuboidModel, BenefitModel benefitModel) {
+        this.cuboidModel = cuboidModel;
+        this.benefitModel = benefitModel;
+    }
+
+    public void reset(CuboidModel cuboidModel, BenefitModel benefitModel) {
+        this.cuboidModel = cuboidModel;
+        this.benefitModel = benefitModel;
+    }
+
+    public BigInteger getCuboidId() {
+        return cuboidModel == null ? null : cuboidModel.cuboidId;
+    }
+
+    public Double getBenefit() {
+        return benefitModel == null ? null : benefitModel.benefit;
+    }
+
+    @Override
+    public String toString() {
+        return "CuboidBenefitModel [cuboidModel=" + cuboidModel + ", benefitModel=" + benefitModel + "]";
+    }
+
+    public static class CuboidModel {
+        public final BigInteger cuboidId;
+
+        public final long recordCount;
+        public final double spaceSize;
+
+        public final double hitProbability;
+        public final long scanCount;
+
+        public CuboidModel(BigInteger cuboId, long recordCount, double spaceSize, double hitProbability,
+                long scanCount) {
+            this.cuboidId = cuboId;
+            this.recordCount = recordCount;
+            this.spaceSize = spaceSize;
+            this.hitProbability = hitProbability;
+            this.scanCount = scanCount;
+        }
+
+        @Override
+        public String toString() {
+            return "CuboidModel [cuboidId=" + cuboidId + ", recordCount=" + recordCount + ", spaceSize=" + spaceSize
+                    + ", hitProbability=" + hitProbability + ", scanCount=" + scanCount + "]";
+        }
+    }
+
+    public static class BenefitModel {
+        public final double benefit;
+        public final int benefitCount;
+
+        public BenefitModel(double benefit, int benefitCount) {
+            this.benefit = benefit;
+            this.benefitCount = benefitCount;
+        }
+
+        @Override
+        public String toString() {
+            return "BenefitModel [benefit=" + benefit + ", benefitCount=" + benefitCount + "]";
+        }
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/CuboidRecommendAlgorithm.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/CuboidRecommendAlgorithm.java
new file mode 100644
index 0000000000..e93e8e6f1d
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/CuboidRecommendAlgorithm.java
@@ -0,0 +1,50 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm;
+
+import java.math.BigInteger;
+import java.util.List;
+
+/**
+ * Algorithm to calculate the cuboid benefit and recommend cost-effective cuboid list based on the cube statistics.
+ */
+public interface CuboidRecommendAlgorithm {
+
+    /**
+     * Return a list of recommended cuboids for the building segment based on expansionRate
+     *
+     */
+    List<BigInteger> recommend(double expansionRate);
+
+    /**
+     * Start the Algorithm running
+     *
+     * @param maxSpaceLimit
+     * @return
+     */
+    List<BigInteger> start(double maxSpaceLimit);
+
+    /**
+     * Cancel the Algorithm running
+     *
+     *  Users can call this method from another thread to can the Algorithm and return the result earlier
+     *
+     */
+    void cancel();
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/CuboidStats.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/CuboidStats.java
new file mode 100644
index 0000000000..68857b5da7
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/CuboidStats.java
@@ -0,0 +1,302 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm;
+
+import java.math.BigInteger;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kylin.common.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+public class CuboidStats {
+    private static final Logger logger = LoggerFactory.getLogger(CuboidStats.class);
+
+    static final double WEIGHT_FOR_UN_QUERY = 0.2;
+    static final double BPUS_MIN_BENEFIT_RATIO = 0.001;
+
+    public static class Builder {
+
+        private static final long THRESHOLD_ROLL_UP_FOR_MANDATORY = 1000L;
+
+        // Required parameters
+        private String key;
+        private BigInteger nTotalCuboids;
+        private BigInteger baseCuboid;
+        private double queryUncertaintyRatio = WEIGHT_FOR_UN_QUERY;
+        private double bpusMinBenefitRatio = BPUS_MIN_BENEFIT_RATIO;
+        private Map<BigInteger, Long> statistics;
+        private Map<BigInteger, Double> size;
+
+        // Optional parameters - initialized to default values
+        private Set<BigInteger> mandatoryCuboids = null;
+        //// These two properties are for generating mandatory cuboids
+        private Map<BigInteger, Map<BigInteger, Pair<Long, Long>>> rollingUpCountSourceMap = null;
+
+        private Map<BigInteger, Long> hitFrequencyMap = null;
+        private Map<BigInteger, Map<BigInteger, Long>> scanCountSourceMap = null;
+
+        public Builder(String key, BigInteger nTotalCuboids, BigInteger baseCuboid, Map<BigInteger, Long> statistics,
+                Map<BigInteger, Double> size) {
+            this.key = key;
+            this.nTotalCuboids = nTotalCuboids;
+            this.baseCuboid = baseCuboid;
+            this.statistics = statistics;
+            this.size = size;
+        }
+
+        public Builder setQueryUncertaintyRatio(double queryUncertaintyRatio) {
+            this.queryUncertaintyRatio = queryUncertaintyRatio;
+            return this;
+        }
+
+        public Builder setBPUSMinBenefitRatio(double bpusMinBenefitRatio) {
+            this.bpusMinBenefitRatio = bpusMinBenefitRatio;
+            return this;
+        }
+
+        public Builder setRollingUpCountSourceMap(
+                Map<BigInteger, Map<BigInteger, Pair<Long, Long>>> rollingUpCountSourceMap) {
+            this.rollingUpCountSourceMap = rollingUpCountSourceMap;
+            return this;
+        }
+
+        public Builder setMandatoryCuboids(Set<BigInteger> mandatoryCuboids) {
+            this.mandatoryCuboids = mandatoryCuboids;
+            return this;
+        }
+
+        public Builder setHitFrequencyMap(Map<BigInteger, Long> hitFrequencyMap) {
+            this.hitFrequencyMap = hitFrequencyMap;
+            return this;
+        }
+
+        public Builder setScanCountSourceMap(Map<BigInteger, Map<BigInteger, Long>> scanCountSourceMap) {
+            this.scanCountSourceMap = scanCountSourceMap;
+            return this;
+        }
+
+        public Map<BigInteger, Double> estimateCuboidsSize(Map<BigInteger, Long> statistics) {
+            return null;
+        }
+
+        public CuboidStats build() {
+            Preconditions.checkNotNull(key, "key should not be null");
+            Preconditions.checkNotNull(baseCuboid, "baseCuboid should not be null");
+            Preconditions.checkNotNull(statistics, "statistics should not be null");
+            Preconditions.checkNotNull(size, "size should not be null");
+            Preconditions.checkNotNull(statistics.get(baseCuboid),
+                    "row count should exist for base cuboid " + baseCuboid);
+            Preconditions.checkState(statistics.keySet().equals(size.keySet()),
+                    "statistics & size should own the same key set");
+            statistics = CuboidStatsUtil.adjustCuboidStats(statistics);
+
+            if (hitFrequencyMap != null && rollingUpCountSourceMap != null) {
+                Map<BigInteger, Double> cuboidHitProbabilityMap = CuboidStatsUtil.calculateCuboidHitProbability(
+                        hitFrequencyMap.keySet(), hitFrequencyMap, nTotalCuboids, queryUncertaintyRatio);
+                Map<BigInteger, Long> srcCuboidsStats = CuboidStatsUtil.generateSourceCuboidStats(statistics,
+                        cuboidHitProbabilityMap, rollingUpCountSourceMap);
+
+                statistics.putAll(srcCuboidsStats);
+
+                Map<BigInteger, Double> estimatedSize = estimateCuboidsSize(statistics);
+                if (estimatedSize != null && !estimatedSize.isEmpty()) {
+                    size = Maps.newHashMap(estimatedSize);
+                }
+            }
+
+            if (mandatoryCuboids == null) {
+                mandatoryCuboids = Sets.newHashSet();
+            } else if (!mandatoryCuboids.isEmpty()) {
+                statistics.putAll(CuboidStatsUtil.complementRowCountForCuboids(statistics, mandatoryCuboids));
+            }
+
+            return new CuboidStats(key, baseCuboid, queryUncertaintyRatio, bpusMinBenefitRatio, mandatoryCuboids,
+                    statistics, size, hitFrequencyMap, scanCountSourceMap);
+        }
+    }
+
+    private String key;
+    private BigInteger baseCuboid;
+    private double bpusMinBenefitRatio;
+    private ImmutableSet<BigInteger> mandatoryCuboidSet;
+    private ImmutableSet<BigInteger> selectionCuboidSet;
+    private ImmutableMap<BigInteger, Long> cuboidCountMap;
+    private ImmutableMap<BigInteger, Double> cuboidSizeMap;
+    private ImmutableMap<BigInteger, Double> cuboidHitProbabilityMap;
+    private ImmutableMap<BigInteger, Long> cuboidScanCountMap;
+
+    private ImmutableMap<BigInteger, List<BigInteger>> directChildrenCache;
+    private Map<BigInteger, Set<BigInteger>> allDescendantsCache;
+
+    private CuboidStats(String key, BigInteger baseCuboidId, double queryUncertaintyRatio, double bpusMinBenefitRatio,
+            Set<BigInteger> mandatoryCuboids, Map<BigInteger, Long> statistics, Map<BigInteger, Double> size,
+            Map<BigInteger, Long> hitFrequencyMap, Map<BigInteger, Map<BigInteger, Long>> scanCountSourceMap) {
+
+        this.key = key;
+        this.baseCuboid = baseCuboidId;
+        this.bpusMinBenefitRatio = bpusMinBenefitRatio;
+        /** Initial mandatory cuboids */
+        Set<BigInteger> cuboidsForMandatory = Sets.newHashSet(mandatoryCuboids);
+        //Always add base cuboid.
+        if (!cuboidsForMandatory.contains(baseCuboid)) {
+            cuboidsForMandatory.add(baseCuboid);
+        }
+        logger.info("Mandatory cuboids: " + cuboidsForMandatory);
+
+        /** Initial selection cuboids */
+        Set<BigInteger> cuboidsForSelection = Sets.newHashSet(statistics.keySet());
+        cuboidsForSelection.removeAll(cuboidsForMandatory);
+
+        //There's no overlap between mandatoryCuboidSet and selectionCuboidSet
+        this.mandatoryCuboidSet = ImmutableSet.<BigInteger> builder().addAll(cuboidsForMandatory).build();
+        this.selectionCuboidSet = ImmutableSet.<BigInteger> builder().addAll(cuboidsForSelection).build();
+        if (selectionCuboidSet.isEmpty()) {
+            logger.warn("The selection set should not be empty!!!");
+        }
+
+        this.cuboidCountMap = ImmutableMap.<BigInteger, Long> builder().putAll(statistics).build();
+        this.cuboidSizeMap = ImmutableMap.<BigInteger, Double> builder().putAll(size).build();
+
+        /** Initialize the hit probability for each selection cuboid */
+        Map<BigInteger, Double> tmpCuboidHitProbabilityMap = CuboidStatsUtil.calculateCuboidHitProbability(
+                selectionCuboidSet, hitFrequencyMap, BigInteger.valueOf(selectionCuboidSet.size()),
+                queryUncertaintyRatio);
+        this.cuboidHitProbabilityMap = ImmutableMap.<BigInteger, Double> builder().putAll(tmpCuboidHitProbabilityMap)
+                .build();
+
+        /** Initialize the scan count when query for each selection cuboid + one base cuboid */
+        Map<BigInteger, Long> tmpCuboidScanCountMap = Maps.newHashMapWithExpectedSize(1 + selectionCuboidSet.size());
+        tmpCuboidScanCountMap.put(baseCuboid, getExpScanCount(baseCuboid, statistics, scanCountSourceMap));
+        for (BigInteger cuboid : selectionCuboidSet) {
+            tmpCuboidScanCountMap.put(cuboid, getExpScanCount(cuboid, statistics, scanCountSourceMap));
+        }
+        this.cuboidScanCountMap = ImmutableMap.<BigInteger, Long> builder().putAll(tmpCuboidScanCountMap).build();
+
+        this.directChildrenCache = ImmutableMap.<BigInteger, List<BigInteger>> builder()
+                .putAll(CuboidStatsUtil.createDirectChildrenCache(statistics.keySet())).build();
+
+        this.allDescendantsCache = Maps.newConcurrentMap();
+    }
+
+    private long getExpScanCount(BigInteger sourceCuboid, Map<BigInteger, Long> statistics,
+            Map<BigInteger, Map<BigInteger, Long>> scanCountSourceMap) {
+        Preconditions.checkNotNull(statistics.get(sourceCuboid),
+                "The statistics for source cuboid " + sourceCuboid + " does not exist!!!");
+        if (scanCountSourceMap == null || scanCountSourceMap.get(sourceCuboid) == null
+                || scanCountSourceMap.get(sourceCuboid).size() <= 0) {
+            return statistics.get(sourceCuboid);
+        } else {
+            Map<BigInteger, Long> scanCountTargetMap = scanCountSourceMap.get(sourceCuboid);
+            long totalEstScanCount = 0L;
+            for (Map.Entry<BigInteger, Long> subEntry : scanCountTargetMap.entrySet()) {
+                BigInteger targetCuboid = subEntry.getKey();
+                Preconditions.checkNotNull(statistics.get(targetCuboid),
+                        "The statistics for target cuboid " + targetCuboid + " does not exist!!!");
+                // Consider the ratio of row count between source cuboid and target cuboid
+                totalEstScanCount += subEntry.getValue() * statistics.get(sourceCuboid) / statistics.get(targetCuboid);
+            }
+            return totalEstScanCount / scanCountTargetMap.size();
+        }
+    }
+
+    public double getBpusMinBenefitRatio() {
+        return bpusMinBenefitRatio;
+    }
+
+    public Set<BigInteger> getAllDescendants(BigInteger cuboid) {
+        Set<BigInteger> allDescendants = Sets.newLinkedHashSet();
+        if (selectionCuboidSet.contains(cuboid)) {
+            if (allDescendantsCache.get(cuboid) != null) {
+                return allDescendantsCache.get(cuboid);
+            } else {
+                getAllDescendants(cuboid, allDescendants);
+                allDescendantsCache.put(cuboid, allDescendants);
+            }
+        }
+        return allDescendants;
+    }
+
+    private void getAllDescendants(BigInteger cuboid, Set<BigInteger> allDescendants) {
+        if (allDescendants.contains(cuboid)) {
+            return;
+        }
+        allDescendants.add(cuboid);
+        for (BigInteger directChild : directChildrenCache.get(cuboid)) {
+            getAllDescendants(directChild, allDescendants);
+        }
+    }
+
+    public ImmutableSet<BigInteger> getAllCuboidsForSelection() {
+        return selectionCuboidSet;
+    }
+
+    public ImmutableSet<BigInteger> getAllCuboidsForMandatory() {
+        return mandatoryCuboidSet;
+    }
+
+    public Long getCuboidQueryCost(BigInteger cuboid) {
+        return cuboidScanCountMap.get(cuboid);
+    }
+
+    public Long getCuboidCount(BigInteger cuboid) {
+        return cuboidCountMap.get(cuboid);
+    }
+
+    public Double getCuboidSize(BigInteger cuboid) {
+        return cuboidSizeMap.get(cuboid);
+    }
+
+    public double getCuboidHitProbability(BigInteger cuboid) {
+        if (mandatoryCuboidSet.contains(cuboid)) {
+            return 1;
+        } else {
+            return cuboidHitProbabilityMap.get(cuboid) == null ? 0 : cuboidHitProbabilityMap.get(cuboid);
+        }
+    }
+
+    public Map<BigInteger, Long> getStatistics() {
+        return cuboidCountMap;
+    }
+
+    public double getBaseCuboidSize() {
+        return getCuboidSize(baseCuboid);
+    }
+
+    public BigInteger getBaseCuboid() {
+        return baseCuboid;
+    }
+
+    public String getKey() {
+        return key;
+    }
+
+    public CuboidBenefitModel.CuboidModel getCuboidModel(BigInteger cuboid) {
+        return new CuboidBenefitModel.CuboidModel(cuboid, getCuboidCount(cuboid), getCuboidSize(cuboid),
+                getCuboidHitProbability(cuboid), getCuboidQueryCost(cuboid));
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/CuboidStatsUtil.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/CuboidStatsUtil.java
new file mode 100644
index 0000000000..7b4240545b
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/CuboidStatsUtil.java
@@ -0,0 +1,371 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.kylin.common.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class CuboidStatsUtil {
+
+    private static final Logger logger = LoggerFactory.getLogger(CuboidStatsUtil.class);
+
+    private CuboidStatsUtil() {
+        throw new IllegalStateException("Class CuboidStatsUtil is an utility class !");
+    }
+
+    /**
+     * According to the cuboid hit frequencies and query uncertainty ratio
+     * calculate each cuboid hit probability
+     * @param selectionCuboidSet subset of cuboid domain which needs probability
+     * @param nTotalCuboids number of cuboids needs to be considered, mainly for each cuboid's uncertainty weight
+     * */
+    public static Map<BigInteger, Double> calculateCuboidHitProbability(Set<BigInteger> selectionCuboidSet,
+            Map<BigInteger, Long> hitFrequencyMap, BigInteger nTotalCuboids, double queryUncertaintyRatio) {
+        Map<BigInteger, Double> cuboidHitProbabilityMap = Maps.newHashMapWithExpectedSize(selectionCuboidSet.size());
+        if (hitFrequencyMap == null || hitFrequencyMap.isEmpty()) {
+            double value = BigDecimal.valueOf(1.0).divide(new BigDecimal(nTotalCuboids), 15, BigDecimal.ROUND_HALF_EVEN)
+                    .doubleValue();
+            for (BigInteger cuboid : selectionCuboidSet) {
+                cuboidHitProbabilityMap.put(cuboid, value);
+            }
+        } else {
+            long totalHitFrequency = 0L;
+            for (Map.Entry<BigInteger, Long> hitFrequency : hitFrequencyMap.entrySet()) {
+                totalHitFrequency += hitFrequency.getValue();
+            }
+
+            final double unitUncertainProb = BigDecimal.valueOf(queryUncertaintyRatio)
+                    .divide(new BigDecimal(nTotalCuboids), 15, BigDecimal.ROUND_HALF_EVEN).doubleValue();
+            for (BigInteger cuboid : selectionCuboidSet) {
+                //Calculate hit probability for each cuboid
+                if (hitFrequencyMap.get(cuboid) != null) {
+                    if (totalHitFrequency != 0)
+                        cuboidHitProbabilityMap.put(cuboid, unitUncertainProb
+                                + (1 - queryUncertaintyRatio) * hitFrequencyMap.get(cuboid) / totalHitFrequency);
+                    else
+                        throw new ArithmeticException("/ by zero");
+                } else {
+                    cuboidHitProbabilityMap.put(cuboid, unitUncertainProb);
+                }
+            }
+        }
+
+        return cuboidHitProbabilityMap;
+    }
+
+    /**
+     * @param statistics for cuboid row count
+     * @param rollingUpSourceMap the key of the outer map is source cuboid,
+     *                           the key of the inner map is target cuboid,
+     *                                  if cube is optimized multiple times, target cuboid may change
+     *                           the first element of the pair is the rollup row count
+     *                           the second element of the pair is the return row count
+     * @return source cuboids with estimated row count
+     */
+    public static Map<BigInteger, Long> generateSourceCuboidStats(Map<BigInteger, Long> statistics,
+            Map<BigInteger, Double> cuboidHitProbabilityMap,
+            Map<BigInteger, Map<BigInteger, Pair<Long, Long>>> rollingUpSourceMap) {
+        Map<BigInteger, Long> srcCuboidsStats = Maps.newHashMap();
+        if (cuboidHitProbabilityMap == null || cuboidHitProbabilityMap.isEmpty() || rollingUpSourceMap == null
+                || rollingUpSourceMap.isEmpty()) {
+            return srcCuboidsStats;
+        }
+
+        for (BigInteger cuboid : cuboidHitProbabilityMap.keySet()) {
+            if (statistics.get(cuboid) != null) {
+                continue;
+            }
+            Map<BigInteger, Pair<Long, Long>> innerRollingUpTargetMap = rollingUpSourceMap.get(cuboid);
+            if (innerRollingUpTargetMap == null || innerRollingUpTargetMap.isEmpty()) {
+                continue;
+            }
+
+            long totalEstRowCount = 0L;
+            int nEffective = 0;
+            boolean ifHasStats = false;
+            // if ifHasStats equals true, then source cuboid row count = (1 - rollup ratio) * target cuboid row count
+            //                            else source cuboid row count = returned row count collected directly
+            for (BigInteger tgtCuboid : innerRollingUpTargetMap.keySet()) {
+                Pair<Long, Long> rollingupStats = innerRollingUpTargetMap.get(tgtCuboid);
+                if (statistics.get(tgtCuboid) != null) {
+                    if (!ifHasStats) {
+                        totalEstRowCount = 0L;
+                        nEffective = 0;
+                        ifHasStats = true;
+                    }
+                    double rollupRatio = calculateRollupRatio(rollingupStats);
+                    totalEstRowCount += (1 - rollupRatio) * statistics.get(tgtCuboid);
+                    nEffective++;
+                } else {
+                    if (ifHasStats) {
+                        continue;
+                    }
+                    totalEstRowCount += rollingupStats.getSecond();
+                    nEffective++;
+                }
+            }
+
+            if (nEffective != 0)
+                srcCuboidsStats.put(cuboid, totalEstRowCount / nEffective);
+            else
+                throw new ArithmeticException("/ by zero");
+        }
+        srcCuboidsStats.remove(BigInteger.ZERO);
+        adjustCuboidStats(srcCuboidsStats, statistics);
+        return srcCuboidsStats;
+    }
+
+    /**
+     * Complement row count for mandatory cuboids
+     * with its best parent's row count
+     * */
+    public static Map<BigInteger, Long> complementRowCountForCuboids(final Map<BigInteger, Long> statistics,
+            Set<BigInteger> cuboids) {
+        Map<BigInteger, Long> result = Maps.newHashMapWithExpectedSize(cuboids.size());
+
+        // Sort entries order by row count asc
+        SortedSet<Map.Entry<BigInteger, Long>> sortedStatsSet = new TreeSet<>(
+                new Comparator<Map.Entry<BigInteger, Long>>() {
+                    public int compare(Map.Entry<BigInteger, Long> o1, Map.Entry<BigInteger, Long> o2) {
+                        int ret = o1.getValue().compareTo(o2.getValue());
+                        return ret == 0 ? o1.getKey().compareTo(o2.getKey()) : ret;
+                    }
+                });
+        //sortedStatsSet.addAll(statistics.entrySet()); KYLIN-3580
+        for (Map.Entry<BigInteger, Long> entry : statistics.entrySet()) {
+            sortedStatsSet.add(entry);
+        }
+        for (BigInteger cuboid : cuboids) {
+            if (statistics.get(cuboid) == null) {
+                // Get estimate row count for mandatory cuboid
+                for (Map.Entry<BigInteger, Long> entry : sortedStatsSet) {
+                    if (isDescendant(cuboid, entry.getKey())) {
+                        result.put(cuboid, entry.getValue());
+                        break;
+                    }
+                }
+            } else {
+                result.put(cuboid, statistics.get(cuboid));
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * adjust cuboid row count, make sure parent not less than child
+     */
+    public static Map<BigInteger, Long> adjustCuboidStats(Map<BigInteger, Long> statistics) {
+        Map<BigInteger, Long> ret = Maps.newHashMapWithExpectedSize(statistics.size());
+
+        List<BigInteger> cuboids = Lists.newArrayList(statistics.keySet());
+        Collections.sort(cuboids);
+
+        for (BigInteger cuboid : cuboids) {
+            Long rowCount = statistics.get(cuboid);
+            for (BigInteger childCuboid : ret.keySet()) {
+                if (isDescendant(childCuboid, cuboid)) {
+                    Long childRowCount = ret.get(childCuboid);
+                    if (rowCount < childRowCount) {
+                        rowCount = childRowCount;
+                    }
+                }
+            }
+            ret.put(cuboid, rowCount);
+        }
+
+        return ret;
+    }
+
+    public static void adjustCuboidStats(Map<BigInteger, Long> mandatoryCuboidsWithStats,
+            Map<BigInteger, Long> statistics) {
+        List<BigInteger> mandatoryCuboids = Lists.newArrayList(mandatoryCuboidsWithStats.keySet());
+        Collections.sort(mandatoryCuboids);
+
+        List<BigInteger> selectedCuboids = Lists.newArrayList(statistics.keySet());
+        Collections.sort(selectedCuboids);
+
+        for (int i = 0; i < mandatoryCuboids.size(); i++) {
+            BigInteger mCuboid = mandatoryCuboids.get(i);
+            if (statistics.get(mCuboid) != null) {
+                mandatoryCuboidsWithStats.put(mCuboid, statistics.get(mCuboid));
+                continue;
+            }
+            int k = 0;
+            // Make sure mCuboid's row count larger than its children's row count in statistics
+            for (; k < selectedCuboids.size(); k++) {
+                BigInteger sCuboid = selectedCuboids.get(k);
+                if (sCuboid.compareTo(mCuboid) > 0) {
+                    // sCuboid > mCuboid
+                    break;
+                }
+                if (isDescendant(sCuboid, mCuboid)) {
+                    Long childRowCount = statistics.get(sCuboid);
+                    if (childRowCount > mandatoryCuboidsWithStats.get(mCuboid)) {
+                        mandatoryCuboidsWithStats.put(mCuboid, childRowCount);
+                    }
+                }
+            }
+            // Make sure mCuboid's row count larger than its children's row count in mandatoryCuboids
+            for (int j = 0; j < i; j++) {
+                BigInteger cCuboid = mandatoryCuboids.get(j);
+                if (isDescendant(cCuboid, mCuboid)) {
+                    Long childRowCount = mandatoryCuboidsWithStats.get(cCuboid);
+                    if (childRowCount > mandatoryCuboidsWithStats.get(mCuboid)) {
+                        mandatoryCuboidsWithStats.put(mCuboid, childRowCount);
+                    }
+                }
+            }
+            // Make sure mCuboid's row count lower than its parents' row count in statistics
+            for (; k < selectedCuboids.size(); k++) {
+                BigInteger sCuboid = selectedCuboids.get(k);
+                if (isDescendant(mCuboid, sCuboid)) {
+                    Long parentRowCount = statistics.get(sCuboid);
+                    if (parentRowCount < mandatoryCuboidsWithStats.get(mCuboid)) {
+                        mandatoryCuboidsWithStats.put(mCuboid, parentRowCount);
+                    }
+                }
+            }
+        }
+    }
+
+    public static Map<BigInteger, List<BigInteger>> createDirectChildrenCache(final Set<BigInteger> cuboidSet) {
+        /**
+         * Sort the list by ascending order:
+         * */
+        final List<BigInteger> cuboidList = Lists.newArrayList(cuboidSet);
+        Collections.sort(cuboidList);
+        /**
+         * Sort the list by ascending order:
+         * 1. the more bit count of its value, the bigger
+         * 2. the larger of its value, the bigger
+         * */
+        List<Integer> layerIdxList = Lists.newArrayListWithExpectedSize(cuboidList.size());
+        for (int i = 0; i < cuboidList.size(); i++) {
+            layerIdxList.add(i);
+        }
+        Collections.sort(layerIdxList, new Comparator<Integer>() {
+            @Override
+            public int compare(Integer i1, Integer i2) {
+                BigInteger o1 = cuboidList.get(i1);
+                BigInteger o2 = cuboidList.get(i2);
+                int nBitDiff = o1.bitCount() - o2.bitCount();
+                if (nBitDiff != 0) {
+                    return nBitDiff;
+                }
+                return o1.compareTo(o2);
+            }
+        });
+        /**
+         * Construct an index array for pointing the position in layerIdxList
+         * (layerCuboidList is for speeding up continuous iteration)
+         * */
+        int[] toLayerIdxArray = new int[layerIdxList.size()];
+        final List<BigInteger> layerCuboidList = Lists.newArrayListWithExpectedSize(cuboidList.size());
+        for (int i = 0; i < layerIdxList.size(); i++) {
+            int cuboidIdx = layerIdxList.get(i);
+            toLayerIdxArray[cuboidIdx] = i;
+            layerCuboidList.add(cuboidList.get(cuboidIdx));
+        }
+
+        int[] previousLayerLastIdxArray = new int[layerIdxList.size()];
+        int currentBitCount = 0;
+        int previousLayerLastIdx = -1;
+        for (int i = 0; i < layerIdxList.size(); i++) {
+            int cuboidIdx = layerIdxList.get(i);
+            // get bit count from the biginteger api
+            int nBits = cuboidList.get(cuboidIdx).bitCount();
+            if (nBits > currentBitCount) {
+                currentBitCount = nBits;
+                previousLayerLastIdx = i - 1;
+            }
+            previousLayerLastIdxArray[i] = previousLayerLastIdx;
+        }
+
+        Map<BigInteger, List<BigInteger>> directChildrenCache = Maps.newHashMap();
+        for (int i = 0; i < cuboidList.size(); i++) {
+            BigInteger currentCuboid = cuboidList.get(i);
+            LinkedList<BigInteger> directChildren = Lists.newLinkedList();
+            int lastLayerIdx = previousLayerLastIdxArray[toLayerIdxArray[i]];
+            /**
+             * Choose one of the two scan strategies
+             * 1. cuboids are sorted by its value, like 1,2,3,4,...
+             * 2. cuboids are layered and sorted, like 1,2,4,8,...,3,5,...
+             * */
+            if (i - 1 <= lastLayerIdx) {
+                /**
+                 * 1. Adding cuboid by descending order
+                 * */
+                for (int j = i - 1; j >= 0; j--) {
+                    checkAndAddDirectChild(directChildren, currentCuboid, cuboidList.get(j));
+                }
+            } else {
+                /**
+                 * 1. Adding cuboid by descending order
+                 * 2. Check from lower cuboid layer
+                 * */
+                for (int j = lastLayerIdx; j >= 0; j--) {
+                    checkAndAddDirectChild(directChildren, currentCuboid, layerCuboidList.get(j));
+                }
+            }
+            directChildrenCache.put(currentCuboid, directChildren);
+        }
+        return directChildrenCache;
+    }
+
+    private static void checkAndAddDirectChild(List<BigInteger> directChildren, BigInteger currentCuboid,
+            BigInteger checkedCuboid) {
+        if (isDescendant(checkedCuboid, currentCuboid)) {
+            boolean ifDirectChild = true;
+            for (BigInteger directChild : directChildren) {
+                if (isDescendant(checkedCuboid, directChild)) {
+                    ifDirectChild = false;
+                    break;
+                }
+            }
+            if (ifDirectChild) {
+                directChildren.add(checkedCuboid);
+            }
+        }
+    }
+
+    private static boolean isDescendant(BigInteger cuboidToCheck, BigInteger parentCuboid) {
+        return (cuboidToCheck.and(parentCuboid)).equals(cuboidToCheck);
+    }
+
+    private static double calculateRollupRatio(Pair<Long, Long> rollupStats) {
+        double rollupInputCount = (double) rollupStats.getFirst() + rollupStats.getSecond();
+        return rollupInputCount == 0 ? 0 : 1.0 * rollupStats.getFirst() / rollupInputCount;
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/PBPUSCalculator.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/PBPUSCalculator.java
new file mode 100644
index 0000000000..68bacf5a3b
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/PBPUSCalculator.java
@@ -0,0 +1,55 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm;
+
+import java.math.BigInteger;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+
+public class PBPUSCalculator extends BPUSCalculator {
+
+    public PBPUSCalculator(final CuboidStats cuboidStats) {
+        super(cuboidStats);
+    }
+
+    protected PBPUSCalculator(CuboidStats cuboidStats, ImmutableMap<BigInteger, Long> initCuboidAggCostMap) {
+        super(cuboidStats, initCuboidAggCostMap);
+    }
+
+    @Override
+    protected double getCostSaving(BigInteger descendant, BigInteger cuboid) {
+        return getCuboidHitProbability(descendant) * super.getCostSaving(descendant, cuboid);
+    }
+
+    protected double getCuboidHitProbability(BigInteger cuboid) {
+        return cuboidStats.getCuboidHitProbability(cuboid);
+    }
+
+    public double getMinBenefitRatio() {
+        int cuboidDomainSize = cuboidStats.getAllCuboidsForSelection().size();
+        Preconditions.checkArgument(cuboidDomainSize > 0, "The set of cuboids for selection is empty!!!");
+        return super.getMinBenefitRatio() / cuboidDomainSize;
+    }
+
+    @Override
+    public BenefitPolicy getInstance() {
+        return new PBPUSCalculator(this.cuboidStats, this.initCuboidAggCostMap);
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/SPBPUSCalculator.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/SPBPUSCalculator.java
new file mode 100644
index 0000000000..fae674cc77
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/SPBPUSCalculator.java
@@ -0,0 +1,44 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm;
+
+import java.math.BigInteger;
+
+import com.google.common.collect.ImmutableMap;
+
+public class SPBPUSCalculator extends PBPUSCalculator {
+
+    public SPBPUSCalculator(final CuboidStats cuboidStats) {
+        super(cuboidStats);
+    }
+
+    protected SPBPUSCalculator(CuboidStats cuboidStats, ImmutableMap<BigInteger, Long> initCuboidAggCostMap) {
+        super(cuboidStats, initCuboidAggCostMap);
+    }
+
+    @Override
+    protected Long getCuboidCost(BigInteger cuboid) {
+        return cuboidStats.getCuboidQueryCost(cuboid);
+    }
+
+    @Override
+    public BenefitPolicy getInstance() {
+        return new SPBPUSCalculator(this.cuboidStats, this.initCuboidAggCostMap);
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/BitsChromosome.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/BitsChromosome.java
new file mode 100644
index 0000000000..3e2e85c4a7
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/BitsChromosome.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.metadata.cube.planner.algorithm.genetic;
+
+import java.math.BigInteger;
+import java.util.BitSet;
+
+import org.apache.commons.math3.genetics.Chromosome;
+import org.apache.kylin.metadata.cube.planner.algorithm.BenefitPolicy;
+import org.apache.kylin.metadata.cube.planner.algorithm.CuboidBenefitModel;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
+public class BitsChromosome extends Chromosome {
+
+    /**
+     * Global unmodified
+     */
+    private final BitsChromosomeHelper helper;
+
+    /**
+     * BitSet representing the chromosome
+     */
+    private final BitSet representation;
+    private final ImmutableSet<BigInteger> cuboids;
+
+    private final BenefitPolicy benefitPolicy;
+
+    private final double spaceCost;
+
+    public BitsChromosome(final BitSet representation, final BenefitPolicy benefitPolicy, BitsChromosomeHelper helper) {
+        this.helper = helper;
+
+        this.representation = representation;
+        this.cuboids = ImmutableSet.copyOf(helper.toCuboidList(representation));
+
+        this.benefitPolicy = benefitPolicy;
+
+        this.spaceCost = helper.getCuboidSize(Sets.newHashSet(cuboids));
+    }
+
+    public BitsChromosome newBitsChromosome(BitSet newRepresentation) {
+        return new BitsChromosome(newRepresentation, benefitPolicy.getInstance(), helper);
+    }
+
+    public BitSet getRepresentation() {
+        return representation;
+    }
+
+    /**
+     * Returns the length of the chromosome.
+     *
+     * @return the length of the chromosome
+     */
+    public int getLength() {
+        return helper.getLength();
+    }
+
+    public ImmutableSet<BigInteger> getCuboids() {
+        return cuboids;
+    }
+
+    @Override
+    public synchronized double fitness() {
+        CuboidBenefitModel.BenefitModel benefitModel = benefitPolicy.calculateBenefitTotal(cuboids,
+                helper.getMandatoryCuboids());
+        double totalBenefit = benefitModel.benefit;
+        if (spaceCost > helper.spaceLimit) {
+            totalBenefit = totalBenefit * helper.spaceLimit / spaceCost;
+        }
+        return totalBenefit;
+    }
+
+    @Override
+    protected boolean isSame(final Chromosome another) {
+        return this.equals(another);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        BitsChromosome that = (BitsChromosome) o;
+
+        if (helper != null ? !helper.equals(that.helper) : that.helper != null)
+            return false;
+        return representation != null ? representation.equals(that.representation) : that.representation == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = helper != null ? helper.hashCode() : 0;
+        result = 31 * result + (representation != null ? representation.hashCode() : 0);
+        return result;
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/BitsChromosomeHelper.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/BitsChromosomeHelper.java
new file mode 100644
index 0000000000..5e305878fc
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/BitsChromosomeHelper.java
@@ -0,0 +1,122 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm.genetic;
+
+import java.math.BigInteger;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.kylin.metadata.cube.planner.algorithm.CuboidStats;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+public class BitsChromosomeHelper {
+
+    public final double spaceLimit;
+    private final CuboidStats cuboidStats;
+    private final CuboidEncoder cuboidEncoder;
+
+    public BitsChromosomeHelper(final double spaceLimit, final CuboidStats cuboidStats) {
+        this.spaceLimit = spaceLimit;
+        this.cuboidStats = cuboidStats;
+        this.cuboidEncoder = new CuboidEncoder(cuboidStats.getAllCuboidsForSelection());
+    }
+
+    public ImmutableSet<BigInteger> getMandatoryCuboids() {
+        return cuboidStats.getAllCuboidsForMandatory();
+    }
+
+    public List<BigInteger> toCuboidList(BitSet bits) {
+        return cuboidEncoder.toCuboidList(bits);
+    }
+
+    public double getCuboidSize(Set<BigInteger> cuboids) {
+        double ret = 0;
+        for (BigInteger cuboid : cuboids) {
+            ret += cuboidStats.getCuboidSize(cuboid);
+        }
+        return ret;
+    }
+
+    public double getCuboidSizeByBitIndex(int index) {
+        return cuboidStats.getCuboidSize(cuboidEncoder.cuboidDomain.get(index));
+    }
+
+    public int getLength() {
+        return cuboidEncoder.cuboidDomain.size();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        BitsChromosomeHelper that = (BitsChromosomeHelper) o;
+
+        return cuboidEncoder != null ? cuboidEncoder.equals(that.cuboidEncoder) : that.cuboidEncoder == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        return cuboidEncoder != null ? cuboidEncoder.hashCode() : 0;
+    }
+
+    private static class CuboidEncoder {
+        public final ImmutableList<BigInteger> cuboidDomain;
+
+        public CuboidEncoder(Set<BigInteger> cuboidSet) {
+            List<BigInteger> cuboidList = Lists.newArrayList(cuboidSet);
+            Collections.sort(cuboidList, Collections.reverseOrder());
+            this.cuboidDomain = ImmutableList.copyOf(cuboidList);
+        }
+
+        public List<BigInteger> toCuboidList(BitSet bits) {
+            List<BigInteger> cuboids = Lists.newArrayListWithExpectedSize(bits.cardinality());
+            for (int i = bits.nextSetBit(0); i >= 0; i = bits.nextSetBit(i + 1)) {
+                cuboids.add(cuboidDomain.get(i));
+            }
+            return cuboids;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            CuboidEncoder that = (CuboidEncoder) o;
+
+            return cuboidDomain != null ? cuboidDomain.equals(that.cuboidDomain) : that.cuboidDomain == null;
+
+        }
+
+        @Override
+        public int hashCode() {
+            return cuboidDomain != null ? cuboidDomain.hashCode() : 0;
+        }
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/BitsMutation.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/BitsMutation.java
new file mode 100644
index 0000000000..1a218b4109
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/BitsMutation.java
@@ -0,0 +1,59 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm.genetic;
+
+import java.util.BitSet;
+
+import org.apache.commons.math3.exception.MathIllegalArgumentException;
+import org.apache.commons.math3.exception.util.DummyLocalizable;
+import org.apache.commons.math3.genetics.Chromosome;
+import org.apache.commons.math3.genetics.GeneticAlgorithm;
+import org.apache.commons.math3.genetics.MutationPolicy;
+
+/**
+ * Modified from the BinaryMutation.java in https://github.com/apache/commons-math
+ * <p>
+ * Mutation for {@link BitsChromosome}s. Randomly changes one gene.
+ */
+public class BitsMutation implements MutationPolicy {
+
+    /**
+     * Mutate the given chromosome. Randomly changes one gene.
+     *
+     * @param original the original chromosome.
+     * @return the mutated chromosome.
+     * @throws IllegalArgumentException if <code>original</code> is not an instance of {@link BitsChromosome}.
+     */
+    public Chromosome mutate(Chromosome original) throws IllegalArgumentException {
+        if (!(original instanceof BitsChromosome)) {
+            throw new MathIllegalArgumentException(new DummyLocalizable("bits mutation only works on BitsChromosome"));
+        }
+
+        BitsChromosome origChrom = (BitsChromosome) original;
+        BitSet newNey = (BitSet) origChrom.getRepresentation().clone();
+
+        // randomly select a gene
+        int geneIndex = GeneticAlgorithm.getRandomGenerator().nextInt(origChrom.getLength());
+        // change it
+        newNey.set(geneIndex, !newNey.get(geneIndex));
+
+        Chromosome newChrom = origChrom.newBitsChromosome(newNey);
+        return newChrom;
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/BitsOnePointCrossover.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/BitsOnePointCrossover.java
new file mode 100644
index 0000000000..2087003dcb
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/BitsOnePointCrossover.java
@@ -0,0 +1,131 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm.genetic;
+
+import java.util.BitSet;
+
+import org.apache.commons.math3.exception.DimensionMismatchException;
+import org.apache.commons.math3.exception.MathIllegalArgumentException;
+import org.apache.commons.math3.exception.util.DummyLocalizable;
+import org.apache.commons.math3.genetics.Chromosome;
+import org.apache.commons.math3.genetics.ChromosomePair;
+import org.apache.commons.math3.genetics.CrossoverPolicy;
+import org.apache.commons.math3.genetics.GeneticAlgorithm;
+
+/**
+ * Modified from the OnePointCrossover.java in https://github.com/apache/commons-math
+ * <p>
+ * One point crossover policy. A random crossover point is selected and the
+ * first part from each parent is copied to the corresponding child, and the
+ * second parts are copied crosswise.
+ * <p>
+ * Example:
+ * <pre>
+ * -C- denotes a crossover point
+ *                   -C-                                 -C-
+ * p1 = (1 0 1 0 0 1  | 0 1 1)    X    p2 = (0 1 1 0 1 0  | 1 1 1)
+ *      \------------/ \-----/              \------------/ \-----/
+ *            ||         (*)                       ||        (**)
+ *            VV         (**)                      VV        (*)
+ *      /------------\ /-----\              /------------\ /-----\
+ * c1 = (1 0 1 0 0 1  | 1 1 1)    X    c2 = (0 1 1 0 1 0  | 0 1 1)
+ * </pre>
+ * <p>
+ * This policy works only on {@link BitsChromosome}, and therefore it
+ * is parameterized by T. Moreover, the chromosomes must have same lengths.
+ */
+public class BitsOnePointCrossover implements CrossoverPolicy {
+
+    /**
+     * Performs one point crossover. A random crossover point is selected and the
+     * first part from each parent is copied to the corresponding child, and the
+     * second parts are copied crosswise.
+     * <p>
+     * Example:
+     * <pre>
+     * -C- denotes a crossover point
+     *                   -C-                                 -C-
+     * p1 = (1 0 1 0 0 1  | 0 1 1)    X    p2 = (0 1 1 0 1 0  | 1 1 1)
+     *      \------------/ \-----/              \------------/ \-----/
+     *            ||         (*)                       ||        (**)
+     *            VV         (**)                      VV        (*)
+     *      /------------\ /-----\              /------------\ /-----\
+     * c1 = (1 0 1 0 0 1  | 1 1 1)    X    c2 = (0 1 1 0 1 0  | 0 1 1)
+     * </pre>
+     *
+     * @param first  first parent (p1)
+     * @param second second parent (p2)
+     * @return pair of two children (c1,c2)
+     * @throws IllegalArgumentException     if one of the chromosomes is
+     *                                      not an instance of {@link BitsChromosome}
+     * @throws MathIllegalArgumentException if the length of the two chromosomes is different
+     */
+    @SuppressWarnings("unchecked") // OK because of instanceof checks
+    public ChromosomePair crossover(final Chromosome first, final Chromosome second)
+            throws DimensionMismatchException, MathIllegalArgumentException {
+
+        if (!(first instanceof BitsChromosome && second instanceof BitsChromosome)) {
+            throw new MathIllegalArgumentException(
+                    new DummyLocalizable("bits one-point crossover only works on BitsChromosome"));
+        }
+        return crossover((BitsChromosome) first, (BitsChromosome) second);
+    }
+
+    /**
+     * Helper for {@link #crossover(Chromosome, Chromosome)}. Performs the actual crossover.
+     *
+     * @param first  the first chromosome.
+     * @param second the second chromosome.
+     * @return the pair of new chromosomes that resulted from the crossover.
+     * @throws DimensionMismatchException if the length of the two chromosomes is different
+     */
+    private ChromosomePair crossover(final BitsChromosome first, final BitsChromosome second)
+            throws DimensionMismatchException {
+        final int length = first.getLength();
+        if (length != second.getLength()) {
+            throw new DimensionMismatchException(second.getLength(), length);
+        }
+
+        final BitSet parent1Key = first.getRepresentation();
+        final BitSet parent2Key = second.getRepresentation();
+
+        final BitSet child1Key = new BitSet(length);
+        final BitSet child2Key = new BitSet(length);
+
+        // select a crossover point at random (0 and length makes no sense)
+        final int crossoverIndex = 1 + (GeneticAlgorithm.getRandomGenerator().nextInt(length - 2));
+
+        BitSet a = (BitSet) parent1Key.clone();
+        a.clear(crossoverIndex, length);
+        BitSet b = (BitSet) parent2Key.clone();
+        b.clear(0, crossoverIndex);
+
+        BitSet c = (BitSet) parent1Key.clone();
+        c.clear(crossoverIndex, length);
+        BitSet d = (BitSet) parent2Key.clone();
+        d.clear(0, crossoverIndex);
+
+        child1Key.or(a);
+        child1Key.or(d);
+
+        child2Key.or(c);
+        child2Key.or(b);
+        return new ChromosomePair(first.newBitsChromosome(child1Key), second.newBitsChromosome(child2Key));
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/CombinedStoppingCondition.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/CombinedStoppingCondition.java
new file mode 100644
index 0000000000..97d5c4b273
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/CombinedStoppingCondition.java
@@ -0,0 +1,47 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm.genetic;
+
+import java.util.List;
+
+import org.apache.commons.math3.genetics.Population;
+import org.apache.commons.math3.genetics.StoppingCondition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CombinedStoppingCondition implements StoppingCondition {
+    private static final Logger logger = LoggerFactory.getLogger(GeneticAlgorithm.class);
+
+    private List<StoppingCondition> conditions;
+
+    public CombinedStoppingCondition(List<StoppingCondition> conditions) {
+        this.conditions = conditions;
+    }
+
+    @Override
+    public boolean isSatisfied(Population population) {
+        for (StoppingCondition condition : conditions) {
+            if (condition.isSatisfied(population)) {
+                logger.info("Stopping condition {} is satisfied", condition);
+                return true;
+            }
+        }
+        return false;
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/GeneticAlgorithm.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/GeneticAlgorithm.java
new file mode 100644
index 0000000000..09935cc899
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/GeneticAlgorithm.java
@@ -0,0 +1,137 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm.genetic;
+
+import java.math.BigInteger;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.math3.genetics.Chromosome;
+import org.apache.commons.math3.genetics.ElitisticListPopulation;
+import org.apache.commons.math3.genetics.FixedGenerationCount;
+import org.apache.commons.math3.genetics.Population;
+import org.apache.commons.math3.genetics.StoppingCondition;
+import org.apache.kylin.metadata.cube.planner.algorithm.AbstractRecommendAlgorithm;
+import org.apache.kylin.metadata.cube.planner.algorithm.BenefitPolicy;
+import org.apache.kylin.metadata.cube.planner.algorithm.CuboidStats;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+public class GeneticAlgorithm extends AbstractRecommendAlgorithm {
+
+    private static final Logger logger = LoggerFactory.getLogger(GeneticAlgorithm.class);
+
+    private final org.apache.commons.math3.genetics.GeneticAlgorithm geneticAlgorithm;
+
+    /**
+     * the rate of crossover for the algorithm.
+     */
+    private final double crossoverRate = 0.9;
+    /**
+     * the rate of mutation for the algorithm.
+     */
+    private final double mutationRate = 0.001;
+    /**
+     * the init population size.
+     */
+    private final int populationSize = 500;
+    /**
+     * the max population size.
+     */
+    private final int maxPopulationSize = 510;
+
+    public GeneticAlgorithm(final long timeout, BenefitPolicy benefitPolicy, CuboidStats cuboidStats) {
+        super(timeout, benefitPolicy, cuboidStats);
+        this.geneticAlgorithm = new org.apache.commons.math3.genetics.GeneticAlgorithm(new BitsOnePointCrossover(),
+                crossoverRate, new BitsMutation(), mutationRate, new RouletteWheelSelection());
+    }
+
+    @Override
+    public List<BigInteger> start(double maxSpaceLimit) {
+        logger.debug("Genetic Algorithm started.");
+
+        //Initial mandatory cuboids
+        double remainingSpace = maxSpaceLimit;
+        for (BigInteger mandatoryOne : cuboidStats.getAllCuboidsForMandatory()) {
+            if (cuboidStats.getCuboidSize(mandatoryOne) != null) {
+                remainingSpace -= cuboidStats.getCuboidSize(mandatoryOne);
+            }
+        }
+
+        BitsChromosomeHelper helper = new BitsChromosomeHelper(remainingSpace, cuboidStats);
+
+        //Generate a population randomly
+        Population initial = initRandomPopulation(helper);
+
+        //Set stopping condition
+        List<StoppingCondition> conditions = Lists.newArrayList();
+        conditions.add(new FixedGenerationCount(550));
+        CombinedStoppingCondition stopCondition = new CombinedStoppingCondition(conditions);
+
+        //Start the evolution
+        Population current = geneticAlgorithm.evolve(initial, stopCondition);
+        BitsChromosome chromosome = (BitsChromosome) current.getFittestChromosome();
+        logger.debug("Genetic Algorithm finished.");
+        List<BigInteger> finalList = Lists.newArrayList();
+        finalList.addAll(helper.getMandatoryCuboids());
+        finalList.addAll(chromosome.getCuboids());
+
+        double totalSpace = 0;
+        if (logger.isTraceEnabled()) {
+            for (BigInteger cuboid : finalList) {
+                Double unitSpace = cuboidStats.getCuboidSize(cuboid);
+                if (unitSpace != null) {
+                    logger.trace(String.format(Locale.ROOT, "cuboidId %d and Space: %f", cuboid, unitSpace));
+                    totalSpace += unitSpace;
+                } else {
+                    logger.trace(String.format(Locale.ROOT, "mandatory cuboidId %d", cuboid));
+                }
+            }
+            logger.trace("Total Space:" + totalSpace);
+            logger.trace("Space Expansion Rate:" + totalSpace / cuboidStats.getBaseCuboidSize());
+        }
+        return finalList;
+    }
+
+    protected Population initRandomPopulation(BitsChromosomeHelper helper) {
+        List<Chromosome> chromosomeList = Lists.newArrayListWithCapacity(populationSize);
+
+        while (chromosomeList.size() < populationSize) {
+            BitSet bitSetForSelection = new BitSet(helper.getLength());
+
+            //Initialize selection genes
+            double totalSpace = 0;
+            while (totalSpace < helper.spaceLimit) {
+                int j = org.apache.commons.math3.genetics.GeneticAlgorithm.getRandomGenerator()
+                        .nextInt(helper.getLength());
+                if (!bitSetForSelection.get(j)) {
+                    totalSpace += helper.getCuboidSizeByBitIndex(j);
+                    bitSetForSelection.set(j);
+                }
+            }
+
+            Chromosome chromosome = new BitsChromosome(bitSetForSelection, benefitPolicy.getInstance(), helper);
+            chromosomeList.add(chromosome);
+        }
+        return new ElitisticListPopulation(chromosomeList, maxPopulationSize, 0.8);
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/RouletteWheelSelection.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/RouletteWheelSelection.java
new file mode 100644
index 0000000000..f7fdfcfd12
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/RouletteWheelSelection.java
@@ -0,0 +1,61 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm.genetic;
+
+import java.util.List;
+
+import org.apache.commons.math3.genetics.Chromosome;
+import org.apache.commons.math3.genetics.ChromosomePair;
+import org.apache.commons.math3.genetics.GeneticAlgorithm;
+import org.apache.commons.math3.genetics.ListPopulation;
+import org.apache.commons.math3.genetics.Population;
+import org.apache.commons.math3.genetics.SelectionPolicy;
+
+import com.google.common.collect.Lists;
+
+public class RouletteWheelSelection implements SelectionPolicy {
+    @Override
+    public ChromosomePair select(Population population) throws IllegalArgumentException {
+        // create a copy of the chromosome list
+        List<Chromosome> chromosomes = Lists.newArrayList(((ListPopulation) population).getChromosomes());
+
+        double maxFitness = 0;
+        double totalFitness = 0;
+        for (Chromosome o : chromosomes) {
+            double fitness = o.getFitness();
+            totalFitness += fitness;
+            if (fitness > maxFitness) {
+                maxFitness = fitness;
+            }
+        }
+        return new ChromosomePair(rouletteWheel(chromosomes, totalFitness), rouletteWheel(chromosomes, totalFitness));
+    }
+
+    private Chromosome rouletteWheel(final List<Chromosome> chromosomes, final double totalFitness) {
+        double rnd = (GeneticAlgorithm.getRandomGenerator().nextDouble() * totalFitness);
+        double runningScore = 0;
+        for (Chromosome o : chromosomes) {
+            if (rnd >= runningScore && rnd <= runningScore + o.getFitness()) {
+                return o;
+            }
+            runningScore += o.getFitness();
+        }
+        return null;
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/greedy/GreedyAlgorithm.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/greedy/GreedyAlgorithm.java
new file mode 100644
index 0000000000..a1b454d2b8
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/planner/algorithm/greedy/GreedyAlgorithm.java
@@ -0,0 +1,162 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm.greedy;
+
+import java.math.BigInteger;
+import java.util.List;
+import java.util.Locale;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.kylin.metadata.cube.planner.algorithm.AbstractRecommendAlgorithm;
+import org.apache.kylin.metadata.cube.planner.algorithm.BenefitPolicy;
+import org.apache.kylin.metadata.cube.planner.algorithm.CuboidBenefitModel;
+import org.apache.kylin.metadata.cube.planner.algorithm.CuboidStats;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * A simple implementation of the Greedy Algorithm , it chooses the cuboids which give
+ * the greatest benefit based on expansion rate and time limitation.
+ */
+public class GreedyAlgorithm extends AbstractRecommendAlgorithm {
+    private static final Logger logger = LoggerFactory.getLogger(GreedyAlgorithm.class);
+
+    private static final int THREAD_NUM = 8;
+    private ExecutorService executor;
+
+    private Set<BigInteger> selected = Sets.newLinkedHashSet();
+    private List<BigInteger> remaining = Lists.newLinkedList();
+
+    public GreedyAlgorithm(final long timeout, BenefitPolicy benefitPolicy, CuboidStats cuboidStats) {
+        super(timeout, benefitPolicy, cuboidStats);
+    }
+
+    @Override
+    public List<BigInteger> start(double spaceLimit) {
+        logger.info("Greedy Algorithm started.");
+        executor = Executors.newFixedThreadPool(THREAD_NUM,
+                new ThreadFactoryBuilder().setNameFormat("greedy-algorithm-benefit-calculator-pool-%d").build());
+
+        //Initial mandatory cuboids
+        selected.clear();
+        double remainingSpace = spaceLimit;
+        for (BigInteger mandatoryOne : cuboidStats.getAllCuboidsForMandatory()) {
+            selected.add(mandatoryOne);
+            if (cuboidStats.getCuboidSize(mandatoryOne) != null) {
+                remainingSpace -= cuboidStats.getCuboidSize(mandatoryOne);
+            }
+        }
+        //Initial remaining cuboid set
+        remaining.clear();
+        remaining.addAll(cuboidStats.getAllCuboidsForSelection());
+
+        long round = 0;
+        while (!shouldCancel()) {
+            // Choose one cuboid having the maximum benefit per unit space in all available list
+            CuboidBenefitModel best = recommendBestOne();
+            // If return null, then we should finish the process and return
+            if (best == null) {
+                logger.info("Greedy algorithm ends due to cannot find next best one");
+                break;
+            }
+            // If we finally find the cuboid selected does not meet a minimum threshold of benefit (for
+            // example, a cuboid with 0.99M roll up from a parent cuboid with 1M
+            // rows), then we should finish the process and return
+            if (!benefitPolicy.ifEfficient(best)) {
+                logger.info("Greedy algorithm ends due to the benefit of the best one is not efficient {}",
+                        best.getBenefit());
+                break;
+            }
+
+            remainingSpace -= cuboidStats.getCuboidSize(best.getCuboidId());
+            // If we finally find there is no remaining space,  then we should finish the process and return
+            if (remainingSpace <= 0) {
+                logger.info("Greedy algorithm ends due to there's no remaining space");
+                break;
+            }
+            selected.add(best.getCuboidId());
+            remaining.remove(best.getCuboidId());
+            benefitPolicy.propagateAggregationCost(best.getCuboidId(), selected);
+            round++;
+            if (logger.isTraceEnabled()) {
+                logger.trace(String.format(Locale.ROOT, "Recommend in round %d : %s", round, best.toString()));
+            }
+        }
+
+        executor.shutdown();
+
+        List<BigInteger> excluded = Lists.newArrayList(remaining);
+        remaining.retainAll(selected);
+        Preconditions.checkArgument(remaining.isEmpty(),
+                "There should be no intersection between excluded list and selected list.");
+        logger.info("Greedy Algorithm finished.");
+
+        if (logger.isTraceEnabled()) {
+            logger.trace("Excluded cuboidId size: {}", excluded.size());
+            logger.trace("Excluded cuboidId detail:");
+            for (BigInteger cuboid : excluded) {
+                logger.trace("cuboidId {} and Cost: {} and Space: {}", cuboid, cuboidStats.getCuboidQueryCost(cuboid),
+                        cuboidStats.getCuboidSize(cuboid));
+            }
+            logger.trace("Total Space: {}", spaceLimit - remainingSpace);
+            logger.trace("Space Expansion Rate: {}", (spaceLimit - remainingSpace) / cuboidStats.getBaseCuboidSize());
+        }
+        return Lists.newArrayList(selected);
+    }
+
+    private CuboidBenefitModel recommendBestOne() {
+        final int selectedSize = selected.size();
+        final AtomicReference<CuboidBenefitModel> best = new AtomicReference<>();
+
+        final CountDownLatch counter = new CountDownLatch(remaining.size());
+        for (final BigInteger cuboid : remaining) {
+            executor.submit(() -> {
+                CuboidBenefitModel currentBest = best.get();
+                assert (selected.size() == selectedSize);
+                CuboidBenefitModel.BenefitModel benefitModel = benefitPolicy.calculateBenefit(cuboid, selected);
+                if (benefitModel != null && (currentBest == null || currentBest.getBenefit() == null
+                        || benefitModel.benefit > currentBest.getBenefit())) {
+                    while (!best.compareAndSet(currentBest,
+                            new CuboidBenefitModel(cuboidStats.getCuboidModel(cuboid), benefitModel))) {
+                        currentBest = best.get();
+                        if (benefitModel.benefit <= currentBest.getBenefit()) {
+                            break;
+                        }
+                    }
+                }
+                counter.countDown();
+            });
+        }
+
+        try {
+            counter.await();
+        } catch (InterruptedException e) {
+        }
+        return best.get();
+    }
+}
diff --git a/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/CostBasePlannerUtilsTest.java b/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/CostBasePlannerUtilsTest.java
new file mode 100644
index 0000000000..647d43ef7c
--- /dev/null
+++ b/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/CostBasePlannerUtilsTest.java
@@ -0,0 +1,34 @@
+package org.apache.kylin.metadata.cube.planner;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.math.BigInteger;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class CostBasePlannerUtilsTest {
+
+    private Map<Integer, Integer> getMap(int dimensionCount) {
+        Map<Integer, Integer> result = new HashMap<>();
+        for (int i = 0; i < dimensionCount; i++) {
+            result.put(i, i);
+        }
+        return result;
+    }
+
+    @Test
+    public void testConvertDimensionsToCuboId() {
+        int maxCountDimension = 12;
+        List<Integer> dimensionIds = Lists.newArrayList(4, 8, 11);
+        BigInteger result = CostBasePlannerUtils.convertDimensionsToCuboId(dimensionIds, maxCountDimension, getMap(12));
+        long expected = 0;
+        expected = 1 << (12 - 1 - 4) | 1 << (12 - 1 - 8) | 1 << (12 - 1 - 11);
+        BigInteger expectedInteger = BigInteger.valueOf(expected);
+        assertEquals(expectedInteger, result);
+    }
+}
\ No newline at end of file
diff --git a/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/AlgorithmTestBase.java b/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/AlgorithmTestBase.java
new file mode 100644
index 0000000000..b1e4356db7
--- /dev/null
+++ b/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/AlgorithmTestBase.java
@@ -0,0 +1,567 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.util.StringUtil;
+import org.junit.After;
+import org.junit.Before;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+public class AlgorithmTestBase {
+
+    public final static Comparator<BigInteger> CuboidSelectComparator = new Comparator<BigInteger>() {
+        @Override
+        public int compare(BigInteger o1, BigInteger o2) {
+            return ComparisonChain.start().compare(o1.bitCount(), o2.bitCount()).compare(o1, o2).result();
+        }
+    };
+
+    private static class TreeNode implements Serializable {
+        @JsonProperty("cuboid_id")
+        BigInteger cuboidId;
+        @JsonIgnore
+        int level;
+        @JsonProperty("children")
+        List<TreeNode> children = Lists.newArrayList();
+
+        public BigInteger getCuboidId() {
+            return cuboidId;
+        }
+
+        public int getLevel() {
+            return level;
+        }
+
+        public List<TreeNode> getChildren() {
+            return children;
+        }
+
+        TreeNode(BigInteger cuboidId, int level) {
+            this.cuboidId = cuboidId;
+            this.level = level;
+        }
+
+        void addChild(BigInteger childId, int parentlevel) {
+            this.children.add(new TreeNode(childId, parentlevel + 1));
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(cuboidId, level);
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj)
+                return true;
+            if (obj == null)
+                return false;
+            if (getClass() != obj.getClass())
+                return false;
+            TreeNode other = (TreeNode) obj;
+            if (cuboidId != other.cuboidId)
+                return false;
+            if (level != other.level)
+                return false;
+            return true;
+        }
+    }
+
+    public static class CuboidTree implements Serializable {
+        private int treeLevels;
+
+        private TreeNode root;
+
+        private Comparator<BigInteger> cuboidComparator;
+
+        private Map<BigInteger, TreeNode> index = new HashMap<>();
+
+        static CuboidTree createFromCuboids(List<BigInteger> allCuboidIds) {
+            return createFromCuboids(allCuboidIds, CuboidSelectComparator);
+        }
+
+        public static CuboidTree createFromCuboids(List<BigInteger> allCuboidIds,
+                                                   Comparator<BigInteger> cuboidComparator) {
+            // sort the cuboid ids in descending order, so that don't need to adjust
+            // the cuboid tree when adding cuboid id to the tree.
+            Collections.sort(allCuboidIds, new Comparator<BigInteger>() {
+                @Override
+                public int compare(BigInteger o1, BigInteger o2) {
+                    return o2.compareTo(o1);
+                }
+            });
+            BigInteger basicCuboidId = allCuboidIds.get(0);
+            CuboidTree cuboidTree = new CuboidTree(cuboidComparator);
+            cuboidTree.setRoot(basicCuboidId);
+
+            for (BigInteger cuboidId : allCuboidIds) {
+                cuboidTree.addCuboid(cuboidId);
+            }
+            cuboidTree.buildIndex();
+            return cuboidTree;
+        }
+
+        private CuboidTree(Comparator<BigInteger> cuboidComparator) {
+            this.cuboidComparator = cuboidComparator;
+        }
+
+        public Set<BigInteger> getAllCuboidIds() {
+            return index.keySet();
+        }
+
+        public List<BigInteger> getSpanningCuboid(BigInteger cuboidId) {
+            TreeNode node = index.get(cuboidId);
+            if (node == null) {
+                throw new IllegalArgumentException("the cuboid:" + cuboidId + " is not exist in the tree");
+            }
+
+            List<BigInteger> result = Lists.newArrayList();
+            for (TreeNode child : node.children) {
+                result.add(child.cuboidId);
+            }
+            return result;
+        }
+
+        public BigInteger findBestMatchCuboid(BigInteger cuboidId) {
+            // exactly match
+            if (isValid(cuboidId)) {
+                return cuboidId;
+            }
+
+            return findBestParent(cuboidId).cuboidId;
+        }
+
+        public boolean isValid(BigInteger cuboidId) {
+            return index.containsKey(cuboidId);
+        }
+
+        private int getCuboidCount(BigInteger cuboidId) {
+            int r = 1;
+            for (BigInteger child : getSpanningCuboid(cuboidId)) {
+                r += getCuboidCount(child);
+            }
+            return r;
+        }
+
+        public void print(PrintWriter out) {
+            int dimensionCnt = root.cuboidId.bitCount();
+            doPrint(root, dimensionCnt, 0, out);
+        }
+
+        private void doPrint(TreeNode node, int dimensionCount, int depth, PrintWriter out) {
+            printCuboid(node.cuboidId, dimensionCount, depth, out);
+
+            for (TreeNode child : node.children) {
+                doPrint(child, dimensionCount, depth + 1, out);
+            }
+        }
+
+        private void printCuboid(BigInteger cuboidID, int dimensionCount, int depth, PrintWriter out) {
+            StringBuffer sb = new StringBuffer();
+            for (int i = 0; i < depth; i++) {
+                sb.append("    ");
+            }
+            String cuboidName = getDisplayName(cuboidID, dimensionCount);
+            sb.append("|---- Cuboid ").append(cuboidName).append("(" + cuboidID + ")");
+            out.println(sb.toString());
+        }
+
+        private String getDisplayName(BigInteger cuboidID, int dimensionCount) {
+            StringBuilder sb = new StringBuilder();
+            for (int i = 0; i < dimensionCount; ++i) {
+
+                if ((cuboidID.and((BigInteger.valueOf(1L << i))).equals(BigInteger.ZERO))) {
+                    sb.append('0');
+                } else {
+                    sb.append('1');
+                }
+            }
+            return StringUtils.reverse(sb.toString());
+        }
+
+        private void setRoot(BigInteger basicCuboidId) {
+            this.root = new TreeNode(basicCuboidId, 0);
+            this.treeLevels = 0;
+        }
+
+        private void buildIndex() {
+            LinkedList<TreeNode> queue = new LinkedList<>();
+            queue.add(root);
+            while (!queue.isEmpty()) {
+                TreeNode node = queue.removeFirst();
+                index.put(node.cuboidId, node);
+                for (TreeNode child : node.children) {
+                    queue.add(child);
+                }
+            }
+        }
+
+        private void addCuboid(BigInteger cuboidId) {
+            TreeNode parent = findBestParent(cuboidId);
+            if (parent != null && !parent.cuboidId.equals(cuboidId)) {
+                parent.addChild(cuboidId, parent.level);
+                this.treeLevels = Math.max(this.treeLevels, parent.level + 1);
+            }
+        }
+
+        private TreeNode findBestParent(BigInteger cuboidId) {
+            TreeNode bestParent = doFindBestParent(cuboidId, root);
+            if (bestParent == null) {
+                throw new IllegalStateException("Cannot find the parent of the cuboid:" + cuboidId);
+            }
+            return bestParent;
+        }
+
+        private TreeNode doFindBestParent(BigInteger cuboidId, TreeNode parentCuboid) {
+            if (!canDerive(cuboidId, parentCuboid.cuboidId)) {
+                return null;
+            }
+
+            List<TreeNode> candidates = Lists.newArrayList();
+            for (TreeNode childCuboid : parentCuboid.children) {
+                TreeNode candidate = doFindBestParent(cuboidId, childCuboid);
+                if (candidate != null) {
+                    candidates.add(candidate);
+                }
+            }
+            if (candidates.isEmpty()) {
+                candidates.add(parentCuboid);
+            }
+
+            return Collections.min(candidates, new Comparator<TreeNode>() {
+                @Override
+                public int compare(TreeNode o1, TreeNode o2) {
+                    return cuboidComparator.compare(o1.cuboidId, o2.cuboidId);
+                }
+            });
+        }
+
+        private boolean canDerive(BigInteger cuboidId, BigInteger parentCuboid) {
+            return (cuboidId.and(parentCuboid.not())).equals(BigInteger.ZERO);
+        }
+    }
+
+    private static class CuboidCostComparator implements Comparator<BigInteger>, Serializable {
+        private Map<BigInteger, Long> cuboidStatistics;
+
+        public CuboidCostComparator(Map<BigInteger, Long> cuboidStatistics) {
+            Preconditions.checkArgument(cuboidStatistics != null,
+                    "the input " + cuboidStatistics + " should not be null!!!");
+            this.cuboidStatistics = cuboidStatistics;
+        }
+
+        @Override
+        public int compare(BigInteger cuboid1, BigInteger cuboid2) {
+            Long rowCnt1 = cuboidStatistics.get(cuboid1);
+            Long rowCnt2 = cuboidStatistics.get(cuboid2);
+            if (rowCnt2 == null || rowCnt1 == null) {
+                return CuboidSelectComparator.compare(cuboid1, cuboid2);
+            }
+            return Long.compare(rowCnt1, rowCnt2);
+        }
+    }
+
+    public CuboidStats cuboidStats;
+
+    private Set<BigInteger> mandatoryCuboids;
+
+    @Before
+    public void setUp() throws Exception {
+
+        mandatoryCuboids = Sets.newHashSet();
+        mandatoryCuboids.add(BigInteger.valueOf(3000L));
+        mandatoryCuboids.add(BigInteger.valueOf(1888L));
+        mandatoryCuboids.add(BigInteger.valueOf(88L));
+        cuboidStats = new CuboidStats.Builder("test", BigInteger.valueOf(4095L), BigInteger.valueOf(4095L),
+                simulateCount(), simulateSpaceSize()).setMandatoryCuboids(mandatoryCuboids)
+                .setHitFrequencyMap(simulateHitFrequency()).setScanCountSourceMap(simulateScanCount()).build();
+    }
+
+    @After
+    public void after() throws Exception {
+    }
+
+    /** better if closer to 1, worse if closer to 0*/
+    public double getQueryCostRatio(CuboidStats cuboidStats, List<BigInteger> recommendList) {
+        CuboidTree cuboidTree = CuboidTree.createFromCuboids(recommendList,
+                new CuboidCostComparator(cuboidStats.getStatistics()));
+        double queryCostBest = 0;
+        for (BigInteger cuboidId : cuboidStats.getAllCuboidsForSelection()) {
+            if (cuboidStats.getCuboidQueryCost(cuboidId) != null) {
+                queryCostBest += cuboidStats.getCuboidHitProbability(cuboidId) * cuboidStats.getCuboidCount(cuboidId);
+                //                queryCostBest += cuboidStats.getCuboidHitProbability(cuboidId) * cuboidStats.getCuboidQueryCost(cuboidId);
+            }
+        }
+
+        double queryCost = 0;
+        for (BigInteger cuboidId : cuboidStats.getAllCuboidsForSelection()) {
+            BigInteger matchCuboidId = cuboidTree.findBestMatchCuboid(cuboidId);
+            if (cuboidStats.getCuboidQueryCost(matchCuboidId) != null) {
+                queryCost += cuboidStats.getCuboidHitProbability(cuboidId) * cuboidStats.getCuboidCount(matchCuboidId);
+                //                queryCost += cuboidStats.getCuboidHitProbability(cuboidId) * cuboidStats.getCuboidQueryCost(matchCuboidId);
+            }
+        }
+
+        return queryCostBest / queryCost;
+    }
+
+    protected Map<BigInteger, Long> simulateCount() {
+        Map<BigInteger, Long> countMap = Maps.newHashMap();
+        BufferedReader br = null;
+
+        try {
+
+            String sCurrentLine;
+
+            br = new BufferedReader(new InputStreamReader(new FileInputStream("src/test/resources/statistics.txt"),
+                    StandardCharsets.UTF_8));
+
+            while ((sCurrentLine = br.readLine()) != null) {
+                String[] statPair = StringUtil.split(sCurrentLine, " ");
+                countMap.put(BigInteger.valueOf(Long.valueOf(statPair[0])), Long.valueOf(statPair[1]));
+            }
+
+        } catch (IOException e) {
+            e.printStackTrace();
+        } finally {
+            try {
+                if (br != null)
+                    br.close();
+            } catch (IOException ex) {
+                ex.printStackTrace();
+            }
+        }
+
+        return countMap;
+    }
+
+    protected Map<BigInteger, Double> simulateSpaceSize() {
+        Map<BigInteger, Double> sizeMap = Maps.newHashMap();
+        Map<BigInteger, Long> countMap = simulateCount();
+        for (Map.Entry<BigInteger, Long> entry : countMap.entrySet()) {
+            sizeMap.put(entry.getKey(), entry.getValue() * 1.0);
+        }
+        return sizeMap;
+    }
+
+    protected Map<BigInteger, Long> simulateHitFrequency() {
+        Map<BigInteger, Long> hitFrequencyMap = Maps.newHashMap();
+
+        hitFrequencyMap.put(BigInteger.valueOf(4095L), 10L);
+        hitFrequencyMap.put(BigInteger.valueOf(3849L), 15L);
+        hitFrequencyMap.put(BigInteger.valueOf(3780L), 31L);
+
+        hitFrequencyMap.put(BigInteger.valueOf(3459L), 16L);
+        hitFrequencyMap.put(BigInteger.valueOf(3145L), 29L);
+
+        hitFrequencyMap.put(BigInteger.valueOf(2861L), 21L);
+        hitFrequencyMap.put(BigInteger.valueOf(2768L), 40L);
+
+        hitFrequencyMap.put(BigInteger.valueOf(1528L), 10L);
+        hitFrequencyMap.put(BigInteger.valueOf(1440L), 9L);
+        hitFrequencyMap.put(BigInteger.valueOf(1152L), 21L);
+
+        hitFrequencyMap.put(BigInteger.valueOf(256L), 23L);
+
+        hitFrequencyMap.put(BigInteger.valueOf(128L), 7L);
+        hitFrequencyMap.put(BigInteger.valueOf(272L), 8L);
+        hitFrequencyMap.put(BigInteger.valueOf(288L), 10L);
+        hitFrequencyMap.put(BigInteger.valueOf(384L), 2L);
+        hitFrequencyMap.put(BigInteger.valueOf(320L), 3L);
+        hitFrequencyMap.put(BigInteger.valueOf(432L), 5L);
+        hitFrequencyMap.put(BigInteger.valueOf(258L), 8L);
+        hitFrequencyMap.put(BigInteger.valueOf(336L), 10L);
+        hitFrequencyMap.put(BigInteger.valueOf(274L), 22L);
+        hitFrequencyMap.put(BigInteger.valueOf(488L), 41L);
+        hitFrequencyMap.put(BigInteger.valueOf(352L), 10L);
+
+        hitFrequencyMap.put(BigInteger.valueOf(16L), 1L);
+        hitFrequencyMap.put(BigInteger.valueOf(32L), 5L);
+        hitFrequencyMap.put(BigInteger.valueOf(34L), 1L);
+
+        hitFrequencyMap.put(BigInteger.valueOf(2L), 21L);
+
+        return hitFrequencyMap;
+    }
+
+    protected Map<BigInteger, Map<BigInteger, Long>> simulateScanCount() {
+        Map<BigInteger, Map<BigInteger, Long>> scanCountMap = Maps.newLinkedHashMap();
+        scanCountMap.put(BigInteger.valueOf(4094L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(4095L), 1833041L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(3849L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(3849L), 276711L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(3780L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(3780L), 129199L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(3459L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(3459L), 168109L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(3145L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(3145L), 299991L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(2861L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(2861L), 2121L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(2768L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(2768L), 40231L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(256L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(256L), 1L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(16L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(16L), 1L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(32L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(32L), 2L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(128L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(128L), 3L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(272L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(272L), 2L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(288L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(288L), 3L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(2L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(2L), 1L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(384L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(384L), 2L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(320L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(320L), 3L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(432L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(432L), 5L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(1152L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(1152L), 21L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(258L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(258L), 2L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(1440L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(1440L), 9L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(336L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(336L), 2L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(336L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(336L), 2L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(274L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(274L), 1L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(488L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(488L), 16L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(352L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(352L), 3L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(1528L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(1528L), 21L);
+            }
+        });
+        scanCountMap.put(BigInteger.valueOf(34L), new HashMap<BigInteger, Long>() {
+            {
+                put(BigInteger.valueOf(34L), 1L);
+            }
+        });
+
+        return scanCountMap;
+    }
+}
\ No newline at end of file
diff --git a/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/GeneticAlgorithmTest.java b/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/GeneticAlgorithmTest.java
new file mode 100644
index 0000000000..c370d1b00f
--- /dev/null
+++ b/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/genetic/GeneticAlgorithmTest.java
@@ -0,0 +1,99 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm.genetic;
+
+import static org.junit.Assert.assertEquals;
+
+import java.math.BigInteger;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.math3.genetics.Chromosome;
+import org.apache.kylin.metadata.cube.planner.algorithm.AlgorithmTestBase;
+import org.apache.kylin.metadata.cube.planner.algorithm.BPUSCalculator;
+import org.apache.kylin.metadata.cube.planner.algorithm.BenefitPolicy;
+import org.apache.kylin.metadata.cube.planner.algorithm.PBPUSCalculator;
+import org.apache.kylin.metadata.cube.planner.algorithm.SPBPUSCalculator;
+import org.junit.Test;
+
+import com.google.common.collect.Sets;
+
+public class GeneticAlgorithmTest extends AlgorithmTestBase {
+
+    @Test
+    public void testChromosomeIsSame() {
+        BenefitPolicy benefitPolicy = new BPUSCalculator(cuboidStats);
+
+        double maxSpaceLimit = cuboidStats.getBaseCuboidSize() * 10;
+        BitsChromosomeHelper helper = new BitsChromosomeHelper(maxSpaceLimit, cuboidStats);
+
+        double maxSpaceLimit1 = cuboidStats.getBaseCuboidSize() * 12;
+        BitsChromosomeHelper helper1 = new BitsChromosomeHelper(maxSpaceLimit1, cuboidStats);
+
+        BitSet representation = new BitSet();
+        representation.set(10);
+        Chromosome chromosome = new BitsChromosome(representation, benefitPolicy, helper);
+        Set<Chromosome> chromosomeSet = Sets.newHashSet(chromosome);
+
+        BitSet representation1 = new BitSet();
+        representation1.set(10);
+        chromosomeSet.add(((BitsChromosome) chromosome).newBitsChromosome(representation1));
+        assertEquals(1, chromosomeSet.size());
+
+        BitSet representation2 = new BitSet();
+        representation2.set(12);
+        chromosomeSet.add(((BitsChromosome) chromosome).newBitsChromosome(representation2));
+        assertEquals(2, chromosomeSet.size());
+
+        BitSet representation3 = new BitSet();
+        representation3.set(12);
+        chromosomeSet.add(new BitsChromosome(representation3, benefitPolicy, helper1));
+        assertEquals(2, chromosomeSet.size());
+    }
+
+    @Test
+    public void testBPUSCalculator() {
+        BenefitPolicy benefitPolicy = new BPUSCalculator(cuboidStats);
+        GeneticAlgorithm algorithm = new GeneticAlgorithm(-1, benefitPolicy, cuboidStats);
+
+        List<BigInteger> recommendList = algorithm.recommend(10);
+        System.out.println("recommendList by BPUSCalculator: " + recommendList);
+        System.out.println("Cost evaluated for each query: " + getQueryCostRatio(cuboidStats, recommendList));
+    }
+
+    @Test
+    public void testPBPUSCalculator() {
+        BenefitPolicy benefitPolicy = new PBPUSCalculator(cuboidStats);
+        GeneticAlgorithm algorithm = new GeneticAlgorithm(-1, benefitPolicy, cuboidStats);
+
+        List<BigInteger> recommendList = algorithm.recommend(10);
+        System.out.println("recommendList by PBPUSCalculator:" + recommendList);
+        System.out.println("Cost evaluated for each query: " + getQueryCostRatio(cuboidStats, recommendList));
+    }
+
+    @Test
+    public void testSPBPUSCalculator() {
+        BenefitPolicy benefitPolicy = new SPBPUSCalculator(cuboidStats);
+        GeneticAlgorithm algorithm = new GeneticAlgorithm(-1, benefitPolicy, cuboidStats);
+
+        List<BigInteger> recommendList = algorithm.recommend(10);
+        System.out.println("recommendList by SPBPUSCalculator:" + recommendList);
+        System.out.println("Cost evaluated for each query: " + getQueryCostRatio(cuboidStats, recommendList));
+    }
+}
diff --git a/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/greedy/GreedyAlgorithmTest.java b/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/greedy/GreedyAlgorithmTest.java
new file mode 100644
index 0000000000..7cb72d5ac0
--- /dev/null
+++ b/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/greedy/GreedyAlgorithmTest.java
@@ -0,0 +1,110 @@
+/*
+ * 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.kylin.metadata.cube.planner.algorithm.greedy;
+
+import java.math.BigInteger;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import io.kyligence.kap.guava20.shaded.common.collect.Lists;
+import org.apache.kylin.metadata.cube.planner.algorithm.AlgorithmTestBase;
+import org.apache.kylin.metadata.cube.planner.algorithm.BPUSCalculator;
+import org.apache.kylin.metadata.cube.planner.algorithm.BenefitPolicy;
+import org.apache.kylin.metadata.cube.planner.algorithm.PBPUSCalculator;
+import org.apache.kylin.metadata.cube.planner.algorithm.SPBPUSCalculator;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class GreedyAlgorithmTest extends AlgorithmTestBase {
+    @Test
+    public void testBPUSCalculator() {
+        BenefitPolicy benefitPolicy = new BPUSCalculator(cuboidStats);
+        GreedyAlgorithm algorithm = new GreedyAlgorithm(-1, benefitPolicy, cuboidStats);
+
+        List<BigInteger> recommendList = algorithm.recommend(10);
+        // The result is copy from the result of 3.1 test
+        List<Integer> expected = Lists.newArrayList(1888, 3000, 88, 4095, 258, 1152, 1456, 274, 448, 34, 386, 336, 1528,
+                290, 322, 434, 1410, 1514, 338, 1976, 1458, 984, 1530, 770, 2040, 1962, 786, 890, 1978, 257, 260, 1994,
+                2042, 273, 289, 276, 385, 292, 321, 388, 1513, 324, 259, 2368, 305, 262, 1532, 401, 337, 2306, 1529,
+                3560, 275, 2384, 1531, 278, 3576, 2041, 1534, 2322, 1980, 3578, 4088, 2046, 2043, 3582, 2047, 4090,
+                2352, 2448, 291, 876, 422, 352, 2338, 387, 769, 308, 323, 2434, 404, 3583, 326, 2370, 1961, 2044, 1444,
+                4092, 435, 2482, 1664, 2037, 340, 3976, 492, 800, 3498, 1451, 3572, 438, 1454, 496, 889, 2920, 4081,
+                872, 1457, 811, 1256, 370, 3504, 2858, 48, 466, 814, 371, 2418, 374, 4094, 1928, 400, 1963, 1992, 1460,
+                988, 891, 4010, 2938);
+        List<BigInteger> expectedBigInteger = expected.stream().map(new Function<Integer, BigInteger>() {
+            @Override
+            public BigInteger apply(Integer integer) {
+                return new BigInteger(integer.toString());
+            }
+        }).collect(Collectors.toList());
+        Assert.assertEquals(expectedBigInteger, recommendList);
+        System.out.println("recommendList by BPUSCalculator: " + recommendList);
+
+        System.out.println("Cost evaluated for each query: " + getQueryCostRatio(cuboidStats, recommendList));
+    }
+
+    @Test
+    public void testPBPUSCalculator() {
+        BenefitPolicy benefitPolicy = new PBPUSCalculator(cuboidStats);
+        GreedyAlgorithm algorithm = new GreedyAlgorithm(-1, benefitPolicy, cuboidStats);
+
+        List<BigInteger> recommendList = algorithm.recommend(10);
+        // The result is copy from the result of 3.1 test
+        List<Integer> expected = Lists.newArrayList(1888, 3000, 88, 4095, 2, 1152, 258, 274, 336, 1440, 488, 34, 432,
+                1528, 386, 290, 322, 434, 1410, 1514, 338, 1976, 1458, 984, 1530, 770, 2040, 1962, 786, 890, 1978, 257,
+                260, 1994, 2768, 2042, 273, 256, 289, 276, 385, 292, 352, 321, 388, 288, 1513, 324, 259, 305, 262, 1532,
+                401, 272, 2306, 337, 1529, 3560, 3780, 128, 275, 1531, 278, 3576, 3145, 2041, 1534, 2322, 384, 3578,
+                1980, 3459, 4013, 2043, 2046, 4088, 320, 32, 3582, 4090, 2047, 3977, 2352, 2448, 291, 876, 422, 2338,
+                387, 769, 308, 323, 2434, 1168, 2384, 2861, 404, 326, 2370, 1961, 2044, 1444, 435, 3583, 2482, 1664,
+                340);
+        List<BigInteger> expectedBigInteger = expected.stream().map(new Function<Integer, BigInteger>() {
+            @Override
+            public BigInteger apply(Integer integer) {
+                return new BigInteger(integer.toString());
+            }
+        }).collect(Collectors.toList());
+        Assert.assertEquals(expectedBigInteger, recommendList);
+        System.out.println("recommendList by PBPUSCalculator:" + recommendList);
+        System.out.println("Cost evaluated for each query: " + getQueryCostRatio(cuboidStats, recommendList));
+    }
+
+    @Test
+    public void testSPBPUSCalculator() {
+        BenefitPolicy benefitPolicy = new SPBPUSCalculator(cuboidStats);
+        GreedyAlgorithm algorithm = new GreedyAlgorithm(-1, benefitPolicy, cuboidStats);
+
+        List<BigInteger> recommendList = algorithm.recommend(10);
+        // The result is copy from the result of 3.1 test
+        List<Integer> expected = Lists.newArrayList(1888, 3000, 88, 4095, 256, 2, 288, 128, 272, 1152, 258, 274, 336,
+                384, 352, 1440, 432, 488, 34, 1528, 386, 768, 290, 322, 434, 1664, 1410, 1960, 784, 1514, 338, 1976,
+                832, 1458, 1992, 1530, 888, 770, 2040, 1962, 786, 890, 1978, 257, 260, 2768, 1994, 2042, 2304, 273, 289,
+                276, 385, 292, 321, 388, 2336, 1513, 324, 259, 305, 2320, 3780, 262, 401, 1532, 337, 3576, 2306, 1529,
+                3145, 275, 4094, 1531, 2861, 3459, 3849, 2041, 2043, 2047, 291, 323, 3583, 435, 2037, 969, 811, 953,
+                370, 4081, 466, 491, 1457, 339, 1961, 1963, 1459, 891, 497, 889, 1535, 1680, 1210, 308, 1515, 1690);
+        List<BigInteger> expectedBigInteger = expected.stream().map(new Function<Integer, BigInteger>() {
+            @Override
+            public BigInteger apply(Integer integer) {
+                return new BigInteger(integer.toString());
+            }
+        }).collect(Collectors.toList());
+        Assert.assertEquals(expectedBigInteger, recommendList);
+        System.out.println("recommendList by SPBPUSCalculator:" + recommendList);
+        System.out.println("Cost evaluated for each query: " + getQueryCostRatio(cuboidStats, recommendList));
+    }
+}
diff --git a/src/core-metadata/src/test/resources/statistics.txt b/src/core-metadata/src/test/resources/statistics.txt
new file mode 100644
index 0000000000..35f69fc8b8
--- /dev/null
+++ b/src/core-metadata/src/test/resources/statistics.txt
@@ -0,0 +1,4092 @@
+1 5408
+2 12
+3 59114
+4 5393
+5 13545958
+6 59352
+7 48900622
+8 234
+9 622632
+10 2513
+11 2684805
+12 661703
+13 51457071
+14 3003220
+15 87905334
+16 4
+17 20057
+18 48
+19 185091
+20 21381
+21 27129912
+22 196464
+23 67684891
+24 905
+25 1409380
+26 8032
+27 5367438
+28 1540851
+29 66914540
+30 5780199
+31 102382118
+32 5
+33 26824
+34 60
+35 250433
+36 26763
+37 30256052
+38 261493
+39 69394709
+40 387
+41 665893
+42 2849
+43 2954813
+44 707431
+45 61390587
+46 3240484
+47 97487859
+48 16
+49 78962
+50 189
+51 648097
+52 80270
+53 46703642
+54 675226
+55 85512868
+56 1138
+57 1463428
+58 8619
+59 5848317
+60 1577917
+61 77312551
+62 6472745
+63 111122050
+64 14
+65 42855
+66 144
+67 337000
+68 44890
+69 33660047
+70 361045
+71 73187752
+72 427
+73 715905
+74 3192
+75 3277233
+76 765421
+77 66688961
+78 3649848
+79 100898382
+80 40
+81 116673
+82 404
+83 846531
+84 123061
+85 51207630
+86 908398
+87 89543966
+89 1645138
+90 9255
+91 6589067
+92 1780584
+93 83045129
+94 7349741
+95 113014051
+96 54
+97 170610
+98 585
+99 1112616
+100 182519
+101 46860213
+102 1270670
+103 83382182
+104 596
+105 823678
+106 3734
+107 3900149
+108 884212
+109 74220549
+110 4350663
+111 106240989
+112 149
+113 407851
+114 1495
+115 2349130
+116 436830
+117 63626889
+118 2661185
+119 98988611
+120 1491
+121 1871944
+122 10494
+123 7619644
+124 2048333
+125 89765741
+126 8703141
+127 117905202
+128 6
+129 31335
+130 72
+131 278154
+132 32528
+133 33299431
+134 289515
+135 72241688
+136 628
+137 671628
+138 3798
+139 2945467
+140 728973
+141 64709349
+142 3205057
+143 99730017
+144 19
+145 86296
+146 217
+147 704603
+148 87692
+149 50352721
+150 728318
+151 89072848
+152 1169
+153 1468982
+154 8697
+155 5821059
+156 1602799
+157 82438973
+158 6402424
+159 113177011
+160 22
+161 111200
+162 263
+163 821474
+164 113822
+165 41785539
+166 872073
+167 76702603
+168 734
+169 753231
+170 4143
+171 3512464
+172 809672
+173 71693751
+174 3856651
+175 104344446
+176 67
+177 273672
+178 744
+179 1742709
+180 285398
+181 57385824
+182 1920352
+183 91758819
+184 1332
+185 1689133
+186 9413
+187 6897407
+188 1821788
+189 88978769
+190 7611629
+191 116410794
+192 51
+193 171884
+194 560
+195 1169383
+196 183414
+197 49288586
+198 1305325
+199 85851630
+200 815
+201 842521
+202 4624
+203 4010811
+204 907399
+205 77349281
+206 4354569
+207 109042390
+208 144
+209 415680
+210 1523
+211 2475079
+212 441787
+213 66734849
+214 2864972
+215 100470889
+216 1505
+217 1871044
+218 10528
+219 7754601
+220 2042714
+221 93029295
+222 8834572
+223 117923381
+224 160
+225 454208
+226 1684
+227 2583569
+228 503525
+229 57311428
+230 2989675
+231 90863923
+232 1048
+233 1140000
+234 6050
+235 5382389
+236 1238979
+237 84241270
+238 6191583
+239 114362710
+240 428
+241 972308
+242 4150
+243 4729682
+244 1089252
+245 72065430
+246 5730712
+247 104402615
+248 1939
+249 2462027
+250 13624
+251 10071894
+252 2712730
+253 97601168
+254 11673557
+255 122190940
+256 2
+257 9498
+258 24
+259 79945
+260 10752
+261 14258615
+262 107239
+263 50856457
+264 452
+265 697726
+266 4490
+267 2822825
+268 932822
+269 53639060
+270 3662872
+271 90489596
+272 8
+273 29798
+274 95
+275 223957
+276 41341
+277 28442950
+278 332878
+279 68930124
+280 1699
+281 1531510
+282 13519
+283 5635662
+284 1989662
+285 69678635
+286 6989874
+287 103854924
+288 10
+289 38592
+290 120
+291 295819
+292 51637
+293 31369690
+294 434324
+295 70938987
+296 671
+297 721499
+298 5038
+299 3147896
+300 968177
+301 64784337
+302 4035226
+303 98482649
+304 32
+305 101297
+306 365
+307 712465
+308 143822
+309 48962693
+310 1050839
+311 88025320
+312 2013
+313 1585225
+314 14261
+315 6129197
+316 2060910
+317 81099274
+318 7634802
+319 111395308
+320 27
+321 57497
+322 257
+323 380724
+324 79504
+325 34879820
+326 587405
+327 76210771
+328 734
+329 789662
+330 5397
+331 3524986
+332 1088152
+333 68215167
+334 4691841
+335 104893830
+336 75
+337 143289
+338 703
+339 925065
+340 214788
+341 53155396
+342 1384457
+343 93315403
+344 2163
+345 1777001
+346 15550
+347 6864447
+348 2381043
+349 85018646
+350 8861967
+351 114587784
+352 103
+353 203441
+354 1031
+355 1216511
+356 296217
+357 48144102
+358 1789235
+359 86052906
+360 965
+361 908466
+362 6345
+363 4128921
+364 1268319
+365 76413089
+366 5553302
+367 108727635
+368 273
+369 459035
+370 2568
+371 2479945
+372 676543
+373 65867786
+374 3570437
+375 101630311
+376 2507
+377 1987608
+378 17331
+379 7980360
+380 2679601
+381 92363718
+382 10275128
+383 117942645
+384 12
+385 45711
+386 143
+387 332427
+388 63139
+389 34188123
+390 513289
+391 74310480
+392 1060
+393 744405
+394 6322
+395 3144421
+396 998196
+397 67000938
+398 4092455
+399 102479781
+400 37
+401 112496
+402 405
+403 801759
+404 159812
+405 51596478
+406 1182427
+407 90983246
+408 2029
+409 1581472
+410 14290
+411 6061271
+412 2047019
+413 84461228
+414 7742372
+415 112474685
+416 43
+417 136352
+418 496
+419 902897
+420 198401
+421 43850605
+422 1324526
+423 78791391
+424 1186
+425 829942
+426 6923
+427 3727578
+428 1138486
+429 73609409
+430 4872398
+431 107810620
+432 126
+433 314911
+434 1339
+435 1863528
+436 460075
+437 59860345
+438 2628087
+439 95065035
+440 2255
+441 1797843
+442 15594
+443 7234754
+444 2398192
+445 90382853
+446 9184313
+447 118024610
+448 100
+449 204678
+450 1013
+451 1288671
+452 311368
+453 51461477
+454 1940945
+455 88461722
+456 1319
+457 913332
+458 7819
+459 4144085
+460 1279201
+461 79110460
+462 5635626
+463 110369460
+464 273
+465 477392
+466 2657
+467 2622297
+468 712921
+469 69125337
+470 3764291
+471 101922838
+472 2531
+473 2001102
+474 17156
+475 8011737
+476 2725091
+477 95464996
+478 10557559
+479 121667004
+480 302
+481 514289
+482 2955
+483 2727750
+484 764150
+485 57613471
+486 3864725
+487 91514836
+488 1671
+489 1216475
+490 10000
+491 5622217
+492 1711059
+493 87066187
+494 7699073
+495 115145694
+496 770
+497 1059840
+498 7002
+499 4971895
+500 1543099
+501 73147277
+502 6992897
+503 105256346
+504 3194
+505 2628403
+506 22318
+507 10634614
+508 3581598
+509 99591089
+510 13694936
+511 123805801
+512 97
+513 438583
+514 1082
+515 2709770
+516 467793
+517 62233795
+518 3236365
+519 92864138
+520 873
+521 1201180
+522 4916
+523 5992226
+524 1295951
+525 89881852
+526 6821855
+527 116111968
+528 290
+529 1027455
+530 3181
+531 5372086
+532 1118502
+533 76874973
+534 6477401
+535 106174660
+536 1934
+537 2683421
+538 13540
+539 11305324
+540 2991570
+541 103334266
+542 13295977
+543 124348132
+544 270
+545 1071956
+546 3026
+547 5786650
+548 1181191
+549 67812072
+550 7082943
+551 98196271
+552 1181
+553 1726665
+554 7259
+555 8291732
+556 1877832
+557 93139506
+558 10029949
+559 119823816
+560 789
+561 2302526
+562 8328
+563 9912121
+564 2601558
+565 82274208
+566 12455751
+567 110555733
+568 2644
+569 3732049
+570 19144
+571 15286252
+572 4134722
+573 106709277
+574 18281589
+575 128436771
+576 528
+577 1488216
+578 5508
+579 7357343
+580 1692952
+581 70505609
+582 9163234
+583 99934902
+584 1509
+585 2198656
+586 9779
+587 10127264
+588 2482129
+589 97628745
+590 12274594
+591 122862474
+592 1472
+593 3124699
+594 14405
+595 12289082
+596 3626702
+597 83834130
+598 15663342
+599 112128796
+600 3392
+601 4553823
+602 24896
+603 17313176
+604 5234246
+605 110223356
+606 21393439
+607 129849677
+608 1706
+609 2983727
+610 15499
+611 12215091
+612 3493957
+613 74709369
+614 15524952
+615 102572764
+616 2894
+617 3633552
+618 20698
+619 14875112
+620 4164119
+621 99922065
+622 18698435
+623 125227448
+624 4256
+625 5549352
+626 36184
+627 18803250
+628 6673860
+629 87286090
+630 24874079
+631 115575487
+632 6459
+633 7016394
+634 47556
+635 23976463
+636 8150783
+637 112752491
+638 30924190
+639 133441422
+640 266
+641 1207316
+642 3052
+643 6636130
+644 1300828
+645 69384103
+646 8156139
+647 98959683
+648 1398
+649 1923357
+650 8325
+651 9397687
+652 2082573
+653 95659787
+654 11220516
+655 119557748
+656 796
+657 2638421
+658 8734
+659 11505399
+660 2920808
+661 82860830
+662 14608740
+663 110360653
+664 2664
+665 4175381
+666 19534
+667 16801270
+668 4630930
+669 108806369
+670 20353452
+671 130528080
+672 939
+673 2827946
+674 10260
+675 12029336
+676 3182512
+677 73512155
+678 15772458
+679 102756341
+680 2344
+681 3476032
+682 16025
+683 14821628
+684 3912061
+685 100378233
+686 18580297
+687 124831025
+688 2621
+689 5304217
+690 25764
+691 18836482
+692 6247823
+693 86432297
+694 24530141
+695 115054234
+696 4916
+697 6820121
+698 37208
+699 24204404
+700 7854361
+701 110867598
+702 30209819
+703 135122998
+704 1604
+705 3317097
+706 15294
+707 13997817
+708 3885042
+709 75085038
+710 17901482
+711 103964100
+712 3112
+713 3999443
+714 21348
+715 16609940
+716 4647775
+717 100722007
+718 21008944
+719 127918093
+720 4244
+721 6276421
+722 36834
+723 21104546
+724 7602232
+725 88383830
+726 27954151
+727 116964820
+728 6670
+729 7958519
+730 48705
+731 26741535
+732 9310460
+733 114567332
+734 34012957
+735 137067139
+736 5334
+737 6688120
+738 45994
+739 21913788
+740 8199538
+741 78744048
+742 28615773
+743 108064945
+744 7301
+745 7374315
+746 51827
+747 24569184
+748 8975876
+749 104718633
+750 31172497
+751 131132532
+752 12668
+753 11226730
+754 98340
+755 30834996
+756 14086985
+757 93251462
+758 40453981
+759 120846110
+760 15512
+761 12738469
+762 110887
+763 36298409
+764 15874117
+765 115913270
+766 45644274
+767 138038027
+768 192
+769 494272
+770 2101
+771 2924324
+772 764314
+773 64313489
+774 4326195
+775 97104611
+776 1311
+777 1305415
+778 8457
+779 6165294
+780 1901623
+781 90611558
+782 8669189
+783 118554901
+784 577
+785 1147131
+786 5889
+787 5536938
+788 1721834
+789 78132034
+790 8046126
+791 108884068
+792 3229
+793 2866087
+794 22554
+795 11657118
+796 4013420
+797 105254429
+798 15495349
+799 127470105
+800 538
+801 1181321
+802 5741
+803 6020531
+804 1803671
+805 69690385
+806 8449098
+807 98671282
+808 1814
+809 1877892
+810 12291
+811 8754810
+812 2759541
+813 97772121
+814 12082368
+815 120788386
+816 1528
+817 2450462
+818 15039
+819 10372968
+820 3544857
+821 83722940
+822 14334373
+823 110816273
+824 4330
+825 3972188
+826 31649
+827 15765886
+828 5545733
+829 109234519
+830 20713321
+831 130127499
+832 1002
+833 1577814
+834 9545
+835 7576178
+836 2439206
+837 71766071
+838 10915295
+839 100458148
+840 2343
+841 2328703
+842 16180
+843 10483353
+844 3441733
+845 99813101
+846 14611780
+847 121316162
+848 2715
+849 3241664
+850 23634
+851 12808809
+852 4752410
+853 85834475
+854 17805351
+855 113391766
+856 5600
+857 4757337
+858 40361
+859 18190693
+860 6739328
+861 112781190
+862 24058555
+863 133824858
+864 2874
+865 3159084
+866 24521
+867 12492065
+868 4494122
+869 76201036
+870 17726844
+871 102131562
+872 4475
+873 3832627
+874 32005
+875 15438631
+876 5410669
+877 103092393
+878 21025747
+879 124629007
+880 7014
+881 5815341
+882 54957
+883 19411364
+884 8002622
+885 88589435
+886 26878537
+887 118293969
+888 10339
+889 7377283
+890 72321
+891 24837882
+892 10041432
+893 114001415
+894 33362816
+895 134556926
+896 528
+897 1302023
+898 5874
+899 6916181
+900 2002364
+901 70992688
+902 9738517
+903 99824048
+904 2204
+905 2060794
+906 13912
+907 9725899
+908 2976365
+909 97543583
+910 13193140
+911 121717746
+912 1564
+913 2787402
+914 15871
+915 12101312
+916 3938817
+917 85048610
+918 16807580
+919 113186566
+920 4445
+921 4309356
+922 32492
+923 17342856
+924 5976510
+925 110425493
+926 23151915
+927 129144702
+928 1707
+929 2935631
+930 17601
+931 12430272
+932 4154926
+933 73795002
+934 17277433
+935 104447950
+936 3748
+937 3655594
+938 26258
+939 15269699
+940 5086842
+941 102039383
+942 20774859
+943 125357623
+944 4693
+945 5489921
+946 42290
+947 19427124
+948 7514459
+949 87672236
+950 26451367
+951 115013581
+952 8006
+953 7124350
+954 60434
+955 24593613
+956 9513495
+957 114569527
+958 33288566
+959 134472524
+960 2900
+961 3466345
+962 25122
+963 14315270
+964 4969555
+965 77208126
+966 19818490
+967 105788076
+968 4990
+969 4221110
+970 33871
+971 17255470
+972 5969058
+973 102263401
+974 23737233
+975 128807532
+976 7382
+977 6537423
+978 58610
+979 21948136
+980 8919704
+981 90888626
+982 30390361
+983 119051840
+984 10928
+985 8203126
+986 76521
+987 27633249
+988 11140419
+989 114440012
+990 36992653
+991 135276469
+992 8907
+993 6923754
+994 70349
+995 22454011
+996 9467754
+997 80151964
+998 31126922
+999 110226875
+1000 11535
+1001 7690684
+1002 80056
+1003 25245134
+1004 10480355
+1005 107198218
+1006 34288000
+1007 129185160
+1008 20582
+1009 11741125
+1010 146083
+1011 32064284
+1012 15900178
+1013 93412095
+1014 42446327
+1015 122663267
+1016 24255
+1017 13339271
+1018 166816
+1019 36578306
+1020 18058472
+1021 118691906
+1022 49121357
+1023 140113432
+1024 31
+1025 150413
+1026 366
+1027 1158644
+1028 157542
+1029 51738914
+1030 1251433
+1031 87564031
+1032 872
+1033 842251
+1034 4476
+1035 4113036
+1036 900240
+1037 78408308
+1038 4567962
+1039 111542416
+1040 93
+1041 402568
+1042 1086
+1043 2554935
+1044 431312
+1045 67925585
+1046 2942411
+1047 102916480
+1048 1527
+1049 1896622
+1050 10210
+1051 8192965
+1052 2063099
+1053 93601052
+1054 9265168
+1055 121858401
+1056 88
+1057 370225
+1058 1025
+1059 2417970
+1060 396557
+1061 61259477
+1062 2777133
+1063 93660336
+1064 1045
+1065 1001241
+1066 5311
+1067 5111023
+1068 1082547
+1069 85511233
+1070 5672325
+1071 116016162
+1072 265
+1073 895954
+1074 2848
+1075 4821805
+1076 995465
+1077 75619690
+1078 5751715
+1079 107422928
+1080 1819
+1081 2306325
+1082 11993
+1083 9782219
+1084 2516472
+1085 100548867
+1086 11462175
+1087 125532977
+1088 192
+1089 629408
+1090 1986
+1091 3545722
+1092 706626
+1093 63252380
+1094 4511461
+1095 95363459
+1096 1256
+1097 1406524
+1098 7031
+1099 6598640
+1100 1581968
+1101 89406248
+1102 7921177
+1103 117127451
+1104 546
+1105 1500980
+1106 5318
+1107 6740546
+1108 1748642
+1109 77488092
+1110 8648849
+1111 107904509
+1112 2351
+1113 3158581
+1114 16590
+1115 12308887
+1116 3591894
+1117 103824241
+1118 15157568
+1119 124572802
+1120 528
+1121 1293488
+1122 5225
+1123 6207972
+1124 1503400
+1125 68975282
+1126 7991050
+1127 97278780
+1128 1646
+1129 1936460
+1130 10057
+1131 8990579
+1132 2211374
+1133 93764350
+1134 10894996
+1135 118538464
+1136 1422
+1137 2711838
+1138 13151
+1139 10697510
+1140 3240677
+1141 82089910
+1142 13911964
+1143 112665340
+1144 3216
+1145 4105046
+1146 23032
+1147 15785201
+1148 4793386
+1149 106967458
+1150 19875798
+1151 128446296
+1152 40
+1153 192297
+1154 475
+1155 1396806
+1156 202836
+1157 52637774
+1158 1606679
+1159 89369726
+1160 902
+1161 828488
+1162 4563
+1163 4057633
+1164 892115
+1165 78947682
+1166 4628052
+1167 111422777
+1168 120
+1169 516183
+1170 1391
+1171 3097751
+1172 564017
+1173 69601145
+1174 3637734
+1175 102905271
+1176 1546
+1177 1933969
+1178 10305
+1179 8269340
+1180 2104002
+1181 95781245
+1182 9426626
+1183 120561698
+1184 89
+1185 369612
+1186 1026
+1187 2408796
+1188 397318
+1189 59904945
+1190 2820020
+1191 94660030
+1192 1045
+1193 999782
+1194 5330
+1195 5094626
+1196 1093516
+1197 86313767
+1198 5690937
+1199 114097583
+1200 267
+1201 891221
+1202 2816
+1203 4842290
+1204 987372
+1205 75279605
+1206 5732936
+1207 105857115
+1208 1809
+1209 2307523
+1210 12149
+1211 9964552
+1212 2526404
+1213 100760890
+1214 11414974
+1215 123666639
+1216 274
+1217 755065
+1218 2729
+1219 3977942
+1220 870519
+1221 64659605
+1222 5120758
+1223 96933647
+1224 1278
+1225 1396320
+1226 7119
+1227 6565673
+1228 1557192
+1229 90355168
+1230 7938956
+1231 115782522
+1232 774
+1233 1720285
+1234 7329
+1235 7194896
+1236 2048618
+1237 78967236
+1238 9406784
+1239 107980362
+1240 2353
+1241 3123420
+1242 16670
+1243 12183951
+1244 3583355
+1245 103563328
+1246 15079836
+1247 124974132
+1248 525
+1249 1315885
+1250 5228
+1251 6242417
+1252 1522222
+1253 68808113
+1254 8105522
+1255 97128744
+1256 1671
+1257 1934414
+1258 10089
+1259 8992037
+1260 2195395
+1261 92646810
+1262 10827707
+1263 120784730
+1264 1418
+1265 2700373
+1266 13078
+1267 10786324
+1268 3231141
+1269 83046435
+1270 13975930
+1271 111979610
+1272 3220
+1273 4102509
+1274 23118
+1275 15607046
+1276 4771277
+1277 106799854
+1278 19520816
+1279 128796256
+1280 62
+1281 184096
+1282 708
+1283 1226094
+1284 286421
+1285 53802398
+1286 1996014
+1287 90620362
+1288 1357
+1289 924841
+1290 7488
+1291 4331679
+1292 1321421
+1293 81221055
+1294 6008660
+1295 111894723
+1296 186
+1297 458807
+1298 2043
+1299 2766075
+1300 720141
+1301 71253190
+1302 4155892
+1303 103750216
+1304 2519
+1305 2084785
+1306 17022
+1307 8646435
+1308 2851706
+1309 98202027
+1310 11287111
+1311 122619487
+1312 176
+1313 420756
+1314 1880
+1315 2561851
+1316 659248
+1317 61062843
+1318 3869343
+1319 95103270
+1320 1599
+1321 1108260
+1322 8792
+1323 5326849
+1324 1588410
+1325 87430404
+1326 7470599
+1327 115919794
+1328 505
+1329 981712
+1330 5058
+1331 5136174
+1332 1520407
+1333 78164384
+1334 7319615
+1335 107199735
+1336 2967
+1337 2463767
+1338 20121
+1339 10261048
+1340 3464105
+1341 100962295
+1342 13606407
+1343 122701080
+1344 364
+1345 701750
+1346 3503
+1347 3800034
+1348 1137022
+1349 64795529
+1350 5848348
+1351 96736498
+1352 1994
+1353 1526157
+1354 11745
+1355 6964018
+1356 2305917
+1357 91172102
+1358 9880389
+1359 119077644
+1360 1013
+1361 1622751
+1362 9579
+1363 7019881
+1364 2523931
+1365 80951123
+1366 10428309
+1367 109408699
+1368 3941
+1369 3277940
+1370 27831
+1371 12852475
+1372 4758310
+1373 106022332
+1374 17495641
+1375 126719249
+1376 972
+1377 1399891
+1378 8883
+1379 6522581
+1380 2149418
+1381 69594418
+1382 9537448
+1383 100195532
+1384 2646
+1385 2075924
+1386 16513
+1387 9174104
+1388 3073036
+1389 96008133
+1390 13150409
+1391 120614695
+1392 2544
+1393 2847171
+1394 21947
+1395 11133816
+1396 4270758
+1397 84317788
+1398 15877777
+1399 112157213
+1400 5329
+1401 4394623
+1402 37521
+1403 16359478
+1404 6221391
+1405 108191440
+1406 22252708
+1407 129282760
+1408 80
+1409 233598
+1410 919
+1411 1529897
+1412 372629
+1413 54941648
+1414 2403095
+1415 93082547
+1416 1405
+1417 931537
+1418 7578
+1419 4290188
+1420 1319386
+1421 81636241
+1422 6049584
+1423 111209518
+1424 239
+1425 588606
+1426 2633
+1427 3298108
+1428 944159
+1429 72146846
+1430 4961109
+1431 105176126
+1432 2542
+1433 2085325
+1434 16845
+1435 8528145
+1436 2872494
+1437 97299053
+1438 11408785
+1439 122170761
+1440 175
+1441 428398
+1442 1857
+1443 2577017
+1444 672969
+1445 62750644
+1446 3850272
+1447 93736662
+1448 1602
+1449 1130498
+1450 8813
+1451 5381949
+1452 1619208
+1453 89143817
+1454 7451329
+1455 115301172
+1456 503
+1457 997074
+1458 5019
+1459 5067096
+1460 1515936
+1461 77314474
+1462 7324790
+1463 107872289
+1464 2949
+1465 2504573
+1466 19777
+1467 10264209
+1468 3491214
+1469 101733794
+1470 13764677
+1471 126159667
+1472 519
+1473 828924
+1474 4847
+1475 4166168
+1476 1340404
+1477 66334297
+1478 6439460
+1479 96008951
+1480 2005
+1481 1507150
+1482 11846
+1483 6968586
+1484 2282402
+1485 91127119
+1486 9843793
+1487 118194925
+1488 1424
+1489 1831180
+1490 12770
+1491 7605776
+1492 2883433
+1493 80125968
+1494 11202391
+1495 109486787
+1496 3918
+1497 3307601
+1498 27919
+1499 12955575
+1500 4738709
+1501 103773671
+1502 17275804
+1503 127329438
+1504 969
+1505 1422291
+1506 8909
+1507 6507932
+1508 2178852
+1509 70387439
+1510 9624012
+1511 101004435
+1512 2633
+1513 2073074
+1514 16450
+1515 9257057
+1516 3095246
+1517 96744708
+1518 13164609
+1519 120792891
+1520 2537
+1521 2849173
+1522 21678
+1523 11126753
+1524 4214940
+1525 83496541
+1526 16227929
+1527 111759246
+1528 5282
+1529 4327984
+1530 36901
+1531 16352366
+1532 6228279
+1533 109868812
+1534 22306845
+1535 128783268
+1536 613
+1537 2730648
+1538 7248
+1539 13278299
+1540 3049614
+1541 79907265
+1542 17488876
+1543 107021427
+1544 2039
+1545 3426089
+1546 13111
+1547 15894701
+1548 3779090
+1549 103817282
+1550 20610011
+1551 127619132
+1552 1850
+1553 5750197
+1554 20084
+1555 21326501
+1556 6829385
+1557 90140817
+1558 29179696
+1559 119891574
+1560 4089
+1561 7286104
+1562 31998
+1563 26759490
+1564 8364412
+1565 116915755
+1566 34853541
+1567 135295601
+1568 1947
+1569 5418503
+1570 20958
+1571 20571143
+1572 6344275
+1573 81816479
+1574 27592777
+1575 108579191
+1576 3733
+1577 6083457
+1578 28179
+1579 23099311
+1580 7149218
+1581 108603139
+1582 30825252
+1583 131097109
+1584 5443
+1585 10100080
+1586 53275
+1587 30137763
+1588 12414498
+1589 94443293
+1590 40499460
+1591 122045858
+1592 8237
+1593 11475670
+1594 66003
+1595 35381710
+1596 13907552
+1597 119123389
+1598 47110549
+1599 139272771
+1600 4061
+1601 7127017
+1602 36882
+1603 24401845
+1604 8922688
+1605 84138449
+1606 33420425
+1607 112205558
+1608 5886
+1609 7772691
+1610 44202
+1611 27042921
+1612 9659516
+1613 108984786
+1614 36408983
+1615 130079215
+1616 10558
+1617 12691029
+1618 91335
+1619 34920030
+1620 16478557
+1621 96095309
+1622 46744861
+1623 123508848
+1624 13624
+1625 14030341
+1626 102580
+1627 39459290
+1628 17864280
+1629 121671572
+1630 53134291
+1631 140735841
+1632 11493
+1633 12014204
+1634 95520
+1635 33843551
+1636 15393515
+1637 86890246
+1638 44868182
+1639 114698301
+1640 14164
+1641 12661312
+1642 102269
+1643 37646637
+1644 16290134
+1645 112225670
+1646 49119976
+1647 135335977
+1648 27272
+1649 19243478
+1650 206764
+1651 45579973
+1652 25651324
+1653 98142216
+1654 58063137
+1655 126425142
+1656 31033
+1657 21408777
+1658 224421
+1659 49646834
+1660 27595106
+1661 122883998
+1662 63969490
+1663 144683259
+1664 617
+1665 2679240
+1666 7142
+1667 13283653
+1668 2952028
+1669 80788327
+1670 17722175
+1671 107927325
+1672 2063
+1673 3445371
+1674 13272
+1675 16066069
+1676 3812631
+1677 103282457
+1678 20565602
+1679 125650673
+1680 1844
+1681 5774665
+1682 20286
+1683 21770311
+1684 6799693
+1685 91092592
+1686 28744229
+1687 119392777
+1688 4083
+1689 7211377
+1690 32261
+1691 26442435
+1692 8343834
+1693 115077456
+1694 35234902
+1695 136484876
+1696 1945
+1697 5426386
+1698 21077
+1699 20802936
+1700 6367298
+1701 82046460
+1702 27747469
+1703 110219853
+1704 3779
+1705 6155539
+1706 28029
+1707 23546836
+1708 7168340
+1709 106466176
+1710 30803094
+1711 131112257
+1712 5341
+1713 10034176
+1714 53170
+1715 30473019
+1716 12384347
+1717 94382900
+1718 40983184
+1719 123154627
+1720 8178
+1721 11463619
+1722 66948
+1723 35258213
+1724 13885331
+1725 115631945
+1726 46896987
+1727 138729236
+1728 4027
+1729 7036632
+1730 36665
+1731 24385441
+1732 8812806
+1733 83382945
+1734 33703350
+1735 111467501
+1736 5937
+1737 7981053
+1738 43270
+1739 27665655
+1740 9770689
+1741 108871177
+1742 35993072
+1743 133627510
+1744 10501
+1745 12708057
+1746 91780
+1747 34599617
+1748 16286618
+1749 94619754
+1750 46654440
+1751 124234980
+1752 13604
+1753 14157541
+1754 104051
+1755 40103516
+1756 17871019
+1757 119564746
+1758 51707472
+1759 141461884
+1760 11643
+1761 12067915
+1762 96253
+1763 34014423
+1764 15465732
+1765 86580327
+1766 45226540
+1767 116041041
+1768 13865
+1769 12746628
+1770 103219
+1771 36241695
+1772 16213272
+1773 111309833
+1774 48414814
+1775 135449250
+1776 27000
+1777 19411950
+1778 205500
+1779 44434714
+1780 25714325
+1781 99059044
+1782 57678269
+1783 126866168
+1784 30973
+1785 21102620
+1786 222239
+1787 50330284
+1788 27371928
+1789 122497188
+1790 64093612
+1791 143298155
+1792 1240
+1793 2924331
+1794 13496
+1795 13631199
+1796 4468294
+1797 80097158
+1798 20106155
+1799 108928714
+1800 3282
+1801 3649010
+1802 22380
+1803 16781603
+1804 5388194
+1805 106628403
+1806 23564226
+1807 128033285
+1808 3618
+1809 6021472
+1810 37198
+1811 22156554
+1812 8609761
+1813 93378196
+1814 31350349
+1815 121356005
+1816 6988
+1817 7594121
+1818 54875
+1819 27270994
+1820 10590865
+1821 117257800
+1822 37828233
+1823 136596579
+1824 3646
+1825 5656922
+1826 36404
+1827 21482255
+1828 7994317
+1829 83454123
+1830 30351318
+1831 109286465
+1832 6270
+1833 6425176
+1834 46201
+1835 23962566
+1836 8980191
+1837 107620761
+1838 33626199
+1839 132733059
+1840 9921
+1841 10458528
+1842 88975
+1843 30993855
+1844 14208475
+1845 95250681
+1846 42769354
+1847 123650134
+1848 13975
+1849 11857235
+1850 106420
+1851 36574084
+1852 16257651
+1853 118851270
+1854 50121180
+1855 140997664
+1856 7158
+1857 7372328
+1858 61335
+1859 25408870
+1860 10522563
+1861 84490786
+1862 35395854
+1863 113763173
+1864 9770
+1865 8320083
+1866 70997
+1867 28165783
+1868 11750214
+1869 109864335
+1870 39198261
+1871 132867109
+1872 18628
+1873 13071342
+1874 147439
+1875 35759298
+1876 18399007
+1877 96994118
+1878 49414422
+1879 125529665
+1880 22337
+1881 14802666
+1882 165092
+1883 41444909
+1884 20532132
+1885 121596514
+1886 54883602
+1887 141485763
+1889 12520099
+1890 145714
+1891 35424339
+1892 17663868
+1893 88137185
+1894 47167541
+1895 116264847
+1896 22685
+1897 13224942
+1898 156980
+1899 37818236
+1900 18457151
+1901 113678427
+1902 51664666
+1903 134162002
+1904 44355
+1905 19805951
+1906 310302
+1907 46353216
+1908 28026476
+1909 100495308
+1910 59486731
+1911 127948851
+1912 49058
+1913 21725542
+1914 329641
+1915 51218460
+1916 30049289
+1917 124537971
+1918 66983942
+1919 144770570
+1920 1227
+1921 2928344
+1922 13437
+1923 13616235
+1924 4444041
+1925 80096207
+1926 19749563
+1927 107817260
+1928 3278
+1929 3626551
+1930 22244
+1931 16535803
+1932 5412984
+1933 106282543
+1934 23433780
+1935 126540705
+1936 3683
+1937 6086630
+1938 37432
+1939 22284984
+1940 8580463
+1941 93622551
+1942 31617410
+1943 121838198
+1944 7043
+1945 7673323
+1946 54983
+1947 27831814
+1948 10704915
+1949 117534939
+1950 37519581
+1951 138812008
+1952 3659
+1953 5702609
+1954 36759
+1955 21267091
+1956 8048006
+1957 83274001
+1958 30193501
+1959 110759141
+1960 6207
+1961 6453541
+1962 47072
+1963 24071255
+1964 8976310
+1965 108266609
+1966 33115615
+1967 132031002
+1968 9967
+1969 10220194
+1970 88636
+1971 31213580
+1972 14434863
+1973 96789994
+1974 42471936
+1975 123722116
+1976 14046
+1977 11895366
+1978 106772
+1979 36685350
+1980 16121825
+1981 118751144
+1982 49926109
+1983 140689183
+1984 7189
+1985 7527179
+1986 61452
+1987 25329724
+1988 10616607
+1989 84677617
+1990 36118985
+1991 112532106
+1992 9835
+1993 8251387
+1994 71885
+1995 28035767
+1996 11715520
+1997 110480934
+1998 38941603
+1999 131575707
+2000 18790
+2001 13144752
+2002 146177
+2003 36022195
+2004 18497284
+2005 97131033
+2006 48509887
+2007 125631086
+2008 22669
+2009 14660464
+2010 164199
+2011 40959811
+2012 20388574
+2013 121600974
+2014 55479463
+2015 143099953
+2016 19157
+2017 12698361
+2018 148484
+2019 34577209
+2020 17790461
+2021 87213355
+2022 47476392
+2023 115442622
+2024 22629
+2025 13179261
+2026 154089
+2027 37538204
+2028 18377556
+2029 114492629
+2030 50854718
+2031 136212446
+2032 44950
+2033 20224756
+2034 313127
+2035 46192224
+2036 28234029
+2037 99809422
+2038 60251012
+2039 128322426
+2040 48643
+2041 21659921
+2042 333423
+2043 51327747
+2044 29698685
+2045 125287720
+2046 65541052
+2047 143363712
+2048 7442
+2049 16865576
+2050 86014
+2051 58493528
+2052 11942184
+2053 89632581
+2054 33477680
+2055 114775367
+2056 963389
+2057 62981401
+2058 4060470
+2059 106535613
+2060 36534798
+2061 117901650
+2062 61611213
+2063 136609704
+2064 28157
+2065 32701475
+2066 266899
+2067 80002045
+2068 20605052
+2069 103283230
+2070 44954693
+2071 126825626
+2072 2250088
+2073 83828002
+2074 8204614
+2075 122431797
+2076 48724593
+2077 127569021
+2078 74101651
+2079 147219073
+2080 36778
+2081 36023749
+2082 365157
+2083 80885359
+2084 22836215
+2085 102704007
+2086 46195873
+2087 124924944
+2088 1007977
+2089 73640279
+2090 4400657
+2091 116541566
+2092 42983632
+2093 123015751
+2094 68629830
+2095 141542925
+2096 107596
+2097 55596098
+2098 928674
+2099 100144751
+2100 32428219
+2101 114069321
+2102 56315717
+2103 132896191
+2104 2301083
+2105 95488907
+2106 8992628
+2107 129755866
+2108 55800262
+2109 133664018
+2110 81270095
+2111 148867980
+2112 60818
+2113 39478388
+2114 496621
+2115 87346550
+2116 24864797
+2117 104142521
+2118 49324042
+2119 127890693
+2120 1085707
+2121 80954470
+2122 5077239
+2123 120109435
+2124 46621404
+2125 125315012
+2126 72216620
+2127 144409677
+2128 172981
+2129 58845074
+2130 1246896
+2131 104330868
+2132 35009260
+2133 116873522
+2134 59436855
+2135 135497858
+2136 2545438
+2137 100546572
+2138 10140144
+2139 135715986
+2140 59536760
+2141 135964268
+2142 85499041
+2143 150029514
+2144 238274
+2145 54642555
+2146 1714172
+2147 98350715
+2148 33013034
+2149 113006861
+2150 56919070
+2151 131203018
+2152 1251207
+2153 89232062
+2154 5957049
+2155 126359432
+2156 51413519
+2157 129235200
+2158 76044270
+2159 144135646
+2160 582156
+2161 75540121
+2162 3578468
+2163 115717529
+2164 43538840
+2165 122547316
+2166 66733129
+2167 138354560
+2168 2911859
+2169 108527925
+2170 11952963
+2171 142266931
+2172 63800323
+2173 140744598
+2174 89649673
+2175 152470976
+2176 43383
+2177 39263661
+2178 411002
+2179 84492190
+2180 24470845
+2181 102456000
+2182 47457739
+2183 126078295
+2184 1013394
+2185 77029197
+2186 4471597
+2187 118855872
+2188 45135873
+2189 125474980
+2190 69892571
+2191 143710048
+2192 115293
+2193 59576930
+2194 995537
+2195 103955764
+2196 34743010
+2197 116341158
+2198 58239452
+2199 136542962
+2200 2302542
+2201 98978702
+2202 8995018
+2203 134284290
+2204 56939156
+2205 134830244
+2206 82705434
+2207 148793193
+2208 150765
+2209 48730052
+2210 1230270
+2211 90512132
+2212 29259164
+2213 107583767
+2214 51446486
+2215 128433785
+2216 1150230
+2217 86816160
+2218 5371747
+2219 123057095
+2220 49396563
+2221 129127656
+2222 74946901
+2223 145933533
+2224 376709
+2225 67879848
+2226 2621339
+2227 106379416
+2228 38951539
+2229 119842421
+2230 62513965
+2231 135958284
+2232 2553200
+2233 106883629
+2234 10619260
+2235 140153140
+2236 61563673
+2237 137448424
+2238 87265684
+2239 149279657
+2240 239491
+2241 57218391
+2242 1766295
+2243 100559934
+2244 34305465
+2245 113606780
+2246 56959277
+2247 132035577
+2248 1261494
+2249 94129845
+2250 5967315
+2251 128412455
+2252 53604191
+2253 129929026
+2254 77263888
+2255 147354036
+2256 583224
+2257 76647238
+2258 3802573
+2259 115801104
+2260 45163854
+2261 123675005
+2262 68379476
+2263 138755787
+2264 2899890
+2265 112476388
+2266 12054541
+2267 141710858
+2268 65930021
+2269 141958107
+2270 91062705
+2271 154106165
+2272 653772
+2273 65588601
+2274 4109532
+2275 104064264
+2276 39061480
+2277 117016055
+2278 63426813
+2279 135467580
+2280 1676456
+2281 99519583
+2282 8403088
+2283 133865809
+2284 57083022
+2285 132631333
+2286 82256852
+2287 149832018
+2288 1424066
+2289 83066825
+2290 7765505
+2291 120475696
+2292 49679924
+2293 126938414
+2294 74105398
+2295 140801362
+2296 3718454
+2297 119135433
+2298 15946010
+2299 146224736
+2300 70148221
+2301 142104933
+2302 96908839
+2303 154958056
+2304 14668
+2305 17710389
+2306 152339
+2307 61146735
+2308 13732533
+2309 90835174
+2310 35444430
+2311 116466781
+2312 1260616
+2313 64410707
+2314 4937216
+2315 108449199
+2316 38277941
+2317 117443097
+2318 64843607
+2319 138452662
+2320 52176
+2321 34204242
+2322 436722
+2323 82606724
+2324 22573114
+2325 103729882
+2326 47080639
+2327 126436720
+2328 2682015
+2329 85118611
+2330 9435654
+2331 125944194
+2332 50647240
+2333 129778613
+2334 76996849
+2335 146263652
+2336 69377
+2337 37104209
+2338 588663
+2339 83520402
+2340 24475142
+2341 103244272
+2342 47687296
+2343 127559194
+2344 1331598
+2345 75930140
+2346 5415786
+2347 117728924
+2348 45678017
+2349 124174235
+2350 71183471
+2351 143058177
+2352 186071
+2353 56437806
+2354 1367660
+2355 103700761
+2356 34386372
+2357 117161023
+2358 59736175
+2359 133747983
+2360 2858417
+2361 97791974
+2362 10391163
+2363 131898874
+2364 58101536
+2365 132432762
+2366 82522443
+2367 151687672
+2368 104359
+2369 41456194
+2370 790346
+2371 88686821
+2372 26691937
+2373 107021017
+2374 50531362
+2375 127309226
+2376 1461417
+2377 82359184
+2378 6191439
+2379 120544277
+2380 48268200
+2381 128982575
+2382 74479582
+2383 143907328
+2384 278912
+2385 62650843
+2386 1788613
+2387 109484800
+2388 37529899
+2389 117701107
+2390 61337841
+2391 136051612
+2392 3231099
+2393 103832323
+2394 11985510
+2395 136039587
+2396 62632023
+2397 135123916
+2398 86208609
+2399 148586372
+2400 376683
+2401 55664253
+2402 2317046
+2403 99899322
+2404 34438393
+2405 112967205
+2406 57994305
+2407 131627917
+2408 1676250
+2409 90463878
+2410 7227855
+2411 127804117
+2412 53805431
+2413 132103563
+2414 78104919
+2415 143270132
+2416 850466
+2417 76920281
+2418 4586905
+2419 116840045
+2420 45485855
+2421 125698265
+2422 69907656
+2423 139989131
+2424 3591859
+2425 112044567
+2426 13888317
+2427 142815326
+2428 66150096
+2429 139567154
+2430 90548743
+2431 150276135
+2432 81959
+2433 40935625
+2434 685066
+2435 85795329
+2436 26541294
+2437 105937421
+2438 49369198
+2439 125730879
+2440 1355339
+2441 80048870
+2442 5384137
+2443 121983603
+2444 47467062
+2445 125698043
+2446 71882909
+2447 140770178
+2448 203883
+2449 61333524
+2450 1543202
+2451 106667809
+2452 36254418
+2453 117017362
+2454 60228564
+2455 136240589
+2456 2854305
+2457 102273023
+2458 10466497
+2459 134877724
+2460 59565228
+2461 137242153
+2462 84018620
+2463 150521373
+2464 256442
+2465 50477471
+2466 1753857
+2467 91888278
+2468 30925017
+2469 108011562
+2470 54282398
+2471 130166879
+2472 1508218
+2473 89534853
+2474 6544535
+2475 126685269
+2476 51444446
+2477 130666832
+2478 75895272
+2479 141711293
+2480 592086
+2481 68911784
+2482 3429824
+2483 109141967
+2484 40924396
+2485 120240847
+2486 65368053
+2487 135068011
+2488 3247228
+2489 108343400
+2490 12296645
+2491 139200103
+2492 64253946
+2493 139464434
+2494 90144018
+2495 150933892
+2496 387203
+2497 60112680
+2498 2471124
+2499 101364240
+2500 36060090
+2501 113115258
+2502 59092846
+2503 133660255
+2504 1688067
+2505 95166867
+2506 7464938
+2507 130201426
+2508 55435813
+2509 131378360
+2510 80339270
+2511 144343901
+2512 891695
+2513 79403496
+2514 4846349
+2515 117591013
+2516 46701564
+2517 127372321
+2518 71594029
+2519 139752761
+2520 3585107
+2521 114633188
+2522 14060404
+2523 142634270
+2524 68291733
+2525 141838947
+2526 92319322
+2527 154201069
+2528 930345
+2529 66936301
+2530 5057724
+2531 108112438
+2532 41230837
+2533 117236757
+2534 63114220
+2535 134889085
+2536 2264842
+2537 102766997
+2538 10189124
+2539 134349467
+2540 59607139
+2541 136132248
+2542 84365276
+2543 145396604
+2544 1934988
+2545 87496829
+2546 8963586
+2547 123821148
+2548 51594867
+2549 126583574
+2550 75453026
+2551 143429208
+2552 4688291
+2553 121133074
+2554 18369851
+2555 146651284
+2556 71294421
+2557 144513528
+2558 98980071
+2559 156474703
+2560 574681
+2561 72013501
+2562 4170731
+2563 108740256
+2564 42160291
+2565 118802622
+2566 65199657
+2567 134318813
+2568 1711436
+2569 106364466
+2570 9377548
+2571 136089932
+2572 61781874
+2573 136063027
+2574 85955728
+2575 149722927
+2576 1460336
+2577 90368418
+2578 8479930
+2579 123461144
+2580 52938120
+2581 127696495
+2582 77264195
+2583 140512222
+2584 4006765
+2585 123742005
+2586 17570016
+2587 149293906
+2588 74254766
+2589 142980282
+2590 99639510
+2591 155578996
+2592 1511044
+2593 78158991
+2594 9250811
+2595 113168463
+2596 47728382
+2597 121344153
+2598 72033721
+2599 140215256
+2600 2570305
+2601 111183745
+2602 13333752
+2603 139295086
+2604 65791739
+2605 139747945
+2606 91745662
+2607 150998682
+2608 3444869
+2609 95018873
+2610 16736273
+2611 129851414
+2612 59218692
+2613 131000976
+2614 84000432
+2615 145265002
+2616 5672362
+2617 129042313
+2618 24831279
+2619 154064778
+2620 79250112
+2621 148089275
+2622 105806379
+2623 159746592
+2624 2109900
+2625 81321389
+2626 11843659
+2627 115581441
+2628 51249627
+2629 121796815
+2630 74697826
+2631 140540844
+2632 3240333
+2633 114853705
+2634 16467288
+2635 141781340
+2636 68579437
+2637 140809504
+2638 94904134
+2639 155028084
+2640 4586544
+2641 99753752
+2642 20819775
+2643 130425050
+2644 63177958
+2645 133283733
+2646 89252057
+2647 148817311
+2648 7053685
+2649 132802587
+2650 29207405
+2651 157030973
+2652 82807923
+2653 147127777
+2654 108975046
+2655 159194852
+2656 4445525
+2657 87538245
+2658 20784213
+2659 118896880
+2660 57340871
+2661 126578306
+2662 82097782
+2663 143862076
+2664 5584146
+2665 118805371
+2666 25221541
+2667 147671549
+2668 74271136
+2669 143730494
+2670 101333431
+2671 159741857
+2672 8851766
+2673 106132315
+2674 33077565
+2675 135998977
+2676 69673935
+2677 138169687
+2678 97158858
+2679 152019357
+2680 11056251
+2681 137568589
+2682 41686204
+2683 158704164
+2684 88686482
+2685 153232585
+2686 119340173
+2687 166854230
+2688 1636939
+2689 79441694
+2690 10758207
+2691 114454589
+2692 48997170
+2693 121025473
+2694 73788585
+2695 140017108
+2696 2803974
+2697 113091943
+2698 15035959
+2699 143213303
+2700 67542300
+2701 139865148
+2702 94357868
+2703 151251593
+2704 3949896
+2705 96756463
+2706 19757814
+2707 127100039
+2708 61307563
+2709 133125189
+2710 86228962
+2711 148212827
+2712 6212546
+2713 131017681
+2714 27930582
+2715 151134460
+2716 82373955
+2717 146627678
+2718 108157839
+2719 161615558
+2720 4214631
+2721 85948560
+2722 20705208
+2723 119264661
+2724 55552169
+2725 124259355
+2726 81871855
+2727 142802420
+2728 5277704
+2729 118871743
+2730 25065926
+2731 145881513
+2732 73629648
+2733 144891803
+2734 101051234
+2735 156434054
+2736 8345645
+2737 103642148
+2738 33131740
+2739 135995086
+2740 69281917
+2741 139691917
+2742 95699348
+2743 155369212
+2744 10648845
+2745 137233318
+2746 41054643
+2747 161076108
+2748 89238030
+2749 153490810
+2750 116768211
+2751 165921864
+2752 4930729
+2753 89048703
+2754 23605140
+2755 120634134
+2756 58062708
+2757 128107941
+2758 83936018
+2759 144465779
+2760 6018705
+2761 121923012
+2762 27889312
+2763 146495022
+2764 77138497
+2765 144984094
+2766 104193266
+2767 157293335
+2768 10010204
+2769 104420755
+2770 37849428
+2771 138752421
+2772 72704805
+2773 139749859
+2774 100078206
+2775 154595744
+2776 12221109
+2777 139139719
+2778 45853920
+2779 162191363
+2780 92493968
+2781 155114895
+2782 120804768
+2783 166649637
+2784 10508975
+2785 95266740
+2786 37787523
+2787 126247609
+2788 67366579
+2789 131265716
+2790 94198384
+2791 149777177
+2792 11574993
+2793 127074826
+2794 42907803
+2795 154953176
+2796 85512056
+2797 149108134
+2798 112059991
+2799 163299205
+2800 18591730
+2801 113565050
+2802 54596835
+2803 142248590
+2804 82610665
+2805 147555374
+2806 109980262
+2807 161349248
+2808 20935997
+2809 146140513
+2810 62193161
+2811 167341203
+2812 100511596
+2813 161199461
+2814 132696320
+2815 173113372
+2816 932098
+2817 73705793
+2818 5502116
+2819 109501785
+2820 43830184
+2821 119884101
+2822 67377838
+2823 135458192
+2824 2433685
+2825 108299931
+2826 11187411
+2827 138677489
+2828 63177309
+2829 135039974
+2830 88575169
+2831 148462632
+2832 2099186
+2833 91644863
+2834 10402545
+2835 123438153
+2836 55789068
+2837 128257948
+2838 79511331
+2839 143224007
+2840 5180676
+2841 126794604
+2842 20185085
+2843 149019808
+2844 75889862
+2845 147076860
+2846 100530181
+2847 156047939
+2848 2189646
+2849 80418880
+2850 10974090
+2851 114024031
+2852 49385284
+2853 120984582
+2854 73806199
+2855 137143764
+2856 3487905
+2857 113058554
+2858 15835036
+2859 142433293
+2860 67052820
+2861 139186514
+2862 93483191
+2863 152104699
+2864 4460768
+2865 97221285
+2866 18890399
+2867 129429882
+2868 60588477
+2869 131574867
+2870 85670376
+2871 147281730
+2872 7220349
+2873 131534709
+2874 27590271
+2875 154379106
+2876 82133206
+2877 146742055
+2878 107964112
+2879 160865797
+2880 2918655
+2881 83271649
+2882 13792791
+2883 116183089
+2884 52800211
+2885 124907760
+2886 77753669
+2887 141152439
+2888 4324336
+2889 116738096
+2890 18630462
+2891 144629016
+2892 70294035
+2893 139799710
+2894 97193310
+2895 152565537
+2896 5872337
+2897 100103262
+2898 23077597
+2899 131420534
+2900 63762812
+2901 134538502
+2902 89113924
+2903 149626971
+2904 8898348
+2905 133046197
+2906 31919716
+2907 155554649
+2908 84411661
+2909 150932008
+2910 110315168
+2911 160806197
+2912 5586151
+2913 88394234
+2914 22724402
+2915 120775622
+2916 58559218
+2917 125781349
+2918 83317030
+2919 144643371
+2920 6824151
+2921 119523033
+2922 27518088
+2923 147717490
+2924 74961984
+2925 143160327
+2926 104434713
+2927 154217660
+2928 10316210
+2929 106559793
+2930 34746065
+2931 137142638
+2932 72468402
+2933 138578200
+2934 97381392
+2935 152493106
+2936 13102378
+2937 138696090
+2938 44108163
+2939 158160891
+2940 91667921
+2941 155595728
+2942 119431285
+2943 169130090
+2944 2459776
+2945 82018742
+2946 12619666
+2947 114673305
+2948 51264060
+2949 121450095
+2950 75216592
+2951 140219369
+2952 3832509
+2953 114922638
+2954 17685242
+2955 142659030
+2956 69266767
+2957 139779462
+2958 94511031
+2959 153169181
+2960 5075213
+2961 98753536
+2962 21902697
+2963 130770037
+2964 62594333
+2965 133307792
+2966 88148296
+2967 149733699
+2968 7890118
+2969 133358775
+2970 30457713
+2971 154517030
+2972 82694085
+2973 149085761
+2974 110386899
+2975 162252240
+2976 5259600
+2977 87806906
+2978 22507445
+2979 119466498
+2980 57691083
+2981 127707815
+2982 82756572
+2983 144327024
+2984 6629992
+2985 120842226
+2986 27524276
+2987 149509918
+2988 75671990
+2989 143701181
+2990 101564735
+2991 158538548
+2992 9864960
+2993 105532019
+2994 35020301
+2995 134985234
+2996 71764499
+2997 139833958
+2998 98352927
+2999 155659614
+3001 139895979
+3002 44146910
+3003 160456800
+3004 91124233
+3005 154702227
+3006 118797971
+3007 165294928
+3008 6120589
+3009 89990599
+3010 26066681
+3011 121371393
+3012 59895248
+3013 128435485
+3014 85609465
+3015 145632698
+3016 7508838
+3017 124043698
+3018 30719698
+3019 148239576
+3020 78652923
+3021 145656165
+3022 105904822
+3023 158329638
+3024 11695663
+3025 108843684
+3026 40286674
+3027 139249337
+3028 74942420
+3029 140565207
+3030 102277201
+3031 155999482
+3032 14623104
+3033 141133804
+3034 48692746
+3035 163882184
+3036 95294425
+3037 156150093
+3038 121729991
+3039 167797321
+3040 12291581
+3041 97604203
+3042 40110999
+3043 127700624
+3044 68699258
+3045 134376636
+3046 94907005
+3047 150694435
+3048 13187179
+3049 129803067
+3050 44804540
+3051 156295692
+3052 86093792
+3053 150835451
+3054 114573263
+3055 162880486
+3056 20622931
+3057 113878196
+3058 56577971
+3059 145053366
+3060 84172186
+3061 145584375
+3062 110457617
+3063 161543161
+3064 23289574
+3065 148412254
+3066 63970562
+3067 169267050
+3068 103908901
+3069 161167731
+3070 131329206
+3071 175230378
+3072 201255
+3073 59731483
+3074 1689145
+3075 101838541
+3076 35442161
+3077 115374765
+3078 59385205
+3079 132757681
+3080 1249145
+3081 93840481
+3082 6199928
+3083 131414132
+3084 53647983
+3085 131373452
+3086 78174796
+3087 148119850
+3088 542664
+3089 79621263
+3090 3876917
+3091 119105159
+3092 46586724
+3093 123445850
+3094 69698852
+3095 138814459
+3096 2850905
+3097 114957027
+3098 12531555
+3099 144048258
+3100 66681331
+3101 139011208
+3102 91800004
+3103 154175942
+3104 501443
+3105 69259471
+3106 3722407
+3107 106654913
+3108 40997621
+3109 118200849
+3110 64188238
+3111 137675699
+3112 1486697
+3113 102261156
+3114 7674235
+3115 134328787
+3116 58868707
+3117 134984447
+3118 83082004
+3119 146802999
+3120 1265809
+3121 88129789
+3122 7583345
+3123 122704229
+3124 51159928
+3125 126733648
+3126 75364346
+3127 142255862
+3128 3467852
+3129 121589303
+3130 15482688
+3131 147039130
+3132 71066829
+3133 143951449
+3134 96175217
+3135 153034960
+3136 861281
+3137 72392234
+3138 5834356
+3139 108865563
+3140 43268392
+3141 118524644
+3142 66502263
+3143 137807945
+3144 2045323
+3145 104683309
+3146 10566776
+3147 137770224
+3148 60762343
+3149 136609754
+3150 85098922
+3151 150234894
+3152 2140794
+3153 91492088
+3154 11452116
+3155 124736891
+3156 54407860
+3157 129707429
+3158 79445660
+3159 143893689
+3160 4684592
+3161 123585345
+3162 20155767
+3163 147059328
+3164 74657088
+3165 145920539
+3166 99082967
+3167 157018711
+3168 1874244
+3169 79482608
+3170 10525107
+3171 114509845
+3172 47568341
+3173 121695906
+3174 73006780
+3175 139854248
+3176 2865098
+3177 112490058
+3178 14492926
+3179 138201457
+3180 65175254
+3181 139228334
+3182 89368342
+3183 151098033
+3184 4027295
+3185 96408561
+3186 18317287
+3187 128917773
+3188 60302029
+3189 132214740
+3190 85437184
+3191 147411661
+3192 6261812
+3193 129089675
+3194 26541042
+3195 153457383
+3196 79407235
+3197 147375924
+3198 105540174
+3199 158537902
+3200 254560
+3201 61140942
+3202 2079586
+3203 102273493
+3204 36013453
+3205 114946694
+3206 59477294
+3207 132402234
+3208 1267132
+3209 94864402
+3210 6231016
+3211 128860857
+3212 55510689
+3213 134523471
+3214 79469469
+3215 144267066
+3216 689334
+3217 80999563
+3218 4741176
+3219 120298631
+3220 46729942
+3221 124911553
+3222 71283833
+3223 142858323
+3224 2917360
+3225 113050357
+3226 12637530
+3227 143188891
+3228 65947496
+3229 141674174
+3230 91921576
+3231 149943922
+3232 510696
+3233 69603870
+3234 3712297
+3235 107292850
+3236 40932153
+3237 119204847
+3238 63187269
+3239 134164175
+3240 1487970
+3241 102514083
+3242 7748745
+3243 133204930
+3244 58322635
+3245 135192943
+3246 83430319
+3247 146396759
+3248 1232255
+3249 87574417
+3250 7651678
+3251 122299134
+3252 51608729
+3253 128652865
+3254 75676785
+3255 142815163
+3256 3468854
+3257 117844144
+3258 15560264
+3259 147203671
+3260 71250379
+3261 143672438
+3262 96002950
+3263 155727097
+3264 1048655
+3265 74155079
+3266 6499011
+3267 110034194
+3268 43914601
+3269 118867445
+3270 67227351
+3271 135365814
+3272 2023563
+3273 105774570
+3274 10504641
+3275 135348409
+3276 60849890
+3277 135709282
+3278 86824366
+3279 150693829
+3280 2469269
+3281 90725508
+3282 12271089
+3283 124976308
+3284 55447798
+3285 129375506
+3286 79498401
+3287 144975788
+3288 4690572
+3289 123486373
+3290 19895041
+3291 149404708
+3292 75273636
+3293 147424425
+3294 101025328
+3295 156058249
+3296 1821572
+3297 79106703
+3298 10467740
+3299 112424288
+3300 48199031
+3301 120802354
+3302 71647258
+3303 138326952
+3304 2832315
+3305 111856440
+3306 14662727
+3307 141361472
+3308 64370479
+3309 139523791
+3310 90738322
+3311 150851981
+3312 4065794
+3313 98065847
+3314 18544144
+3315 129932639
+3316 60550138
+3317 132351196
+3318 85055106
+3319 147011736
+3320 6176123
+3321 127322362
+3322 26055854
+3323 151150982
+3324 78358984
+3325 149599777
+3326 106664741
+3327 161191718
+3328 350593
+3329 62615146
+3330 2475087
+3331 105515755
+3332 37175376
+3333 115242950
+3334 60368793
+3335 132396982
+3336 1699513
+3337 95617775
+3338 7787453
+3339 133223916
+3340 56259828
+3341 133484737
+3342 81377286
+3343 149232501
+3344 858869
+3345 82127696
+3346 5179638
+3347 120611324
+3348 47840005
+3349 125486557
+3350 71196706
+3351 139346777
+3352 3698770
+3353 117057797
+3354 15010389
+3355 145651686
+3356 68464703
+3357 142337481
+3358 94123522
+3359 152664625
+3360 810868
+3361 71435120
+3362 4884725
+3363 108137227
+3364 42669279
+3365 118935892
+3366 65940242
+3367 133387634
+3368 2087068
+3369 103438507
+3370 9642219
+3371 134688226
+3372 59815905
+3373 136729236
+3374 84315427
+3375 148160277
+3376 1834089
+3377 90884068
+3378 9333392
+3379 124007072
+3380 53305703
+3381 128876427
+3382 77547121
+3383 143821763
+3384 4505074
+3385 123088299
+3386 17648934
+3387 146362250
+3388 73780102
+3389 143504508
+3390 98199359
+3391 157873555
+3392 1337048
+3393 74881950
+3394 7264025
+3395 109793559
+3396 45830866
+3397 120901486
+3398 67681752
+3399 138388219
+3400 2816706
+3401 107402506
+3402 12765570
+3403 137283378
+3404 62752866
+3405 138156137
+3406 88580702
+3407 148944346
+3408 2987142
+3409 92819040
+3410 13096788
+3411 127400556
+3412 56565901
+3413 130756832
+3414 79908377
+3415 143567227
+3416 6082836
+3417 124942291
+3418 22532602
+3419 151346490
+3420 76952048
+3421 144749367
+3422 102343617
+3423 160225164
+3424 2627108
+3425 79629865
+3426 12164042
+3427 115416910
+3428 49005476
+3429 120791323
+3430 74641972
+3431 141840413
+3432 3821445
+3433 112556854
+3434 16913159
+3435 142643053
+3436 67970843
+3437 139957281
+3438 92610573
+3439 152762121
+3440 5183868
+3441 99146577
+3442 20499242
+3443 130410802
+3444 62158814
+3445 131570277
+3446 87977786
+3447 146079192
+3448 7860273
+3449 131888627
+3450 29380094
+3451 153231401
+3452 82532063
+3453 149181432
+3454 108077797
+3455 160061719
+3456 448558
+3457 63997628
+3458 3051379
+3459 106948567
+3460 38635225
+3461 116580574
+3462 62407879
+3463 133515235
+3464 1708070
+3465 95151317
+3466 7819865
+3467 131643934
+3468 55533763
+3469 131034853
+3470 81019985
+3471 149347855
+3472 1089455
+3473 84404068
+3474 6209900
+3475 120399838
+3476 48719388
+3477 128661147
+3478 73019135
+3479 141018145
+3480 3707767
+3481 116847386
+3482 14903533
+3483 143973354
+3484 69185726
+3485 142333567
+3486 94771167
+3487 150723344
+3488 817452
+3489 71396363
+3490 4931212
+3491 110286309
+3492 42883103
+3493 119084740
+3494 64387696
+3495 136636787
+3496 2055186
+3497 103627193
+3498 9500925
+3499 135086433
+3500 60041609
+3501 134632091
+3502 84624888
+3503 148490588
+3504 1858365
+3505 89686444
+3506 9305349
+3507 124344509
+3508 53413734
+3509 128872221
+3510 78558083
+3511 142207216
+3512 4491162
+3513 121922352
+3514 17784671
+3515 148211704
+3516 72578771
+3517 146768726
+3518 97314728
+3519 157049318
+3520 1580102
+3521 75159932
+3522 7962971
+3523 111114388
+3524 45461001
+3525 120296287
+3526 68775879
+3527 137697161
+3528 2869451
+3529 107741463
+3530 12607076
+3531 137331688
+3532 62805277
+3533 138370160
+3534 88923323
+3535 150257590
+3536 3335540
+3537 93223854
+3538 14000270
+3539 127710180
+3540 56658490
+3541 129786341
+3542 81952551
+3543 143096923
+3544 6000400
+3545 125142371
+3546 22581786
+3547 149624640
+3548 76611832
+3549 145766276
+3550 102217662
+3551 157994443
+3552 2575183
+3553 80831058
+3554 12200074
+3555 114103951
+3556 49804376
+3557 124017676
+3558 73639089
+3559 140005200
+3560 3884995
+3561 114574972
+3562 16723532
+3563 141109021
+3564 66556403
+3565 140322656
+3566 93046065
+3567 152324952
+3568 5168347
+3569 97021723
+3570 20559333
+3571 131608906
+3572 61290469
+3573 133579343
+3574 86967466
+3575 148169247
+3576 7875825
+3577 130424793
+3578 29377930
+3579 152507544
+3580 81322755
+3581 149718803
+3582 109354073
+3583 160127036
+3584 3726135
+3585 91888578
+3586 22805448
+3587 122842903
+3588 59706806
+3589 130232409
+3590 87703611
+3591 142673584
+3592 4872154
+3593 122704994
+3594 26785682
+3595 149257786
+3596 78509420
+3597 145061395
+3598 103470785
+3599 158630687
+3600 8794472
+3601 111443442
+3602 38080921
+3603 139507784
+3604 74551970
+3605 141079438
+3606 101812836
+3607 156351010
+3608 11476360
+3609 142926106
+3610 46322583
+3611 161946524
+3612 94719905
+3613 157007824
+3614 121211705
+3615 165873804
+3616 8188726
+3617 99290499
+3618 36625361
+3619 127939220
+3620 68270127
+3621 133202894
+3622 93523749
+3623 150951776
+3624 9182722
+3625 128432957
+3626 40984717
+3627 153625950
+3628 84549231
+3629 150253640
+3630 111900940
+3631 161050892
+3632 16209191
+3633 114985361
+3634 54419135
+3635 147726743
+3636 83483363
+3637 146137758
+3638 111762010
+3639 161710373
+3640 18336638
+3641 148728085
+3642 63205736
+3643 168326815
+3644 102105951
+3645 160896705
+3646 132399544
+3647 174609400
+3648 10714844
+3649 101869528
+3650 42693587
+3651 130497073
+3652 72196740
+3653 132499698
+3654 98929648
+3655 153139228
+3656 11926795
+3657 131041879
+3658 47612221
+3659 156795719
+3660 88459247
+3661 150765295
+3662 117091664
+3663 165541260
+3664 21111486
+3665 119884532
+3666 61555467
+3667 147577354
+3668 89493423
+3669 149441682
+3670 116510728
+3671 164060933
+3672 23041575
+3673 152359899
+3674 71426527
+3675 169450215
+3676 107145617
+3677 163121178
+3678 136452524
+3679 177584037
+3680 19328205
+3681 107805060
+3682 59719884
+3683 136212434
+3684 80650901
+3685 138835798
+3686 108352478
+3687 157489665
+3688 20802749
+3689 138228385
+3690 63967180
+3691 162472890
+3692 98479953
+3693 155841859
+3694 127950472
+3695 172055482
+3696 33743981
+3697 127576808
+3698 78051912
+3699 154280783
+3700 99061038
+3701 155764227
+3702 125692192
+3703 169270871
+3704 35480592
+3705 157438506
+3706 86461028
+3707 178810281
+3708 118409088
+3709 170976241
+3710 145852243
+3711 183822189
+3712 3735859
+3713 91686737
+3714 22895422
+3715 123688127
+3716 59775111
+3717 127487333
+3718 86088421
+3719 145517426
+3720 4898731
+3721 123414169
+3722 26582530
+3723 150339150
+3724 75899398
+3725 146300399
+3726 104432363
+3727 157670963
+3728 8855984
+3729 110673280
+3730 39149334
+3731 139460356
+3732 74793328
+3733 138165158
+3734 101512017
+3735 158630201
+3736 11141718
+3737 142118103
+3738 47434191
+3739 162686098
+3740 93208144
+3741 155014052
+3742 124793588
+3743 169404304
+3744 8186836
+3745 97763296
+3746 36170716
+3747 128470561
+3748 67106265
+3749 132354207
+3750 95264332
+3751 150705101
+3752 9231275
+3753 129105020
+3754 40848945
+3755 153322838
+3756 85578785
+3757 148962913
+3758 111322308
+3759 159842536
+3760 16372892
+3761 116748506
+3762 54446528
+3763 143690555
+3764 83618046
+3765 146427155
+3766 113319840
+3767 161963596
+3768 18477817
+3769 147449384
+3770 62683945
+3771 167252968
+3772 102779223
+3773 161514717
+3774 132380572
+3775 172394265
+3776 10956812
+3777 101102074
+3778 43002053
+3779 130205756
+3780 71299226
+3781 133920023
+3782 99038459
+3783 151777016
+3784 12088141
+3785 132592265
+3786 47608058
+3787 158079280
+3788 88290584
+3789 151867427
+3790 116615227
+3791 162490543
+3792 20850605
+3793 118248660
+3794 62870898
+3795 149571216
+3796 89090723
+3797 148092040
+3798 114054838
+3799 163913391
+3800 23445531
+3801 149794142
+3802 70882460
+3803 171875921
+3804 107900818
+3805 163115546
+3806 135721673
+3807 180067125
+3808 19699492
+3809 108070773
+3810 59910687
+3811 135935221
+3812 80568288
+3813 141276298
+3814 108819382
+3815 157845656
+3816 20514116
+3817 137746409
+3818 63866545
+3819 161378691
+3820 97766135
+3821 157558868
+3822 127330397
+3823 171548999
+3824 33555620
+3825 124700332
+3826 77630536
+3827 155131475
+3828 98890344
+3829 155391062
+3830 126564270
+3831 170709292
+3832 35510149
+3833 159262817
+3834 86808777
+3835 179591788
+3836 119007141
+3837 171607763
+3838 145185651
+3839 179584537
+3840 5248834
+3841 94381395
+3842 25142684
+3843 125029977
+3844 62374088
+3845 128849043
+3846 87871804
+3847 144188233
+3848 6665412
+3849 127671610
+3850 29784285
+3851 149529628
+3852 79920105
+3853 144288880
+3854 106776349
+3855 157779438
+3856 10815298
+3857 113220237
+3858 40445543
+3859 140762877
+3860 75773653
+3861 140632143
+3862 103623603
+3863 157234398
+3864 13664172
+3865 141496529
+3866 49047139
+3867 164327348
+3868 96423815
+3869 156918208
+3870 123983494
+3871 169639736
+3872 9955276
+3873 98929189
+3874 38782591
+3875 129402611
+3876 69690028
+3877 132179393
+3878 96671076
+3879 149545854
+3880 11299540
+3881 131048249
+3882 43683026
+3883 153593878
+3884 86980793
+3885 150579225
+3886 113721479
+3887 163433378
+3888 18288118
+3889 117626580
+3890 57268114
+3891 147686268
+3892 85663912
+3893 149237379
+3894 114987586
+3895 161312153
+3896 21048857
+3897 148158173
+3898 65424235
+3899 169280130
+3900 103771906
+3901 161537030
+3902 134288168
+3903 173792979
+3904 12960511
+3905 102519133
+3906 45369427
+3907 130691609
+3908 74068241
+3909 134680537
+3910 100765318
+3911 154124084
+3912 14381256
+3913 132503803
+3914 50468497
+3915 155422899
+3916 89987601
+3917 153357094
+3918 119133919
+3919 164333818
+3920 23082249
+3921 119365524
+3922 64705274
+3923 151757145
+3924 90824050
+3925 152245676
+3926 115777440
+3927 163515343
+3928 26156712
+3929 150905780
+3930 73560531
+3931 172536320
+3932 108057388
+3933 165768765
+3934 141263878
+3935 176260371
+3936 21942088
+3937 107555201
+3938 62268749
+3939 138199607
+3940 82465961
+3941 140193308
+3942 110310594
+3943 157889133
+3944 22933325
+3945 142090051
+3946 66132058
+3947 161192310
+3948 99645335
+3949 157021730
+3950 128658914
+3951 171411723
+3952 35270945
+3953 127187831
+3954 80057312
+3955 156845398
+3956 101331709
+3957 156490842
+3958 128083673
+3959 170342057
+3960 38086484
+3961 160501835
+3962 89844274
+3963 177897904
+3964 120122911
+3965 170382486
+3966 148410215
+3967 184966002
+3968 5237981
+3969 93411871
+3970 25319643
+3971 123864599
+3972 62432373
+3973 126750787
+3974 87295995
+3975 143712293
+3976 6594523
+3977 123879692
+3978 29920319
+3979 149917197
+3980 78206479
+3981 147166497
+3982 106047644
+3983 157156928
+3984 10859551
+3985 111691920
+3986 40631403
+3987 140513806
+3988 76634426
+3989 139063133
+3990 103771092
+3991 153582149
+3992 13641885
+3993 143790093
+3994 49695303
+3995 163514194
+3996 96529521
+3997 155556407
+3998 123318955
+3999 169289615
+4000 10112943
+4001 98439518
+4002 38567208
+4003 129333738
+4004 69035501
+4005 133651770
+4006 96969489
+4007 150635675
+4008 11249013
+4009 131233435
+4010 43363397
+4011 156228080
+4012 86040917
+4013 151548517
+4014 114912316
+4015 160874557
+4016 18240181
+4017 118555607
+4018 56463158
+4019 147103894
+4020 85801277
+4021 146905477
+4022 114321362
+4023 162251141
+4024 21145968
+4025 146676116
+4026 65586396
+4027 169390668
+4028 102946385
+4029 160439373
+4030 132905650
+4031 173373924
+4032 12954347
+4033 102563760
+4034 45569611
+4035 131327590
+4036 72940525
+4037 134777311
+4038 101461656
+4039 152372568
+4040 14264745
+4041 132941325
+4042 50226554
+4043 156912202
+4044 90498278
+4045 153052604
+4046 118498746
+4047 166768968
+4048 23284733
+4049 120538340
+4050 64549010
+4051 150848894
+4052 90680852
+4053 147981786
+4054 117662854
+4055 164602162
+4056 26085488
+4057 152423406
+4058 73920499
+4059 170538392
+4060 109528224
+4061 167335477
+4062 137072713
+4063 175327679
+4064 21847541
+4065 108696160
+4066 61227549
+4067 136930815
+4068 83359820
+4069 141490316
+4070 110275654
+4071 158333630
+4072 23189254
+4073 139009910
+4074 64714534
+4075 164912506
+4076 101215697
+4077 157685236
+4078 128683417
+4079 171643294
+4080 35611916
+4081 129566457
+4082 79616243
+4083 153844045
+4084 101873461
+4085 157591516
+4086 127786211
+4087 172202256
+4088 38748208
+4089 159566906
+4090 87797284
+4091 176448362
+4092 120944029
+4093 171350081
+4094 146605383
+4095 183301627
diff --git a/src/modeling-service/src/main/java/org/apache/kylin/rest/service/FusionIndexService.java b/src/modeling-service/src/main/java/org/apache/kylin/rest/service/FusionIndexService.java
index 491c17120a..80043d3fd5 100644
--- a/src/modeling-service/src/main/java/org/apache/kylin/rest/service/FusionIndexService.java
+++ b/src/modeling-service/src/main/java/org/apache/kylin/rest/service/FusionIndexService.java
@@ -34,8 +34,6 @@ import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.job.constant.JobStatusEnum;
 import org.apache.kylin.job.execution.JobTypeEnum;
-import org.apache.kylin.rest.response.AggIndexResponse;
-import org.apache.kylin.rest.response.DiffRuleBasedIndexResponse;
 import org.apache.kylin.metadata.cube.cuboid.NAggregationGroup;
 import org.apache.kylin.metadata.cube.model.IndexEntity;
 import org.apache.kylin.metadata.cube.model.IndexEntity.Range;
@@ -53,7 +51,9 @@ import org.apache.kylin.rest.aspect.Transaction;
 import org.apache.kylin.rest.request.AggShardByColumnsRequest;
 import org.apache.kylin.rest.request.CreateTableIndexRequest;
 import org.apache.kylin.rest.request.UpdateRuleBasedCuboidRequest;
+import org.apache.kylin.rest.response.AggIndexResponse;
 import org.apache.kylin.rest.response.BuildIndexResponse;
+import org.apache.kylin.rest.response.DiffRuleBasedIndexResponse;
 import org.apache.kylin.rest.response.IndexResponse;
 import org.apache.kylin.streaming.manager.StreamingJobManager;
 import org.apache.kylin.streaming.metadata.StreamingJobMeta;
@@ -416,7 +416,8 @@ public class FusionIndexService extends BasicService {
             String modelId) {
         if ((streamResponse.getDecreaseLayouts() > 0 || streamResponse.getIncreaseLayouts() > 0)
                 && checkStreamingJobAndSegments(project, modelId)) {
-            throw new KylinException(ServerErrorCode.STREAMING_INDEX_UPDATE_DISABLE, MsgPicker.getMsg().getStreamingIndexesEdit());
+            throw new KylinException(ServerErrorCode.STREAMING_INDEX_UPDATE_DISABLE,
+                    MsgPicker.getMsg().getStreamingIndexesEdit());
         }
     }
 
@@ -434,7 +435,8 @@ public class FusionIndexService extends BasicService {
     private static void checkStreamingIndexEnabled(String project, NDataModel model) throws KylinException {
         if (NDataModel.ModelType.STREAMING == model.getModelType()
                 && checkStreamingJobAndSegments(project, model.getUuid())) {
-            throw new KylinException(ServerErrorCode.STREAMING_INDEX_UPDATE_DISABLE, MsgPicker.getMsg().getStreamingIndexesDelete());
+            throw new KylinException(ServerErrorCode.STREAMING_INDEX_UPDATE_DISABLE,
+                    MsgPicker.getMsg().getStreamingIndexesDelete());
         }
     }
 
diff --git a/src/modeling-service/src/main/java/org/apache/kylin/rest/service/IndexPlanService.java b/src/modeling-service/src/main/java/org/apache/kylin/rest/service/IndexPlanService.java
index 04b1a93bb3..e45016aeeb 100644
--- a/src/modeling-service/src/main/java/org/apache/kylin/rest/service/IndexPlanService.java
+++ b/src/modeling-service/src/main/java/org/apache/kylin/rest/service/IndexPlanService.java
@@ -260,7 +260,7 @@ public class IndexPlanService extends BasicService implements TableIndexPlanSupp
         return createTableIndex(project, request.getModelId(), newLayout, request.isLoadData());
     }
 
-    public BuildIndexResponse createTableIndex(String project, String modelId, LayoutEntity newLayout,
+    private BuildIndexResponse createTableIndex(String project, String modelId, LayoutEntity newLayout,
             boolean loadData) {
         NIndexPlanManager indexPlanManager = getManager(NIndexPlanManager.class, project);
         val jobManager = getManager(JobManager.class, project);
@@ -272,6 +272,7 @@ public class IndexPlanService extends BasicService implements TableIndexPlanSupp
         }
         int layoutIndex = indexPlan.getWhitelistLayouts().indexOf(newLayout);
         if (layoutIndex != -1) {
+            // find the target layout in this index plan
             indexPlanManager.updateIndexPlan(indexPlan.getUuid(), copyForWrite -> {
                 val oldLayout = copyForWrite.getWhitelistLayouts().get(layoutIndex);
                 oldLayout.setManual(true);
@@ -281,6 +282,8 @@ public class IndexPlanService extends BasicService implements TableIndexPlanSupp
             modelChangeSupporters.forEach(listener -> listener.onUpdate(project, modelId));
             return new BuildIndexResponse(BuildIndexResponse.BuildIndexType.NO_LAYOUT);
         } else {
+            // create a new index for this layout
+            // update this index plan with the new layout
             indexPlanManager.updateIndexPlan(indexPlan.getUuid(), copyForWrite -> {
                 val newCuboid = new IndexEntity();
                 newCuboid.setId(newLayout.getId() - 1);
@@ -456,11 +459,13 @@ public class IndexPlanService extends BasicService implements TableIndexPlanSupp
     public AggIndexResponse calculateAggIndexCount(UpdateRuleBasedCuboidRequest request) {
         aclEvaluate.checkProjectWritePermission(request.getProject());
         val maxCount = getConfig().getCubeAggrGroupMaxCombination();
+        // The agg group for updates which includes all agg group for the index
         List<NAggregationGroup> aggregationGroups = request.getAggregationGroups();
         val indexPlan = getIndexPlan(request.getProject(), request.getModelId()).copy();
         AggIndexCombResult totalResult;
         AggIndexCombResult aggIndexResult;
 
+        // Filter the invalid agg group
         val aggregationGroupsCopy = aggregationGroups.stream()
                 .filter(aggGroup -> aggGroup.getIncludes() != null && aggGroup.getIncludes().length != 0)
                 .collect(Collectors.toList());
@@ -484,7 +489,6 @@ public class IndexPlanService extends BasicService implements TableIndexPlanSupp
                                     StringUtils.join(notExistCols.iterator(), ",")));
                 }
             }
-
             indexPlan.setRuleBasedIndex(ruleBasedIndex);
         } catch (OutOfMaxCombinationException oe) {
             invalid = true;
diff --git a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/JobStepType.java b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/JobStepType.java
index 38f14c56b9..8191f57fe8 100644
--- a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/JobStepType.java
+++ b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/JobStepType.java
@@ -25,9 +25,11 @@ import org.apache.kylin.job.SecondStorageStepFactory;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.DefaultExecutable;
 import org.apache.kylin.job.execution.DefaultExecutableOnModel;
+import org.apache.kylin.metadata.cube.model.NBatchConstants;
 
 import static org.apache.kylin.engine.spark.job.StageType.BUILD_DICT;
 import static org.apache.kylin.engine.spark.job.StageType.BUILD_LAYER;
+import static org.apache.kylin.engine.spark.job.StageType.COST_BASED_PLANNER;
 import static org.apache.kylin.engine.spark.job.StageType.GATHER_FLAT_TABLE_STATS;
 import static org.apache.kylin.engine.spark.job.StageType.GENERATE_FLAT_TABLE;
 import static org.apache.kylin.engine.spark.job.StageType.MATERIALIZED_FACT_TABLE;
@@ -83,6 +85,10 @@ public enum JobStepType {
             MATERIALIZED_FACT_TABLE.createStage(parent, config);
             BUILD_DICT.createStage(parent, config);
             GENERATE_FLAT_TABLE.createStage(parent, config);
+            String enablePlanner = parent.getParam(NBatchConstants.P_JOB_ENABLE_PLANNER);
+            if (enablePlanner != null && Boolean.valueOf(enablePlanner)) {
+                COST_BASED_PLANNER.createStage(parent, config);
+            }
             GATHER_FLAT_TABLE_STATS.createStage(parent, config);
             BUILD_LAYER.createStage(parent, config);
             REFRESH_COLUMN_BYTES.createStage(parent, config);
diff --git a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/NSparkCubingJob.java b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/NSparkCubingJob.java
index 00dfe98882..b1cfb7cde6 100644
--- a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/NSparkCubingJob.java
+++ b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/NSparkCubingJob.java
@@ -18,23 +18,22 @@
 
 package org.apache.kylin.engine.spark.job;
 
-import static java.util.stream.Collectors.joining;
-import static org.apache.kylin.engine.spark.stats.utils.HiveTableRefChecker.isNeedCleanUpTransactionalTableJob;
-import static org.apache.kylin.job.factory.JobFactoryConstant.CUBE_JOB_FACTORY;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.stream.Collectors;
-
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import io.kyligence.kap.guava20.shaded.common.annotations.VisibleForTesting;
+import io.kyligence.kap.secondstorage.SecondStorageConstants;
+import io.kyligence.kap.secondstorage.SecondStorageUtil;
+import io.kyligence.kap.secondstorage.enums.LockTypeEnum;
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.val;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.KylinConfigExt;
+import org.apache.kylin.common.exception.JobErrorCode;
+import org.apache.kylin.common.exception.KylinException;
 import org.apache.kylin.common.util.RandomUtil;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.DefaultExecutableOnModel;
@@ -42,6 +41,7 @@ import org.apache.kylin.job.execution.ExecutableParams;
 import org.apache.kylin.job.execution.JobSchedulerModeEnum;
 import org.apache.kylin.job.execution.JobTypeEnum;
 import org.apache.kylin.job.factory.JobFactory;
+import org.apache.kylin.metadata.cube.model.IndexPlan;
 import org.apache.kylin.metadata.cube.model.LayoutEntity;
 import org.apache.kylin.metadata.cube.model.NBatchConstants;
 import org.apache.kylin.metadata.cube.model.NDataSegment;
@@ -54,17 +54,18 @@ import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
 
-import io.kyligence.kap.guava20.shaded.common.annotations.VisibleForTesting;
-import io.kyligence.kap.secondstorage.SecondStorageConstants;
-import io.kyligence.kap.secondstorage.SecondStorageUtil;
-import io.kyligence.kap.secondstorage.enums.LockTypeEnum;
-import lombok.AllArgsConstructor;
-import lombok.Getter;
-import lombok.val;
+import static java.util.stream.Collectors.joining;
+import static org.apache.kylin.engine.spark.stats.utils.HiveTableRefChecker.isNeedCleanUpTransactionalTableJob;
+import static org.apache.kylin.job.factory.JobFactoryConstant.CUBE_JOB_FACTORY;
 
 public class NSparkCubingJob extends DefaultExecutableOnModel {
 
@@ -101,6 +102,13 @@ public class NSparkCubingJob extends DefaultExecutableOnModel {
                 buckets);
     }
 
+    @VisibleForTesting
+    public static NSparkCubingJob createIncBuildJob(Set<NDataSegment> segments, Set<LayoutEntity> layouts, String submitter,
+                                                    Set<JobBucket> buckets) {
+        return create(segments, layouts, submitter, JobTypeEnum.INC_BUILD, RandomUtil.randomUUIDStr(), null, null,
+                buckets);
+    }
+
     @VisibleForTesting
     public static NSparkCubingJob create(Set<NDataSegment> segments, Set<LayoutEntity> layouts, String submitter,
             JobTypeEnum jobType, String jobId, Set<String> ignoredSnapshotTables, Set<Long> partitions,
@@ -164,6 +172,9 @@ public class NSparkCubingJob extends DefaultExecutableOnModel {
         if (CollectionUtils.isNotEmpty(buckets)) {
             job.setParam(NBatchConstants.P_BUCKETS, ExecutableParams.toBucketParam(buckets));
         }
+
+        enableCostBasedPlannerIfNeed(df, segments, job);
+
         job.setParam(NBatchConstants.P_JOB_ID, jobId);
         job.setParam(NBatchConstants.P_PROJECT_NAME, df.getProject());
         job.setParam(NBatchConstants.P_TARGET_MODEL, job.getTargetSubject());
@@ -422,4 +433,55 @@ public class NSparkCubingJob extends DefaultExecutableOnModel {
         private final AbstractExecutable secondStorage;
         private final AbstractExecutable cleanUpTransactionalTable;
     }
+
+    private static void enableCostBasedPlannerIfNeed(NDataflow df, Set<NDataSegment> segments, NSparkCubingJob job) {
+        // need run the cost based planner:
+        // 1. config enable the cube planner
+        // 2. the model dose not have the `layout_cost_based_pruned_list`
+        // 3. rule index has agg group
+        // 4. just only one segment to be built/refresh(other case will throw exception)
+        IndexPlan indexPlan = df.getIndexPlan();
+        KylinConfig kylinConfig = indexPlan.getConfig();
+        boolean needCostRecommendIndex = indexPlan.getRuleBasedIndex() != null
+                && indexPlan.getRuleBasedIndex().getLayoutsOfCostBasedList() == null
+                && !indexPlan.getRuleBasedIndex().getAggregationGroups().isEmpty();
+        if (kylinConfig.enableCostBasedIndexPlanner() && needCostRecommendIndex
+                && canEnablePlannerJob(job.getJobType())) {
+            // must run the cost based planner
+            if (segments.size() == 1) {
+                if (noBuildingSegmentExist(df.getProject(), job.getTargetSubject(), kylinConfig)) {
+                    // check the count of rowkey:
+                    // if the count of row key exceed the 63, throw exception
+                    if (indexPlan.getRuleBasedIndex().countOfIncludeDimension() > (Long.SIZE - 1)) {
+                        throw new KylinException(JobErrorCode.COST_BASED_PLANNER_ERROR,
+                                String.format(Locale.ROOT,
+                                        "The count of row key %d can't be larger than 63, when use the cube planner",
+                                        indexPlan.getRuleBasedIndex().countOfIncludeDimension()));
+                    }
+                    // Add the parameter `P_JOB_ENABLE_PLANNER` which is used to decide whether to use the  cube planner
+                    job.setParam(NBatchConstants.P_JOB_ENABLE_PLANNER, Boolean.TRUE.toString());
+                } else {
+                    throw new KylinException(JobErrorCode.COST_BASED_PLANNER_ERROR, String.format(Locale.ROOT,
+                            "There are running job for this model when submit the build job with cost based planner, "
+                                    + "please wait for other jobs to finish or cancel them"));
+                }
+            } else {
+                throw new KylinException(JobErrorCode.COST_BASED_PLANNER_ERROR,
+                        String.format(Locale.ROOT, "The number of segments to be built or refreshed must be 1, "
+                                + "This is the first time to submit build job with enable cost based planner"));
+            }
+        }
+    }
+
+    private static boolean noBuildingSegmentExist(String project, String modelId, KylinConfig kylinConfig) {
+        NDataflowManager nDataflowManager = NDataflowManager.getInstance(kylinConfig, project);
+        NDataflow dataflow = nDataflowManager.getDataflow(modelId);
+        // There are no other tasks in building
+        return dataflow.getSegments(SegmentStatusEnum.NEW).size() <= 1;
+    }
+
+    private static boolean canEnablePlannerJob(JobTypeEnum jobType) {
+        // just support: INC_BUILD and INDEX_REFRESH to recommend/prune index
+        return JobTypeEnum.INC_BUILD.equals(jobType) || JobTypeEnum.INDEX_REFRESH.equals(jobType);
+    }
 }
diff --git a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/StageType.java b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/StageType.java
index 431ab31693..6bc41da5ed 100644
--- a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/StageType.java
+++ b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/StageType.java
@@ -32,6 +32,7 @@ import org.apache.kylin.engine.spark.job.stage.StageExec;
 import org.apache.kylin.engine.spark.job.stage.WaiteForResource;
 import org.apache.kylin.engine.spark.job.stage.build.BuildDict;
 import org.apache.kylin.engine.spark.job.stage.build.BuildLayer;
+import org.apache.kylin.engine.spark.job.stage.build.CostBasedPlanner;
 import org.apache.kylin.engine.spark.job.stage.build.GatherFlatTableStats;
 import org.apache.kylin.engine.spark.job.stage.build.GenerateFlatTable;
 import org.apache.kylin.engine.spark.job.stage.build.MaterializedFactTableView;
@@ -39,6 +40,7 @@ import org.apache.kylin.engine.spark.job.stage.build.RefreshColumnBytes;
 import org.apache.kylin.engine.spark.job.stage.build.RefreshSnapshots;
 import org.apache.kylin.engine.spark.job.stage.build.partition.PartitionBuildDict;
 import org.apache.kylin.engine.spark.job.stage.build.partition.PartitionBuildLayer;
+import org.apache.kylin.engine.spark.job.stage.build.partition.PartitionCostBasedPlanner;
 import org.apache.kylin.engine.spark.job.stage.build.partition.PartitionGatherFlatTableStats;
 import org.apache.kylin.engine.spark.job.stage.build.partition.PartitionGenerateFlatTable;
 import org.apache.kylin.engine.spark.job.stage.build.partition.PartitionMaterializedFactTableView;
@@ -120,6 +122,20 @@ public enum StageType {
             return new NStageForBuild(ExecutableConstants.STAGE_NAME_GENERATE_FLAT_TABLE);
         }
     },
+    COST_BASED_PLANNER {
+        @Override
+        public StageExec create(SparkApplication jobContext, NDataSegment dataSegment, BuildParam buildParam) {
+            if (isPartitioned(jobContext)) {
+                return new PartitionCostBasedPlanner((SegmentJob) jobContext, dataSegment, buildParam);
+            }
+            return new CostBasedPlanner((SegmentJob) jobContext, dataSegment, buildParam);
+        }
+
+        @Override
+        protected StageBase create(NSparkExecutable parent, KylinConfig config) {
+            return new NStageForBuild(ExecutableConstants.STAGE_NAME_COST_BASED_PLANNER);
+        }
+    },
     GATHER_FLAT_TABLE_STATS {
         @Override
         public StageExec create(SparkApplication jobContext, NDataSegment dataSegment, BuildParam buildParam) {
diff --git a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/merger/AfterBuildResourceMerger.java b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/merger/AfterBuildResourceMerger.java
index 117d381df8..cfd0d474ad 100644
--- a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/merger/AfterBuildResourceMerger.java
+++ b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/merger/AfterBuildResourceMerger.java
@@ -110,7 +110,6 @@ public class AfterBuildResourceMerger extends SparkJobMetadataMerger {
         dfUpdate.setToUpdateSegs(theSeg);
         dfUpdate.setToRemoveSegs(toRemoveSegments.toArray(new NDataSegment[toRemoveSegments.size()]));
         dfUpdate.setToAddOrUpdateLayouts(theSeg.getSegDetails().getLayouts().toArray(new NDataLayout[0]));
-
         localDataflowManager.updateDataflow(dfUpdate);
         updateIndexPlan(flowName, remoteStore);
         return dfUpdate.getToAddOrUpdateLayouts();
diff --git a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/merger/AfterMergeOrRefreshResourceMerger.java b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/merger/AfterMergeOrRefreshResourceMerger.java
index b9b4414685..9ac88ccb0b 100644
--- a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/merger/AfterMergeOrRefreshResourceMerger.java
+++ b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/merger/AfterMergeOrRefreshResourceMerger.java
@@ -26,21 +26,23 @@ import java.util.Set;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.engine.spark.ExecutableUtils;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.JobTypeEnum;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.apache.kylin.engine.spark.ExecutableUtils;
 import org.apache.kylin.metadata.cube.model.NDataLayout;
 import org.apache.kylin.metadata.cube.model.NDataSegment;
 import org.apache.kylin.metadata.cube.model.NDataflow;
 import org.apache.kylin.metadata.cube.model.NDataflowManager;
 import org.apache.kylin.metadata.cube.model.NDataflowUpdate;
 import org.apache.kylin.metadata.cube.model.PartitionStatusEnum;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
 
 import com.clearspring.analytics.util.Lists;
 
 import lombok.val;
+import lombok.extern.slf4j.Slf4j;
 
+@Slf4j
 public class AfterMergeOrRefreshResourceMerger extends SparkJobMetadataMerger {
 
     public AfterMergeOrRefreshResourceMerger(KylinConfig config, String project) {
diff --git a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/merger/SparkJobMetadataMerger.java b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/merger/SparkJobMetadataMerger.java
index e9cf08bec7..8403c976ab 100644
--- a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/merger/SparkJobMetadataMerger.java
+++ b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/merger/SparkJobMetadataMerger.java
@@ -33,7 +33,6 @@ import org.apache.kylin.job.dao.JobStatisticsManager;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.JobTypeEnum;
 import org.apache.kylin.job.execution.NExecutableManager;
-import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.cube.model.IndexPlan;
 import org.apache.kylin.metadata.cube.model.LayoutEntity;
 import org.apache.kylin.metadata.cube.model.LayoutPartition;
@@ -45,6 +44,7 @@ import org.apache.kylin.metadata.cube.model.NIndexPlanManager;
 import org.apache.kylin.metadata.cube.model.PartitionStatusEnum;
 import org.apache.kylin.metadata.cube.model.SegmentPartition;
 import org.apache.kylin.metadata.model.NTableMetadataManager;
+import org.apache.kylin.metadata.project.ProjectInstance;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -214,6 +214,20 @@ public abstract class SparkJobMetadataMerger extends MetadataMerger {
         NIndexPlanManager indexPlanManager = NIndexPlanManager.getInstance(getConfig(), getProject());
         indexPlanManager.updateIndexPlan(dfId, copyForWrite -> {
             copyForWrite.setLayoutBucketNumMapping(remoteIndexPlan.getLayoutBucketNumMapping());
+            // This is used for the cube planner
+            // In the function of `updateIndexPlanIfNeed`, we may add recommended index for this index plan.
+            // We need to update the `RuleBasedIndex` for the index plan to kylin metadata
+            val remoteRuleIndex = remoteIndexPlan.getRuleBasedIndex();
+            val currentRuleIndex = copyForWrite.getRuleBasedIndex();
+            if (remoteRuleIndex != null && currentRuleIndex != null) {
+                // remote store has the recommended index
+                // current store does not contain the recommended index
+                if (remoteRuleIndex.getLayoutsOfCostBasedList() != null
+                        && currentRuleIndex.getLayoutsOfCostBasedList() == null) {
+                    currentRuleIndex.setLayoutsOfCostBasedList(remoteRuleIndex.getLayoutsOfCostBasedList());
+                }
+                copyForWrite.setRuleBasedIndex(currentRuleIndex);
+            }
         });
     }
 }
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/PartitionExec.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/PartitionExec.scala
index 324e981191..887b252c01 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/PartitionExec.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/PartitionExec.scala
@@ -137,7 +137,7 @@ private[job] trait PartitionExec {
           dataLayout.setByteSize(partitions.map(_.getByteSize).sum)
           dataLayout
         }.toSeq
-
+        logInfo(s"Partition segment $segmentId update the data layouts $dataLayouts")
         updateDataLayouts(manager, dataLayouts)
       }
     }
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/RDSegmentBuildJob.java b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/RDSegmentBuildJob.java
index 1a553eaf9b..f4809375ab 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/RDSegmentBuildJob.java
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/RDSegmentBuildJob.java
@@ -65,6 +65,6 @@ public class RDSegmentBuildJob extends SegmentJob implements ResourceDetect {
 
     private void writeCountDistinct() {
         ResourceDetectUtils.write(new Path(rdSharedPath, ResourceDetectUtils.countDistinctSuffix()), //
-                ResourceDetectUtils.findCountDistinctMeasure(readOnlyLayouts));
+                ResourceDetectUtils.findCountDistinctMeasure(getReadOnlyLayouts()));
     }
 }
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentBuildJob.java b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentBuildJob.java
index a6be082423..b9663c8d26 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentBuildJob.java
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentBuildJob.java
@@ -29,10 +29,10 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.transaction.UnitOfWork;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.engine.spark.builder.SnapshotBuilder;
-import org.apache.kylin.engine.spark.job.LogJobInfoUtils;
 import org.apache.kylin.engine.spark.job.exec.BuildExec;
 import org.apache.kylin.engine.spark.job.stage.BuildParam;
 import org.apache.kylin.engine.spark.job.stage.StageExec;
+import org.apache.kylin.metadata.cube.model.NBatchConstants;
 import org.apache.kylin.metadata.cube.model.NDataSegment;
 import org.apache.kylin.metadata.cube.model.NDataflow;
 import org.apache.kylin.metadata.cube.model.NDataflowManager;
@@ -50,6 +50,7 @@ import java.util.stream.Stream;
 
 import static org.apache.kylin.engine.spark.job.StageType.BUILD_DICT;
 import static org.apache.kylin.engine.spark.job.StageType.BUILD_LAYER;
+import static org.apache.kylin.engine.spark.job.StageType.COST_BASED_PLANNER;
 import static org.apache.kylin.engine.spark.job.StageType.GATHER_FLAT_TABLE_STATS;
 import static org.apache.kylin.engine.spark.job.StageType.GENERATE_FLAT_TABLE;
 import static org.apache.kylin.engine.spark.job.StageType.MATERIALIZED_FACT_TABLE;
@@ -60,11 +61,22 @@ import static org.apache.kylin.engine.spark.job.StageType.WAITE_FOR_RESOURCE;
 @Slf4j
 public class SegmentBuildJob extends SegmentJob {
 
+    private boolean usePlanner = false;
+
     public static void main(String[] args) {
         SegmentBuildJob segmentBuildJob = new SegmentBuildJob();
         segmentBuildJob.execute(args);
     }
 
+    @Override
+    protected final void extraInit() {
+        super.extraInit();
+        String enablePlanner = getParam(NBatchConstants.P_JOB_ENABLE_PLANNER);
+        if (enablePlanner != null && Boolean.valueOf(enablePlanner)) {
+            usePlanner = true;
+        }
+    }
+
     @Override
     protected String generateInfo() {
         return LogJobInfoUtils.dfBuildJobInfo();
@@ -135,6 +147,11 @@ public class SegmentBuildJob extends SegmentJob {
                 MATERIALIZED_FACT_TABLE.createStage(this, seg, buildParam, exec);
                 BUILD_DICT.createStage(this, seg, buildParam, exec);
                 GENERATE_FLAT_TABLE.createStage(this, seg, buildParam, exec);
+                // enable cost based planner according to the parameter
+                if (usePlanner) {
+                    COST_BASED_PLANNER.createStage(this, seg, buildParam, exec);
+                }
+
                 GATHER_FLAT_TABLE_STATS.createStage(this, seg, buildParam, exec);
                 BUILD_LAYER.createStage(this, seg, buildParam, exec);
 
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentExec.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentExec.scala
index 800c90e853..e0f635acad 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentExec.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentExec.scala
@@ -229,6 +229,7 @@ trait SegmentExec extends Logging {
           dataLayout.setByteSize(taskStats.numBytes)
           dataLayout
         }
+        logInfo(s"Segment $segmentId update the data layouts $dataLayouts")
         updateDataLayouts(manager, dataLayouts)
       }
     }
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentJob.java b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentJob.java
index 479428f09a..d4dfddbee1 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentJob.java
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentJob.java
@@ -20,8 +20,10 @@ package org.apache.kylin.engine.spark.job;
 
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.function.Predicate;
@@ -29,6 +31,7 @@ import java.util.stream.Collectors;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.persistence.transaction.UnitOfWork;
 import org.apache.kylin.engine.spark.application.SparkApplication;
 import org.apache.kylin.engine.spark.scheduler.JobRuntime;
 import org.apache.kylin.job.execution.ExecutableParams;
@@ -38,12 +41,18 @@ import org.apache.kylin.metadata.cube.model.NBatchConstants;
 import org.apache.kylin.metadata.cube.model.NDataSegment;
 import org.apache.kylin.metadata.cube.model.NDataflow;
 import org.apache.kylin.metadata.cube.model.NDataflowManager;
+import org.apache.kylin.metadata.cube.model.NIndexPlanManager;
 import org.apache.kylin.metadata.job.JobBucket;
 import org.apache.spark.tracker.BuildContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import lombok.val;
 
 public abstract class SegmentJob extends SparkApplication {
 
@@ -54,9 +63,8 @@ public abstract class SegmentJob extends SparkApplication {
     protected IndexPlan indexPlan;
 
     protected String dataflowId;
-
-    protected Set<LayoutEntity> readOnlyLayouts;
-
+    // In order to support the cost based planner, `readOnlyLayouts` can be changed
+    private Set<LayoutEntity> readOnlyLayouts;
     protected Set<NDataSegment> readOnlySegments;
 
     // Resource detection results output path
@@ -76,8 +84,71 @@ public abstract class SegmentJob extends SparkApplication {
         return readOnlyLayouts;
     }
 
+    private Set<List<Integer>> recommendAggColOrders = new HashSet<>();
+
+    public void setRecommendAggColOrders(Set<List<Integer>> recommendAggColOrders) {
+        this.recommendAggColOrders = recommendAggColOrders;
+    }
+
+    public Set<List<Integer>> getRecommendAggColOrders() {
+        return recommendAggColOrders;
+    }
+
+    public boolean updateIndexPlanIfNeed() {
+        // when run the cost based index planner, there will be some recommended index layouts for this model
+        if (getRecommendAggColOrders().size() != 0) {
+            UnitOfWork.doInTransactionWithRetry(() -> {
+                // update and add the recommended index layout to the index plan
+                val recommendAggLayouts = Lists.newArrayList(getRecommendAggColOrders());
+                NIndexPlanManager indexPlanManager = NIndexPlanManager.getInstance(config, project);
+                logger.debug("Update the index plan and add recommended agg index {}", recommendAggLayouts);
+                indexPlanManager.updateIndexPlan(dataflowId, copyForWrite -> {
+                    // construct the map: colOrder of layout -> id
+                    val allRuleLayouts = copyForWrite.getRuleBasedIndex().genCuboidLayouts();
+                    Map<List<Integer>, Long> colOrder2Id = Maps.newHashMap();
+                    allRuleLayouts.forEach(layoutEntity -> {
+                        colOrder2Id.put(layoutEntity.getColOrder(), layoutEntity.getId());
+                    });
+                    logger.debug("All rule base layouts {}", allRuleLayouts);
+                    Set<Long> costBasedResult = Sets.newHashSet();
+                    for (List<Integer> colOrder : recommendAggLayouts) {
+                        if (colOrder2Id.containsKey(colOrder)) {
+                            costBasedResult.add(colOrder2Id.get(colOrder));
+                        } else {
+                            logger.debug("Can't find the layout {} in the rule base index", colOrder);
+                        }
+                    }
+                    // reset the rule base layouts
+                    logger.debug("Set the rule pruning cost based list layouts {}", costBasedResult);
+                    val ruleBaseIndex = copyForWrite.getRuleBasedIndex();
+                    ruleBaseIndex.setLayoutsOfCostBasedList(costBasedResult);
+                    copyForWrite.setRuleBasedIndex(ruleBaseIndex);
+                });
+                return null;
+            }, project);
+            updateJobLayoutsIfNeed();
+            return true;
+        } else {
+            logger.info("There is no recommended agg index");
+            return false;
+        }
+    }
+
+    private void updateJobLayoutsIfNeed() {
+        // update this job layouts when there are recommended index layouts for this build
+        NDataflowManager dataflowManager = NDataflowManager.getInstance(config, project);
+        // get the new index plan
+        indexPlan = dataflowManager.getDataflow(dataflowId).getIndexPlan();
+        // get the new layout
+        val newJobLayouts = indexPlan.getAllLayouts();
+        logger.debug("Update Job layouts count from {} to {}", readOnlyLayouts.size(), newJobLayouts.size());
+        readOnlyLayouts = new HashSet<>(newJobLayouts);
+        // rewrite the `P_LAYOUT_IDS` parameters
+        setParam(NBatchConstants.P_LAYOUT_IDS, NSparkCubingUtil.ids2Str(NSparkCubingUtil.toLayoutIds(readOnlyLayouts)));
+    }
+
     @Override
-    protected final void extraInit() {
+    protected void extraInit() {
         partialBuild = Boolean.parseBoolean(getParam(NBatchConstants.P_PARTIAL_BUILD));
         Set<String> segmentIDs = Arrays.stream(getParam(NBatchConstants.P_SEGMENT_IDS).split(COMMA))
                 .collect(Collectors.toCollection(LinkedHashSet::new));
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/BuildStage.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/BuildStage.scala
index 49d53d7b31..414335d9fa 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/BuildStage.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/BuildStage.scala
@@ -60,7 +60,12 @@ abstract class BuildStage(private val jobContext: SegmentJob,
   protected final val sparkSession = jobContext.getSparkSession
   protected final val resourceContext = jobContext.getBuildContext
   protected final val runtime = jobContext.getRuntime
-  protected final val readOnlyLayouts = jobContext.getReadOnlyLayouts
+
+  // These parameters can be changed when running the cube planner, should use the
+  // `def` to get the latest data
+  protected def readOnlyLayouts = jobContext.getReadOnlyLayouts
+  private def spanningTree = buildParam.getSpanningTree
+  private def flatTableDesc = buildParam.getFlatTableDesc
 
   // Needed variables from data segment.
   protected final val segmentId = dataSegment.getId
@@ -69,10 +74,6 @@ abstract class BuildStage(private val jobContext: SegmentJob,
   protected final val dataModel = dataSegment.getModel
   protected final val storageType = dataModel.getStorageType
 
-  private lazy val spanningTree = buildParam.getSpanningTree
-
-  private lazy val flatTableDesc = buildParam.getFlatTableDesc
-
   private lazy val flatTable = buildParam.getBuildFlatTable
 
   private lazy val flatTableDS: Dataset[Row] = buildParam.getFlatTable
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/MaterializedFactTableView.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/CostBasedPlanner.scala
similarity index 57%
copy from src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/MaterializedFactTableView.scala
copy to src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/CostBasedPlanner.scala
index 584538f47d..08ba41e46d 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/MaterializedFactTableView.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/CostBasedPlanner.scala
@@ -25,33 +25,29 @@ import org.apache.kylin.engine.spark.smarter.IndexDependencyParser
 import org.apache.kylin.metadata.cube.cuboid.AdaptiveSpanningTree
 import org.apache.kylin.metadata.cube.cuboid.AdaptiveSpanningTree.AdaptiveTreeBuilder
 import org.apache.kylin.metadata.cube.model.NDataSegment
-import org.apache.spark.sql.{Dataset, Row}
 
-class MaterializedFactTableView(jobContext: SegmentJob, dataSegment: NDataSegment, buildParam: BuildParam)
+class CostBasedPlanner(jobContext: SegmentJob, dataSegment: NDataSegment, buildParam: BuildParam)
   extends FlatTableAndDictBase(jobContext, dataSegment, buildParam) {
-
   override def execute(): Unit = {
-    logInfo(s"Build SEGMENT $segmentId")
-    val spanTree = new AdaptiveSpanningTree(config, new AdaptiveTreeBuilder(dataSegment, readOnlyLayouts))
-    buildParam.setSpanningTree(spanTree)
-
-    val flatTableDesc: SegmentFlatTableDesc = if (jobContext.isPartialBuild) {
-      val parser = new IndexDependencyParser(dataModel)
-      val relatedTableAlias =
-        parser.getRelatedTablesAlias(jobContext.getReadOnlyLayouts)
-      new SegmentFlatTableDesc(config, dataSegment, spanningTree, relatedTableAlias)
-    } else {
-      new SegmentFlatTableDesc(config, dataSegment, spanningTree)
-    }
-    buildParam.setFlatTableDesc(flatTableDesc)
+    val (cost, sourceCount) = generateCostTable()
+    getRecommendedLayoutAndUpdateMetadata(cost, sourceCount)
 
-    val factTableDS: Dataset[Row] = newFactTableDS()
-    buildParam.setFactTableDS(factTableDS)
+    val result = jobContext.updateIndexPlanIfNeed()
+    if (result) {
+      // update span tree and table desc with the new build job layouts
+      val spanTree = new AdaptiveSpanningTree(config, new AdaptiveTreeBuilder(dataSegment, readOnlyLayouts))
+      buildParam.setSpanningTree(spanTree)
 
-    val fastFactTableDS: Dataset[Row] = newFastFactTableDS()
-    buildParam.setFastFactTableDS(fastFactTableDS)
-    if (buildParam.isSkipMaterializedFactTableView) {
-      onStageSkipped()
+      // update table desc with new span tree
+      val flatTableDesc: SegmentFlatTableDesc = if (jobContext.isPartialBuild) {
+        val parser = new IndexDependencyParser(dataModel)
+        val relatedTableAlias =
+          parser.getRelatedTablesAlias(readOnlyLayouts)
+        new SegmentFlatTableDesc(config, dataSegment, spanningTree, relatedTableAlias)
+      } else {
+        new SegmentFlatTableDesc(config, dataSegment, spanningTree)
+      }
+      buildParam.setFlatTableDesc(flatTableDesc)
     }
   }
 }
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/FlatTableAndDictBase.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/FlatTableAndDictBase.scala
index c44fe1f9e0..38a191d461 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/FlatTableAndDictBase.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/FlatTableAndDictBase.scala
@@ -31,9 +31,11 @@ import org.apache.kylin.engine.spark.job.NSparkCubingUtil.convertFromDot
 import org.apache.kylin.engine.spark.job.stage.{BuildParam, StageExec}
 import org.apache.kylin.engine.spark.job.{FiltersUtil, SegmentJob, TableMetaManager}
 import org.apache.kylin.engine.spark.model.SegmentFlatTableDesc
+import org.apache.kylin.engine.spark.model.planner.{CuboIdToLayoutUtils, FlatTableToCostUtils}
 import org.apache.kylin.engine.spark.utils.LogEx
 import org.apache.kylin.engine.spark.utils.SparkDataSource._
 import org.apache.kylin.metadata.cube.model.NDataSegment
+import org.apache.kylin.metadata.cube.planner.CostBasePlannerUtils
 import org.apache.kylin.metadata.model._
 import org.apache.kylin.query.util.QueryUtil
 import org.apache.spark.sql.KapFunctions.dict_encode_v3
@@ -43,6 +45,7 @@ import org.apache.spark.sql.types.StructField
 import org.apache.spark.sql.util.SparderTypeUtil
 import org.apache.spark.utils.ProxyThreadUtils
 
+import java.math.BigInteger
 import java.util.concurrent.{CountDownLatch, TimeUnit}
 import java.util.{Locale, Objects, Timer, TimerTask}
 import scala.collection.JavaConverters._
@@ -64,8 +67,11 @@ abstract class FlatTableAndDictBase(private val jobContext: SegmentJob,
 
   import FlatTableAndDictBase._
 
-  protected lazy val spanningTree = buildParam.getSpanningTree
-  protected lazy val tableDesc = buildParam.getFlatTableDesc
+  // These parameters can be changed when running the cube planner, should use the
+  // `def` to get the latest data
+  protected def spanningTree = buildParam.getSpanningTree
+
+  protected def tableDesc = buildParam.getFlatTableDesc
 
   protected lazy final val indexPlan = tableDesc.getIndexPlan
   protected lazy final val segmentRange = tableDesc.getSegmentRange
@@ -140,6 +146,41 @@ abstract class FlatTableAndDictBase(private val jobContext: SegmentJob,
     changeSchemeToColumnId(flatTableDS, tableDesc)
   }
 
+  protected def generateCostTable(): (java.util.Map[BigInteger, java.lang.Long], Long) = {
+    val rowkeyCount = indexPlan.getRuleBasedIndex.countOfIncludeDimension()
+    val stepDesc = s"Segment $segmentId generate the cost for the planner from the flat table, " +
+      s"rowkey count is $rowkeyCount"
+    logInfo(stepDesc)
+    sparkSession.sparkContext.setJobDescription(stepDesc)
+    // get the cost from the flat table
+    val javaRddFlatTable = FLAT_TABLE.javaRDD
+    // log dimension and table desc
+    logInfo(s"Segment $segmentId calculate the cost, the dimension in rule index is: " +
+      s"${indexPlan.getRuleBasedIndex.getDimensions}, " +
+      s"the column in flat table is: ${tableDesc.getColumnIds}")
+    val cuboIdsCost = FlatTableToCostUtils.generateCost(javaRddFlatTable, config, indexPlan.getRuleBasedIndex, tableDesc)
+    // get the count for the flat table
+    val sourceCount = FLAT_TABLE.count()
+    logInfo(s"The total source count is $sourceCount")
+    sparkSession.sparkContext.setJobDescription(null)
+    val cuboIdToRowCount = FlatTableToCostUtils.getCuboidRowCountMapFromSampling(cuboIdsCost)
+    (cuboIdToRowCount, sourceCount)
+  }
+
+  protected def getRecommendedLayoutAndUpdateMetadata(cuboIdToRowCount: java.util.Map[BigInteger, java.lang.Long],
+                                                      sourceCount: Long): Unit = {
+    logDebug(s"Segment $segmentId get the row count cost $cuboIdToRowCount")
+    val cuboIdToSize = FlatTableToCostUtils.
+      getCuboidSizeMapFromSampling(cuboIdToRowCount, sourceCount, indexPlan.getRuleBasedIndex, config, tableDesc)
+    logDebug(s"Segment $segmentId get the size cost $cuboIdToSize")
+    val cuboids = CostBasePlannerUtils.
+      getRecommendCuboidList(indexPlan.getRuleBasedIndex, config, dataModel.getAlias, cuboIdToRowCount, cuboIdToSize)
+    logDebug(s"Segment $segmentId get the recommended cuboid ${cuboids.keySet()}")
+    val allRecommendedAggColOrders = CuboIdToLayoutUtils.convertCuboIdsToAggIndexColOrders(cuboids, indexPlan.getRuleBasedIndex)
+    logInfo(s"Segment $segmentId get ${allRecommendedAggColOrders.size()} recommended layouts with duplicate layouts removed.")
+    jobContext.setRecommendAggColOrders(allRecommendedAggColOrders)
+  }
+
   protected def generateFlatTable(): Dataset[Row] = {
     val recoveredDS = tryRecoverFTDS()
     if (recoveredDS.nonEmpty) {
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/MaterializedFactTableView.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/MaterializedFactTableView.scala
index 584538f47d..f467058559 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/MaterializedFactTableView.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/MaterializedFactTableView.scala
@@ -38,7 +38,7 @@ class MaterializedFactTableView(jobContext: SegmentJob, dataSegment: NDataSegmen
     val flatTableDesc: SegmentFlatTableDesc = if (jobContext.isPartialBuild) {
       val parser = new IndexDependencyParser(dataModel)
       val relatedTableAlias =
-        parser.getRelatedTablesAlias(jobContext.getReadOnlyLayouts)
+        parser.getRelatedTablesAlias(readOnlyLayouts)
       new SegmentFlatTableDesc(config, dataSegment, spanningTree, relatedTableAlias)
     } else {
       new SegmentFlatTableDesc(config, dataSegment, spanningTree)
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/partition/PartitionMaterializedFactTableView.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/partition/PartitionCostBasedPlanner.scala
similarity index 57%
copy from src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/partition/PartitionMaterializedFactTableView.scala
copy to src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/partition/PartitionCostBasedPlanner.scala
index 75063e3596..3aac410594 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/partition/PartitionMaterializedFactTableView.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/partition/PartitionCostBasedPlanner.scala
@@ -18,6 +18,8 @@
 
 package org.apache.kylin.engine.spark.job.stage.build.partition
 
+import java.util.stream.Collectors
+
 import org.apache.kylin.engine.spark.job.SegmentJob
 import org.apache.kylin.engine.spark.job.stage.BuildParam
 import org.apache.kylin.engine.spark.model.PartitionFlatTableDesc
@@ -26,39 +28,31 @@ import org.apache.kylin.metadata.cube.cuboid.PartitionSpanningTree
 import org.apache.kylin.metadata.cube.cuboid.PartitionSpanningTree.PartitionTreeBuilder
 import org.apache.kylin.metadata.cube.model.NDataSegment
 import org.apache.kylin.metadata.job.JobBucket
-import org.apache.spark.sql.{Dataset, Row}
-
-import java.util.stream.Collectors
 
-class PartitionMaterializedFactTableView(jobContext: SegmentJob, dataSegment: NDataSegment, buildParam: BuildParam)
+class PartitionCostBasedPlanner(jobContext: SegmentJob, dataSegment: NDataSegment, buildParam: BuildParam)
   extends PartitionFlatTableAndDictBase(jobContext, dataSegment, buildParam) {
-
-
   override def execute(): Unit = {
-    logInfo(s"Build SEGMENT $segmentId")
-    val spanTree = new PartitionSpanningTree(config,
-      new PartitionTreeBuilder(dataSegment, readOnlyLayouts, jobId, partitions,
-        jobContext.getReadOnlyBuckets.stream.filter(_.getSegmentId.equals(segmentId)).collect(Collectors.toSet[JobBucket])))
-    buildParam.setPartitionSpanningTree(spanTree)
-
-    val tableDesc = if (jobContext.isPartialBuild) {
-      val parser = new IndexDependencyParser(dataModel)
-      val relatedTableAlias =
-        parser.getRelatedTablesAlias(jobContext.getReadOnlyLayouts)
-      new PartitionFlatTableDesc(config, dataSegment, spanTree, relatedTableAlias, jobId, partitions)
-    } else {
-      new PartitionFlatTableDesc(config, dataSegment, spanTree, jobId, partitions)
-    }
-    buildParam.setTableDesc(tableDesc)
-
-    val factTableDS: Dataset[Row] = newFactTableDS()
-    buildParam.setFactTableDS(factTableDS)
-
-    val fastFactTableDS: Dataset[Row] = newFastFactTableDS()
-    buildParam.setFastFactTableDS(fastFactTableDS)
-
-    if (buildParam.isSkipMaterializedFactTableView) {
-      onStageSkipped()
+    val (cost, sourceCount) = generateCostTable()
+    getRecommendedLayoutAndUpdateMetadata(cost, sourceCount)
+    //    jobContext.addMockIndex()
+    val result = jobContext.updateIndexPlanIfNeed()
+    if (result) {
+      // update span tree and table desc with the new build job layouts
+      val spanTree = new PartitionSpanningTree(config,
+        new PartitionTreeBuilder(dataSegment, readOnlyLayouts, jobId, partitions,
+          jobContext.getReadOnlyBuckets.stream.filter(_.getSegmentId.equals(segmentId)).collect(Collectors.toSet[JobBucket])))
+      buildParam.setPartitionSpanningTree(spanTree)
+
+      // update table desc with new span tree
+      val tableDesc = if (jobContext.isPartialBuild) {
+        val parser = new IndexDependencyParser(dataModel)
+        val relatedTableAlias =
+          parser.getRelatedTablesAlias(readOnlyLayouts)
+        new PartitionFlatTableDesc(config, dataSegment, spanTree, relatedTableAlias, jobId, partitions)
+      } else {
+        new PartitionFlatTableDesc(config, dataSegment, spanTree, jobId, partitions)
+      }
+      buildParam.setTableDesc(tableDesc)
     }
   }
 }
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/partition/PartitionMaterializedFactTableView.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/partition/PartitionMaterializedFactTableView.scala
index 75063e3596..e8f2fca99f 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/partition/PartitionMaterializedFactTableView.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/partition/PartitionMaterializedFactTableView.scala
@@ -44,7 +44,7 @@ class PartitionMaterializedFactTableView(jobContext: SegmentJob, dataSegment: ND
     val tableDesc = if (jobContext.isPartialBuild) {
       val parser = new IndexDependencyParser(dataModel)
       val relatedTableAlias =
-        parser.getRelatedTablesAlias(jobContext.getReadOnlyLayouts)
+        parser.getRelatedTablesAlias(readOnlyLayouts)
       new PartitionFlatTableDesc(config, dataSegment, spanTree, relatedTableAlias, jobId, partitions)
     } else {
       new PartitionFlatTableDesc(config, dataSegment, spanTree, jobId, partitions)
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/model/planner/CuboIdToLayoutUtils.java b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/model/planner/CuboIdToLayoutUtils.java
new file mode 100644
index 0000000000..ec6cb2555d
--- /dev/null
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/model/planner/CuboIdToLayoutUtils.java
@@ -0,0 +1,140 @@
+/*
+ * 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.kylin.engine.spark.model.planner;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kylin.metadata.cube.cuboid.NAggregationGroup;
+import org.apache.kylin.metadata.cube.model.RuleBasedIndex;
+
+import com.google.common.collect.Lists;
+
+public class CuboIdToLayoutUtils {
+    /**
+     * convert the cuboids to column orders
+     * @param cuboids
+     * @param ruleBasedIndex
+     * @return
+     */
+    public static Set<List<Integer>> convertCuboIdsToAggIndexColOrders(Map<BigInteger, Long> cuboids,
+            RuleBasedIndex ruleBasedIndex) {
+        // convert the cuboid to each agg group
+        Set<List<Integer>> result = new HashSet<>();
+        List<NAggregationGroup> aggregationGroups = ruleBasedIndex != null ? ruleBasedIndex.getAggregationGroups()
+                : Lists.newArrayList();
+        for (NAggregationGroup group : aggregationGroups) {
+            // dimension order in this agg group
+            List<Integer> dimensionOrder = Lists.newArrayList(group.getIncludes());
+            // measure order in this agg group
+            List<Integer> measuresIds = Lists.newArrayList(group.getMeasures());
+            Set<List<Integer>> colOrders = convertCuboIdsToColOrders(cuboids, ruleBasedIndex.countOfIncludeDimension(),
+                    measuresIds, ruleBasedIndex.getRowKeyIdToColumnId(), dimensionOrder);
+            result.addAll(colOrders);
+        }
+
+        // base agg layout for each agg group
+        for (NAggregationGroup group : aggregationGroups) {
+            List<Integer> colOrders = Lists.newArrayList();
+            // all dimension in the agg
+            colOrders.addAll(Lists.newArrayList(group.getIncludes()));
+            // all measure in this agg
+            colOrders.addAll(Lists.newArrayList(group.getMeasures()));
+            result.add(colOrders);
+        }
+        return result;
+    }
+
+    /**
+     * convert the cuboids to column order set which contain dimension ids and measure ids
+     * @param cuboids
+     * @param dimensionCount
+     * @param measuresIds
+     * @return
+     */
+    protected static Set<List<Integer>> convertCuboIdsToColOrders(Map<BigInteger, Long> cuboids, int dimensionCount,
+            List<Integer> measuresIds, Map<Integer, Integer> rowkeyIdToColumnId, List<Integer> sortOfDims) {
+        Set<List<Integer>> result = new HashSet<>();
+        for (BigInteger cuboid : cuboids.keySet()) {
+            // 1. get the dimension with order
+            // convert the cuboid to the order of dimension which is sorted by the order of `sortOfDims`
+            List<Integer> colOrder = converCuboidToDimensionColOrder(cuboid, dimensionCount, rowkeyIdToColumnId,
+                    sortOfDims);
+            if (colOrder.isEmpty()) {
+                // If the cuboid can't match the `sortOfDims`, and will not get the column order for this layout
+                continue;
+            }
+            // 2. get the measure with order
+            // In the current cube planner, each layout should contains all of the measures in the model
+            colOrder.addAll(measuresIds);
+            result.add(colOrder);
+        }
+        return result;
+    }
+
+    /**
+     * convert the cuboid to the column order which just contains the dimension ids
+     *
+     * @param cuboid
+     * @param maxDimensionCount
+     * @return
+     */
+    public static List<Integer> converCuboidToDimensionColOrder(BigInteger cuboid, int maxDimensionCount,
+            Map<Integer, Integer> rowkeyIdToColumnId, List<Integer> sortOfDims) {
+        // If cuboid is 00000000,00000000,00000000,10001001, and the max dimension count is 12
+        // It will be converted to [4,8,11]
+        List<Integer> colOrder = new ArrayList<>();
+        for (int rowkeyId = 0; rowkeyId < maxDimensionCount; rowkeyId++) {
+            int rightShift = maxDimensionCount - rowkeyId - 1;
+            boolean exist = cuboid.testBit(rightShift);
+            if (exist) {
+                if (!rowkeyIdToColumnId.containsKey(rowkeyId)) {
+                    throw new RuntimeException("Can't find the column id from the rowkey id");
+                }
+                int columnId = rowkeyIdToColumnId.get(rowkeyId);
+                if (sortOfDims.indexOf(columnId) < 0) {
+                    // can't find the dimension in the sorted dimension
+                    // this cuboid is not recommended from the rule base index
+                    // just skip and ignore it
+                    colOrder.clear();
+                    break;
+                }
+                colOrder.add(columnId);
+            }
+        }
+        if (!colOrder.isEmpty()) {
+            // sort the dimension by the sortOfDims
+            Collections.sort(colOrder, new Comparator<Integer>() {
+                @Override
+                public int compare(Integer o1, Integer o2) {
+                    int index1 = sortOfDims.indexOf(o1);
+                    int index2 = sortOfDims.indexOf(o2);
+                    return index1 - index2;
+                }
+            });
+        }
+        return colOrder;
+    }
+}
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/model/planner/FlatTableToCostUtils.java b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/model/planner/FlatTableToCostUtils.java
new file mode 100644
index 0000000000..7bb5fa3528
--- /dev/null
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/model/planner/FlatTableToCostUtils.java
@@ -0,0 +1,518 @@
+/*
+ * 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.kylin.engine.spark.model.planner;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.engine.spark.model.SegmentFlatTableDesc;
+import org.apache.kylin.measure.BufferedMeasureCodec;
+import org.apache.kylin.measure.hllc.HLLCounter;
+import org.apache.kylin.measure.hllc.RegisterType;
+import org.apache.kylin.metadata.cube.model.IndexPlan;
+import org.apache.kylin.metadata.cube.model.LayoutEntity;
+import org.apache.kylin.metadata.cube.model.RuleBasedIndex;
+import org.apache.kylin.metadata.cube.planner.CostBasePlannerUtils;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.NDataModel;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.spark.Partitioner;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFlatMapFunction;
+import org.apache.spark.sql.Row;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+
+import lombok.extern.slf4j.Slf4j;
+import scala.Tuple2;
+
+@Slf4j
+public class FlatTableToCostUtils {
+
+    private static LayoutEntity createMockRuleBaseLayout(RuleBasedIndex ruleBasedIndex) {
+        // create a mock base layout which include all dimensions in the rule index without order
+        LayoutEntity baseLayouts = new LayoutEntity();
+        baseLayouts.setColOrder(ruleBasedIndex.getDimensions());
+        return baseLayouts;
+    }
+
+    public static Map<BigInteger, HLLCounter> generateCost(JavaRDD<Row> input, KylinConfig kylinConfig,
+            RuleBasedIndex ruleBasedIndex, SegmentFlatTableDesc flatTableDesc) throws IOException {
+        // step1: convert each cell to string data type, and get RDD[String[]]
+        JavaRDD<String[]> flatTableRDD = input.map(new Function<Row, String[]>() {
+            @Override
+            public String[] call(Row row) throws Exception {
+                String[] result = new String[row.length()];
+                for (int i = 0; i < row.length(); i++) {
+                    final Object o = row.get(i);
+                    if (o != null) {
+                        result[i] = o.toString();
+                    } else {
+                        result[i] = null;
+                    }
+                }
+                return result;
+            }
+        });
+        // step2: calculate the cost for each partition, and get the new RDD.
+        // The key is cuboid, and the value is the data encoded from the hll for each partition.
+        int rowKeyCount = ruleBasedIndex.countOfIncludeDimension();
+        // layouts from the rule index(agg group)
+        Set<LayoutEntity> inputLayouts = ruleBasedIndex.genCuboidLayouts();
+        // add the mock layout which include all of the dimensions in the rule index
+        inputLayouts.add(createMockRuleBaseLayout(ruleBasedIndex));
+        BigInteger[] inputCuboids = getCuboIdsFromLayouts(Lists.newArrayList(inputLayouts), rowKeyCount,
+                ruleBasedIndex.getColumnIdToRowKeyId());
+        // rowkey id ->  column index in the flat table of the flat dataset.
+        int[] rowkeyColumnIndexes = getRowkeyColumnIndexes(ruleBasedIndex, flatTableDesc);
+
+        int hllPrecision = kylinConfig.getStatsHLLPrecision();
+        log.info("The row key count is {}, and the index/column map is {}", rowKeyCount,
+                Lists.newArrayList(rowkeyColumnIndexes));
+        JavaPairRDD<BigInteger, byte[]> costRddByPartition = flatTableRDD.mapPartitionsToPair(
+                new FlatOutputFunction(hllPrecision, rowKeyCount, inputCuboids, rowkeyColumnIndexes));
+
+        // step3: reduce by cuboid, and merge hll data
+        // The key is the cuboid, the value is data encoded from the hll
+        int partitionNum = getCuboidHLLCounterReducerNum(inputCuboids.length, kylinConfig);
+        log.info("Get the partition count for the HLL reducer: {}", partitionNum);
+        JavaPairRDD<BigInteger, byte[]> costRDD = costRddByPartition.reduceByKey(new Partitioner() {
+            private int num = partitionNum;
+            private BigInteger bigIntegerMod = BigInteger.valueOf(num);
+
+            @Override
+            public int numPartitions() {
+                return num;
+            }
+
+            @Override
+            public int getPartition(Object key) {
+                // key is the biginteger
+                BigInteger value = (BigInteger) key;
+                return value.mod(bigIntegerMod).intValue();
+            }
+        }, new Function2<byte[], byte[], byte[]>() {
+            private int precision = hllPrecision;
+
+            @Override
+            public byte[] call(byte[] array1, byte[] array2) throws Exception {
+                // hll1
+                HLLCounter hll1 = new HLLCounter(precision);
+                ByteBuffer buffer1 = ByteBuffer.wrap(array1, 0, array1.length);
+                hll1.readRegisters(buffer1);
+                // hll2
+                HLLCounter hll2 = new HLLCounter(precision);
+                ByteBuffer buffer2 = ByteBuffer.wrap(array2, 0, array2.length);
+                hll2.readRegisters(buffer2);
+
+                // merge two hll
+                hll1.merge(hll2);
+                ByteBuffer hllBuf = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);
+                hll1.writeRegisters(hllBuf);
+                byte[] value = new byte[hllBuf.position()];
+                System.arraycopy(hllBuf.array(), 0, value, 0, hllBuf.position());
+                return value;
+            }
+        });
+        // step4: collect the final result, and convert value(text) to hll
+        // The key is the cuboid, and the value is the estimated statistics
+        Map<BigInteger, HLLCounter> resultCost = Maps.newHashMap();
+        for (Tuple2<BigInteger, byte[]> pair : costRDD.collect()) {
+            HLLCounter hll = new HLLCounter(kylinConfig.getStatsHLLPrecision());
+            // value
+            ByteArray byteArray = new ByteArray(pair._2);
+            hll.readRegisters(byteArray.asBuffer());
+            // put key and value
+            resultCost.put(pair._1, hll);
+        }
+        // log data
+        if (log.isDebugEnabled()) {
+            logMapperAndCuboidStatistics(resultCost, 100);
+        }
+        return resultCost;
+    }
+
+    /**
+     * @return reducer number for calculating hll
+     */
+    private static int getCuboidHLLCounterReducerNum(int nCuboids, KylinConfig kylinConfig) {
+        int shardBase = (nCuboids - 1) / kylinConfig.getJobPerReducerHLLCuboidNumber() + 1;
+        int hllMaxReducerNumber = kylinConfig.getJobHLLMaxReducerNumber();
+        if (shardBase > hllMaxReducerNumber) {
+            shardBase = hllMaxReducerNumber;
+        }
+        return Math.max(shardBase, 1);
+    }
+
+    private static BigInteger[] getCuboIdsFromLayouts(List<LayoutEntity> allLayouts, int dimensionCount,
+            Map<Integer, Integer> columnIdToRowkeyId) {
+        Set<BigInteger> set = new HashSet<>();
+        for (LayoutEntity layoutEntity : allLayouts) {
+            BigInteger cuboId = CostBasePlannerUtils.convertDimensionsToCuboId(layoutEntity.getDimsIds(),
+                    dimensionCount, columnIdToRowkeyId);
+            set.add(cuboId);
+        }
+        return set.toArray(new BigInteger[set.size()]);
+    }
+
+    private static int[] getRowkeyColumnIndexes(RuleBasedIndex ruleBasedIndex, SegmentFlatTableDesc flatTableDesc) {
+        int rowKeyCount = ruleBasedIndex.countOfIncludeDimension();
+        List<Integer> columnIds = flatTableDesc.getColumnIds();
+        int[] rowkeyColumnIndexes = new int[rowKeyCount];
+        Map<Integer, Integer> rowkeyIdToColumnId = ruleBasedIndex.getRowKeyIdToColumnId();
+        for (int rowkeyId = 0; rowkeyId < rowKeyCount; rowkeyId++) {
+            if (!rowkeyIdToColumnId.containsKey(rowkeyId)) {
+                throw new RuntimeException("Can't find the column id from the rowkey id");
+            }
+            int columnId = rowkeyIdToColumnId.get(rowkeyId);
+            int index = columnIds.indexOf(columnId);
+            if (index >= 0) {
+                // find the i-th dimension in the index-th column in the flat table or flat data set
+                rowkeyColumnIndexes[rowkeyId] = index;
+            } else {
+                // not find the column id
+                throw new RuntimeException(
+                        String.format("Can't find the column id %d, column ids %s", columnId, columnIds.toString()));
+            }
+        }
+        return rowkeyColumnIndexes;
+    }
+
+    private static class FlatOutputFunction implements PairFlatMapFunction<Iterator<String[]>, BigInteger, byte[]> {
+        private transient volatile boolean initialized = false;
+        private transient CuboidStatCalculator cuboidStatCalculator;
+        private final int samplingPercent = 100;
+        private final int hllPrecision;
+        private final int rowKeyCount;
+        private final BigInteger[] cuboidIds;
+        private final int[] rowkeyColumnIndexes;
+
+        public FlatOutputFunction(int hllPrecision, int rowKeyCount, BigInteger[] cuboidIds,
+                int[] rowkeyColumnIndexes) {
+            this.hllPrecision = hllPrecision;
+            this.rowKeyCount = rowKeyCount;
+            this.cuboidIds = cuboidIds;
+            this.rowkeyColumnIndexes = rowkeyColumnIndexes;
+        }
+
+        private Integer[][] getCuboidBitSet(BigInteger[] cuboidIds, int nRowKey) {
+            Integer[][] allCuboidsBitSet = new Integer[cuboidIds.length][];
+            for (int j = 0; j < cuboidIds.length; j++) {
+                BigInteger cuboidId = cuboidIds[j];
+
+                allCuboidsBitSet[j] = new Integer[cuboidId.bitCount()];
+                int position = 0;
+                for (int i = 0; i < nRowKey; i++) {
+                    BigInteger bigMask = BigInteger.ZERO.setBit(nRowKey - 1 - i);
+                    if ((bigMask.and(cuboidId).compareTo(BigInteger.ZERO)) > 0) {
+                        // bigMask & cuboid > 0
+                        allCuboidsBitSet[j][position] = i;
+                        position++;
+                    }
+                }
+            }
+            return allCuboidsBitSet;
+        }
+
+        private HLLCounter[] getInitCuboidsHLL(int cuboidSize, int hllPrecision) {
+            HLLCounter[] cuboidsHLL = new HLLCounter[cuboidSize];
+            for (int i = 0; i < cuboidSize; i++) {
+                cuboidsHLL[i] = new HLLCounter(hllPrecision, RegisterType.DENSE);
+            }
+            return cuboidsHLL;
+        }
+
+        private void init() {
+            Integer[][] cuboidsBitSet = getCuboidBitSet(cuboidIds, rowKeyCount);
+            HLLCounter[] cuboidsHLL = getInitCuboidsHLL(cuboidIds.length, hllPrecision);
+            cuboidStatCalculator = new CuboidStatCalculator(rowkeyColumnIndexes, cuboidIds, cuboidsBitSet, true,
+                    cuboidsHLL);
+            initialized = true;
+        }
+
+        @Override
+        public Iterator<Tuple2<BigInteger, byte[]>> call(Iterator<String[]> iterator) throws Exception {
+            if (initialized == false) {
+                // just sync this object
+                synchronized (this) {
+                    if (initialized == false) {
+                        init();
+                    }
+                }
+            }
+
+            // One tuple is a cost pair, the left is the cuboid and the right is the cost
+            int rowCount = 0;
+            while (iterator.hasNext()) {
+                String[] row = iterator.next();
+                if (rowCount % 100 < samplingPercent) {
+                    cuboidStatCalculator.putRow(row);
+                }
+                rowCount++;
+            }
+
+            List<Tuple2<BigInteger, byte[]>> result = Lists.newArrayList();
+            ByteBuffer hllBuf = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);
+            BigInteger[] cuboidIds = cuboidStatCalculator.getCuboidIds();
+            HLLCounter[] cuboidsHLL = cuboidStatCalculator.getHLLCounters();
+            HLLCounter hll;
+            for (int i = 0; i < cuboidIds.length; i++) {
+                // key
+                BigInteger outputKey = cuboidIds[i];
+                // value
+                hll = cuboidsHLL[i];
+                hllBuf.clear();
+                hll.writeRegisters(hllBuf);
+                byte[] value = new byte[hllBuf.position()];
+                System.arraycopy(hllBuf.array(), 0, value, 0, hllBuf.position());
+                result.add(new Tuple2<BigInteger, byte[]>(outputKey, value));
+            }
+            return result.iterator();
+        }
+    }
+
+    static class CuboidStatCalculator {
+        private final int nRowKey;
+        private final int[] rowkeyColIndex;
+        private final BigInteger[] cuboidIds;
+        private final Integer[][] cuboidsBitSet;
+        private HLLCounter[] cuboidsHLL;
+
+        //about details of the new algorithm, please see KYLIN-2518
+        private final boolean isNewAlgorithm;
+        private final HashFunction hf;
+        private long[] rowHashCodesLong;
+
+        public CuboidStatCalculator(int[] rowkeyColIndex, BigInteger[] cuboidIds, Integer[][] cuboidsBitSet,
+                boolean isUsePutRowKeyToHllNewAlgorithm, HLLCounter[] cuboidsHLL) {
+            this.nRowKey = rowkeyColIndex.length;
+            this.rowkeyColIndex = rowkeyColIndex;
+            this.cuboidIds = cuboidIds;
+            this.cuboidsBitSet = cuboidsBitSet;
+            this.isNewAlgorithm = isUsePutRowKeyToHllNewAlgorithm;
+            if (!isNewAlgorithm) {
+                this.hf = Hashing.murmur3_32();
+            } else {
+                rowHashCodesLong = new long[nRowKey];
+                this.hf = Hashing.murmur3_128();
+            }
+            this.cuboidsHLL = cuboidsHLL;
+        }
+
+        public void putRow(final String[] row) {
+            String[] copyRow = Arrays.copyOf(row, row.length);
+
+            if (isNewAlgorithm) {
+                putRowKeyToHLLNew(copyRow);
+            } else {
+                putRowKeyToHLLOld(copyRow);
+            }
+        }
+
+        private void putRowKeyToHLLOld(String[] row) {
+            //generate hash for each row key column
+            byte[][] rowHashCodes = new byte[nRowKey][];
+            for (int i = 0; i < nRowKey; i++) {
+                Hasher hc = hf.newHasher();
+                String colValue = row[rowkeyColIndex[i]];
+                if (colValue != null) {
+                    rowHashCodes[i] = hc.putString(colValue).hash().asBytes();
+                } else {
+                    rowHashCodes[i] = hc.putInt(0).hash().asBytes();
+                }
+            }
+
+            // user the row key column hash to get a consolidated hash for each cuboid
+            for (int i = 0, n = cuboidsBitSet.length; i < n; i++) {
+                Hasher hc = hf.newHasher();
+                for (int position = 0; position < cuboidsBitSet[i].length; position++) {
+                    hc.putBytes(rowHashCodes[cuboidsBitSet[i][position]]);
+                }
+
+                cuboidsHLL[i].add(hc.hash().asBytes());
+            }
+        }
+
+        private void putRowKeyToHLLNew(String[] row) {
+            //generate hash for each row key column
+            for (int i = 0; i < nRowKey; i++) {
+                Hasher hc = hf.newHasher();
+                String colValue = row[rowkeyColIndex[i]];
+                if (colValue == null)
+                    colValue = "0";
+                byte[] bytes = hc.putString(colValue).hash().asBytes();
+                //add column ordinal to the hash value to distinguish between (a,b) and (b,a)
+                rowHashCodesLong[i] = (Bytes.toLong(bytes) + i);
+            }
+
+            // user the row key column hash to get a consolidated hash for each cuboid
+            for (int i = 0, n = cuboidsBitSet.length; i < n; i++) {
+                long value = 0;
+                for (int position = 0; position < cuboidsBitSet[i].length; position++) {
+                    value += rowHashCodesLong[cuboidsBitSet[i][position]];
+                }
+                cuboidsHLL[i].addHashDirectly(value);
+            }
+        }
+
+        public HLLCounter[] getHLLCounters() {
+            return cuboidsHLL;
+        }
+
+        public BigInteger[] getCuboidIds() {
+            return cuboidIds;
+        }
+    }
+
+    private static void logMapperAndCuboidStatistics(Map<BigInteger, HLLCounter> cuboidHLLMap, int samplingPercentage) {
+        log.debug("Total cuboid number: \t" + cuboidHLLMap.size());
+        log.debug("Sampling percentage: \t" + samplingPercentage);
+        log.debug("The following statistics are collected based on sampling data.");
+
+        List<BigInteger> allCuboids = Lists.newArrayList(cuboidHLLMap.keySet());
+        Collections.sort(allCuboids);
+        for (BigInteger i : allCuboids) {
+            log.debug("Cuboid " + i + " row count is: \t " + cuboidHLLMap.get(i).getCountEstimate());
+        }
+    }
+
+    public static Map<BigInteger, Long> getCuboidRowCountMapFromSampling(Map<BigInteger, HLLCounter> hllcMap) {
+        Map<BigInteger, Long> cuboidRowCountMap = Maps.newHashMap();
+        for (Map.Entry<BigInteger, HLLCounter> entry : hllcMap.entrySet()) {
+            // No need to adjust according sampling percentage. Assumption is that data set is far
+            // more than cardinality. Even a percentage of the data should already see all cardinalities.
+            cuboidRowCountMap.put(entry.getKey(), entry.getValue().getCountEstimate());
+        }
+        return cuboidRowCountMap;
+    }
+
+    private static Map<BigInteger, Double> getCuboidSizeMapFromSamplingByCount(Map<BigInteger, Long> rowCountMap,
+            long sourceCount, RuleBasedIndex ruleBasedIndex, KylinConfig kylinConfig,
+            SegmentFlatTableDesc segmentFlatTableDesc) {
+        // use the row count to replace the size
+        Map<BigInteger, Double> cuboidSizeMap = Maps.newHashMap();
+        for (Map.Entry<BigInteger, Long> entry : rowCountMap.entrySet()) {
+            double value = entry.getValue();
+            cuboidSizeMap.put(entry.getKey(), value);
+        }
+        return cuboidSizeMap;
+    }
+
+    public static Map<BigInteger, Double> getCuboidSizeMapFromSampling(Map<BigInteger, Long> rowCountMap,
+            long sourceCount, RuleBasedIndex ruleBasedIndex, KylinConfig kylinConfig,
+            SegmentFlatTableDesc segmentFlatTableDesc) {
+        // replace the size with the row count
+        return getCuboidSizeMapFromSamplingByCount(rowCountMap, sourceCount, ruleBasedIndex, kylinConfig,
+                segmentFlatTableDesc);
+    }
+
+    private static List<Integer> getRowkeyColumnSize(IndexPlan indexPlan, SegmentFlatTableDesc flatTableDesc) {
+        int rowKeyCount = indexPlan.getRuleBasedIndex().countOfIncludeDimension();
+        List<Integer> columnIds = flatTableDesc.getColumnIds();
+        List<TblColRef> tblColRefs = flatTableDesc.getColumns();
+        List<Integer> rowkeyColumnSize = Lists.newArrayList();
+
+        for (int i = 0; i < rowKeyCount; i++) {
+            int index = columnIds.indexOf(i);
+            if (index >= 0) {
+                // find the i-th dimension in the index-th column in the flat table or flat data set.
+                TblColRef tblColRef = tblColRefs.get(index);
+                // get DimensionEncoding for this table column ref.
+                // Noe we just use the row count as the weight
+                int length = 0;
+                rowkeyColumnSize.add(length);
+            } else {
+                // not find the column id
+                throw new RuntimeException(
+                        String.format("Can't find the column id %d, column ids %s", i, columnIds.toString()));
+            }
+        }
+        // the index means column id for the dimension, the value is the estimated size for this dimension
+        return rowkeyColumnSize;
+    }
+
+    private static double estimateCuboidStorageSize(Set<NDataModel.Measure> measureDescs, long cuboidId, long rowCount,
+            long baseCuboidId, long baseCuboidCount, List<Integer> rowKeyColumnLength, long sourceRowCount,
+            KylinConfig kylinConfig) {
+        // row key header
+        int rowkeyLength = 8; // 8 or 10
+        long mask = Long.highestOneBit(baseCuboidId);
+        // actual: the parentCuboidIdActualLength is rowkey count
+        long parentCuboidIdActualLength = (long) Long.SIZE - Long.numberOfLeadingZeros(baseCuboidId);
+        // dimension length
+        for (int i = 0; i < parentCuboidIdActualLength; i++) {
+            if ((mask & cuboidId) > 0) {
+                rowkeyLength += rowKeyColumnLength.get(i); //colIO.getColumnLength(columnList.get(i));
+            }
+            mask = mask >> 1;
+        }
+        // measure size
+        int normalSpace = rowkeyLength;
+        int countDistinctSpace = 0;
+        double percentileSpace = 0;
+        int topNSpace = 0;
+        for (MeasureDesc measureDesc : measureDescs) {
+            if (rowCount == 0)
+                break;
+            DataType returnType = measureDesc.getFunction().getReturnDataType();
+            if (measureDesc.getFunction().getExpression().equals(FunctionDesc.FUNC_COUNT_DISTINCT)) {
+                long estimateDistinctCount = sourceRowCount / rowCount;
+                estimateDistinctCount = estimateDistinctCount == 0 ? 1L : estimateDistinctCount;
+                countDistinctSpace += returnType.getStorageBytesEstimate(estimateDistinctCount);
+            } else if (measureDesc.getFunction().getExpression().equals(FunctionDesc.FUNC_PERCENTILE)) {
+                percentileSpace += returnType.getStorageBytesEstimate(baseCuboidCount * 1.0 / rowCount);
+            } else if (measureDesc.getFunction().getExpression().equals(FunctionDesc.FUNC_TOP_N)) {
+                long estimateTopNCount = sourceRowCount / rowCount;
+                estimateTopNCount = estimateTopNCount == 0 ? 1L : estimateTopNCount;
+                topNSpace += returnType.getStorageBytesEstimate(estimateTopNCount);
+            } else {
+                normalSpace += returnType.getStorageBytesEstimate();
+            }
+        }
+
+        double cuboidSizeRatio = kylinConfig.getJobCuboidSizeRatio();
+        double cuboidSizeMemHungryRatio = kylinConfig.getJobCuboidSizeCountDistinctRatio();
+        double cuboidSizeTopNRatio = kylinConfig.getJobCuboidSizeTopNRatio();
+
+        double ret = (1.0 * normalSpace * rowCount * cuboidSizeRatio
+                + 1.0 * countDistinctSpace * rowCount * cuboidSizeMemHungryRatio + 1.0 * percentileSpace * rowCount
+                + 1.0 * topNSpace * rowCount * cuboidSizeTopNRatio) / (1024L * 1024L);
+        return ret;
+    }
+}
diff --git a/src/spark-project/engine-spark/src/test/java/org/apache/kylin/engine/spark/model/planner/CuboIdToLayoutUtilsTest.java b/src/spark-project/engine-spark/src/test/java/org/apache/kylin/engine/spark/model/planner/CuboIdToLayoutUtilsTest.java
new file mode 100644
index 0000000000..f072e601e3
--- /dev/null
+++ b/src/spark-project/engine-spark/src/test/java/org/apache/kylin/engine/spark/model/planner/CuboIdToLayoutUtilsTest.java
@@ -0,0 +1,215 @@
+/*
+ * 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.kylin.engine.spark.model.planner;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.kylin.metadata.cube.planner.CostBasePlannerUtils;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class CuboIdToLayoutUtilsTest {
+
+    private Map<Integer, Integer> getMap(int dimensionCount) {
+        Map<Integer, Integer> result = new HashMap<>();
+        for (int i = 0; i < dimensionCount; i++) {
+            result.put(i, i);
+        }
+        return result;
+    }
+
+    private List<Integer> getSortList(int dimensionCount) {
+        List<Integer> result = new ArrayList<>();
+        for (int i = 0; i < dimensionCount; i++) {
+            result.add(i);
+        }
+        return result;
+    }
+
+    @Test
+    public void testConvertCuboIdsToColOrders() {
+        int maxCountDimension = 12;
+        Map<Long, Long> cuboids = new HashMap<>();
+        Map<BigInteger, Long> bigIntegerCuboIds = new HashMap<>();
+        long cuboid = 1 << 0 | 1 << 3 | 1 << 7;
+        cuboids.put(cuboid, (long) 0);
+        bigIntegerCuboIds.put(BigInteger.valueOf(cuboid), (long) 0);
+        cuboid = 1 << 4 | 1 << 10 | 1 << 5;
+        cuboids.put(cuboid, (long) 0);
+        bigIntegerCuboIds.put(BigInteger.valueOf(cuboid), (long) 0);
+        List<Integer> measureIds = new ArrayList<>();
+        measureIds.add(1001);
+        measureIds.add(1002);
+        measureIds.add(1003);
+
+        Set<List<Integer>> result = CuboIdToLayoutUtils.convertCuboIdsToColOrders(bigIntegerCuboIds, maxCountDimension,
+                measureIds, getMap(maxCountDimension), getSortList(maxCountDimension));
+        Set<List<Integer>> expected = new LinkedHashSet<>();
+        expected.add(Lists.newArrayList(11 - 7, 11 - 3, 11 - 0, 1001, 1002, 1003));
+        expected.add(Lists.newArrayList(11 - 10, 11 - 5, 11 - 4, 1001, 1002, 1003));
+        assertEquals(expected, result);
+    }
+
+    @Test
+    public void testConvertCuboIdsToColOrdersWithReverseOrder() {
+        int maxCountDimension = 12;
+        Map<BigInteger, Long> cuboids = new HashMap<>();
+        long cuboid = 1 << 0 | 1 << 3 | 1 << 7;
+        BigInteger bigIntegerCuboId = BigInteger.valueOf(cuboid);
+        cuboids.put(bigIntegerCuboId, (long) 0);
+        cuboid = 1 << 4 | 1 << 10 | 1 << 5;
+        bigIntegerCuboId = BigInteger.valueOf(cuboid);
+        cuboids.put(bigIntegerCuboId, (long) 0);
+        List<Integer> measureIds = new ArrayList<>();
+        measureIds.add(1001);
+        measureIds.add(1002);
+        measureIds.add(1003);
+        List<Integer> sortOrder = getSortList(maxCountDimension);
+        Collections.reverse(sortOrder);
+        Set<List<Integer>> result = CuboIdToLayoutUtils.convertCuboIdsToColOrders(cuboids, maxCountDimension,
+                measureIds, getMap(maxCountDimension), sortOrder);
+        Set<List<Integer>> expected = new LinkedHashSet<>();
+        expected.add(Lists.newArrayList(11 - 0, 11 - 3, 11 - 7, 1001, 1002, 1003));
+        expected.add(Lists.newArrayList(11 - 4, 11 - 5, 11 - 10, 1001, 1002, 1003));
+        assertEquals(expected, result);
+    }
+
+    @Test
+    public void testConvertCuboIdsToColOrdersWithEmptyOrder() {
+        int maxCountDimension = 12;
+        Map<BigInteger, Long> cuboids = new HashMap<>();
+        long cuboid = 1 << 0 | 1 << 3 | 1 << 7;
+        BigInteger bigIntegerCuboId = BigInteger.valueOf(cuboid);
+        cuboids.put(bigIntegerCuboId, (long) 0);
+        cuboid = 1 << 4 | 1 << 10 | 1 << 5;
+        bigIntegerCuboId = BigInteger.valueOf(cuboid);
+        cuboids.put(bigIntegerCuboId, (long) 0);
+        List<Integer> measureIds = new ArrayList<>();
+        measureIds.add(1001);
+        measureIds.add(1002);
+        measureIds.add(1003);
+        List<Integer> sortOrder = getSortList(maxCountDimension - 8);
+        Collections.reverse(sortOrder);
+        Set<List<Integer>> result = CuboIdToLayoutUtils.convertCuboIdsToColOrders(cuboids, maxCountDimension,
+                measureIds, getMap(maxCountDimension), sortOrder);
+        Set<List<Integer>> expected = new LinkedHashSet<>();
+        assertEquals(expected, result);
+    }
+
+    @Test
+    public void testConvertCuboIdsToColOrdersWithMultiOrder() {
+        int maxCountDimension = 12;
+        Map<BigInteger, Long> cuboids = new HashMap<>();
+        long cuboid = 1 << 0 | 1 << 3 | 1 << 7;
+        BigInteger bigIntegerCuboId = BigInteger.valueOf(cuboid);
+        cuboids.put(bigIntegerCuboId, (long) 0);
+        cuboid = 1 << 4 | 1 << 10 | 1 << 5;
+        bigIntegerCuboId = BigInteger.valueOf(cuboid);
+        cuboids.put(bigIntegerCuboId, (long) 0);
+        List<Integer> measureIds = new ArrayList<>();
+        measureIds.add(1001);
+        measureIds.add(1002);
+        measureIds.add(1003);
+        List<Integer> sortOrder = getSortList(maxCountDimension);
+        // order1: 0,1,2,3,4,5,6,7,8,9,10,11
+        Set<List<Integer>> result = CuboIdToLayoutUtils.convertCuboIdsToColOrders(cuboids, maxCountDimension,
+                measureIds, getMap(maxCountDimension), sortOrder);
+        Set<List<Integer>> expected = new LinkedHashSet<>();
+        expected.add(Lists.newArrayList(11 - 7, 11 - 3, 11 - 0, 1001, 1002, 1003)); // 4,8,11
+        expected.add(Lists.newArrayList(11 - 10, 11 - 5, 11 - 4, 1001, 1002, 1003)); // 1,6,7
+        // order2: 11,10,9,8,7,6,5,4,3,2,1,0
+        Collections.reverse(sortOrder);
+        result.addAll(CuboIdToLayoutUtils.convertCuboIdsToColOrders(cuboids, maxCountDimension, measureIds,
+                getMap(maxCountDimension), sortOrder));
+        expected.add(Lists.newArrayList(11 - 0, 11 - 3, 11 - 7, 1001, 1002, 1003)); // 11,8,4
+        expected.add(Lists.newArrayList(11 - 4, 11 - 5, 11 - 10, 1001, 1002, 1003)); // 7,6,1
+        assertEquals(expected, result);
+    }
+
+    @Test
+    public void randomTestForDimensionWithCuboid() {
+        int maxCountDimension = 63;
+        Random random = new Random();
+        for (int i = 0; i < 100; i++) {
+            long cuboId = random.nextLong();
+            if (cuboId < 0) {
+                cuboId = Math.abs(cuboId);
+            }
+            BigInteger bigIntegerCuboId = BigInteger.valueOf(cuboId);
+            List<Integer> dimensionIds = CuboIdToLayoutUtils.converCuboidToDimensionColOrder(bigIntegerCuboId,
+                    maxCountDimension, getMap(maxCountDimension), getSortList(maxCountDimension));
+            BigInteger expected = CostBasePlannerUtils.convertDimensionsToCuboId(dimensionIds, maxCountDimension,
+                    getMap(maxCountDimension));
+            assertEquals(expected, BigInteger.valueOf(cuboId));
+        }
+    }
+
+    @Test
+    public void testConvertLongToOrder() {
+        // max id = 11
+        int maxCountDimension = 12;
+        long cuboid = 1 << 0 | 1 << 3 | 1 << 7;
+        BigInteger bigIntegerCuboId = BigInteger.valueOf(cuboid);
+        List<Integer> result = CuboIdToLayoutUtils.converCuboidToDimensionColOrder(bigIntegerCuboId, maxCountDimension,
+                getMap(maxCountDimension), getSortList(maxCountDimension));
+        List<Integer> expected = Lists.newArrayList(11 - 7, 11 - 3, 11 - 0);
+        assertEquals(expected, result);
+
+        cuboid = 1 << 4 | 1 << 10 | 1 << 5;
+        bigIntegerCuboId = BigInteger.valueOf(cuboid);
+        result = CuboIdToLayoutUtils.converCuboidToDimensionColOrder(bigIntegerCuboId, maxCountDimension,
+                getMap(maxCountDimension), getSortList(maxCountDimension));
+        expected = Lists.newArrayList(11 - 10, 11 - 5, 11 - 4);
+        assertEquals(expected, result);
+    }
+
+    @Test
+    public void testMore64DimensionCase() {
+        int maxCountDimension = 100;
+        BigInteger cuboid = BigInteger.ZERO;
+        cuboid = cuboid.setBit(0);
+        cuboid = cuboid.setBit(3);
+        cuboid = cuboid.setBit(7);
+        cuboid = cuboid.setBit(77);
+        List<Integer> result = CuboIdToLayoutUtils.converCuboidToDimensionColOrder(cuboid, maxCountDimension,
+                getMap(maxCountDimension), getSortList(maxCountDimension));
+        List<Integer> expected = Lists.newArrayList(99 - 77, 99 - 7, 99 - 3, 99 - 0);
+        assertEquals(expected, result);
+
+        cuboid = BigInteger.ZERO;
+        cuboid = cuboid.setBit(4);
+        cuboid = cuboid.setBit(10);
+        cuboid = cuboid.setBit(5);
+        cuboid = cuboid.setBit(88);
+        result = CuboIdToLayoutUtils.converCuboidToDimensionColOrder(cuboid, maxCountDimension,
+                getMap(maxCountDimension), getSortList(maxCountDimension));
+        expected = Lists.newArrayList(99 - 88, 99 - 10, 99 - 5, 99 - 4);
+        assertEquals(expected, result);
+    }
+}
\ No newline at end of file
diff --git a/src/spark-project/spark-it/src/test/resources/index_planner/dataflow/d863b37c-e1a9-717f-7df7-74991815b1eb.json b/src/spark-project/spark-it/src/test/resources/index_planner/dataflow/d863b37c-e1a9-717f-7df7-74991815b1eb.json
new file mode 100644
index 0000000000..6d62651457
--- /dev/null
+++ b/src/spark-project/spark-it/src/test/resources/index_planner/dataflow/d863b37c-e1a9-717f-7df7-74991815b1eb.json
@@ -0,0 +1,13 @@
+{
+  "uuid" : "d863b37c-e1a9-717f-7df7-74991815b1eb",
+  "last_modified" : 0,
+  "create_time" : 1675840561793,
+  "version" : "4.0.0.0",
+  "status" : "OFFLINE",
+  "last_status" : null,
+  "cost" : 50,
+  "query_hit_count" : 0,
+  "last_query_time" : 0,
+  "layout_query_hit_count" : { },
+  "segments" : [ ]
+}
\ No newline at end of file
diff --git a/src/spark-project/spark-it/src/test/resources/index_planner/index_plan/d863b37c-e1a9-717f-7df7-74991815b1eb.json b/src/spark-project/spark-it/src/test/resources/index_planner/index_plan/d863b37c-e1a9-717f-7df7-74991815b1eb.json
new file mode 100644
index 0000000000..70bea94702
--- /dev/null
+++ b/src/spark-project/spark-it/src/test/resources/index_planner/index_plan/d863b37c-e1a9-717f-7df7-74991815b1eb.json
@@ -0,0 +1,43 @@
+{
+  "uuid" : "d863b37c-e1a9-717f-7df7-74991815b1eb",
+  "last_modified" : 1675840571235,
+  "create_time" : 1675840561720,
+  "version" : "4.0.0.0",
+  "description" : null,
+  "rule_based_index" : {
+    "dimensions" : [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 ],
+    "measures" : [ 100000 ],
+    "global_dim_cap" : null,
+    "aggregation_groups" : [ {
+      "includes" : [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 ],
+      "measures" : [ 100000 ],
+      "select_rule" : {
+        "hierarchy_dims" : [ ],
+        "mandatory_dims" : [ 0 ],
+        "joint_dims" : [ ]
+      },
+      "index_range" : "EMPTY"
+    } ],
+    "layout_id_mapping" : [ 10001, 20001, 30001, 40001, 50001, 60001, 70001, 80001, 90001, 100001, 110001, 120001, 130001, 140001, 150001, 160001, 170001, 180001, 190001, 200001, 210001, 220001, 230001, 240001, 250001, 260001, 270001, 280001, 290001, 300001, 310001, 320001, 330001, 340001, 350001, 360001, 370001, 380001, 390001, 400001, 410001, 420001, 430001, 440001, 450001, 460001, 470001, 480001, 490001, 500001, 510001, 520001, 530001, 540001, 550001, 560001, 570001, 580001, 590001, 6 [...]
+    "parent_forward" : 3,
+    "index_start_id" : 10000,
+    "last_modify_time" : 1675840582677,
+    "layout_black_list" : [ ],
+    "layout_cost_based_pruned_list" : null,
+    "scheduler_version" : 2,
+    "index_update_enabled" : true
+  },
+  "indexes" : [ ],
+  "override_properties" : { },
+  "to_be_deleted_indexes" : [ ],
+  "auto_merge_time_ranges" : null,
+  "retention_range" : 0,
+  "engine_type" : 80,
+  "next_aggregation_index_id" : 20490000,
+  "next_table_index_id" : 20000000000,
+  "agg_shard_by_columns" : [ ],
+  "extend_partition_columns" : [ ],
+  "layout_bucket_num" : { },
+  "approved_additional_recs" : 0,
+  "approved_removal_recs" : 0
+}
\ No newline at end of file
diff --git a/src/spark-project/spark-it/src/test/resources/index_planner/model_desc/d863b37c-e1a9-717f-7df7-74991815b1eb.json b/src/spark-project/spark-it/src/test/resources/index_planner/model_desc/d863b37c-e1a9-717f-7df7-74991815b1eb.json
new file mode 100644
index 0000000000..34a49ab95e
--- /dev/null
+++ b/src/spark-project/spark-it/src/test/resources/index_planner/model_desc/d863b37c-e1a9-717f-7df7-74991815b1eb.json
@@ -0,0 +1,128 @@
+{
+  "uuid" : "d863b37c-e1a9-717f-7df7-74991815b1eb",
+  "last_modified" : 1675840561719,
+  "create_time" : 1675840561668,
+  "version" : "4.0.0.0",
+  "alias" : "TEST_CUBE_PLANNER",
+  "owner" : "ADMIN",
+  "config_last_modifier" : null,
+  "config_last_modified" : 0,
+  "description" : "",
+  "fact_table" : "DEFAULT.TEST_KYLIN_FACT",
+  "fact_table_alias" : null,
+  "management_type" : "MODEL_BASED",
+  "join_tables" : [ ],
+  "filter_condition" : "",
+  "partition_desc" : {
+    "partition_date_column" : "TEST_KYLIN_FACT.CAL_DT",
+    "partition_date_start" : 0,
+    "partition_date_format" : "yyyy-MM-dd",
+    "partition_type" : "APPEND",
+    "partition_condition_builder" : "org.apache.kylin.metadata.model.PartitionDesc$DefaultPartitionConditionBuilder"
+  },
+  "capacity" : "MEDIUM",
+  "segment_config" : {
+    "auto_merge_enabled" : null,
+    "auto_merge_time_ranges" : null,
+    "volatile_range" : null,
+    "retention_range" : null,
+    "create_empty_segment_enabled" : false
+  },
+  "data_check_desc" : null,
+  "semantic_version" : 0,
+  "storage_type" : 0,
+  "model_type" : "BATCH",
+  "all_named_columns" : [ {
+    "id" : 0,
+    "name" : "TRANS_ID",
+    "column" : "TEST_KYLIN_FACT.TRANS_ID",
+    "status" : "DIMENSION"
+  }, {
+    "id" : 1,
+    "name" : "LEAF_CATEG_ID",
+    "column" : "TEST_KYLIN_FACT.LEAF_CATEG_ID",
+    "status" : "DIMENSION"
+  }, {
+    "id" : 2,
+    "name" : "LSTG_SITE_ID",
+    "column" : "TEST_KYLIN_FACT.LSTG_SITE_ID",
+    "status" : "DIMENSION"
+  }, {
+    "id" : 3,
+    "name" : "ITEM_COUNT",
+    "column" : "TEST_KYLIN_FACT.ITEM_COUNT",
+    "status" : "DIMENSION"
+  }, {
+    "id" : 4,
+    "name" : "ORDER_ID",
+    "column" : "TEST_KYLIN_FACT.ORDER_ID",
+    "status" : "DIMENSION"
+  }, {
+    "id" : 5,
+    "name" : "CAL_DT",
+    "column" : "TEST_KYLIN_FACT.CAL_DT",
+    "status" : "DIMENSION"
+  }, {
+    "id" : 6,
+    "name" : "LSTG_FORMAT_NAME",
+    "column" : "TEST_KYLIN_FACT.LSTG_FORMAT_NAME",
+    "status" : "DIMENSION"
+  }, {
+    "id" : 7,
+    "name" : "TEST_COUNT_DISTINCT_BITMAP",
+    "column" : "TEST_KYLIN_FACT.TEST_COUNT_DISTINCT_BITMAP",
+    "status" : "DIMENSION"
+  }, {
+    "id" : 8,
+    "name" : "IS_EFFECTUAL",
+    "column" : "TEST_KYLIN_FACT.IS_EFFECTUAL",
+    "status" : "DIMENSION"
+  }, {
+    "id" : 9,
+    "name" : "SLR_SEGMENT_CD",
+    "column" : "TEST_KYLIN_FACT.SLR_SEGMENT_CD",
+    "status" : "DIMENSION"
+  }, {
+    "id" : 10,
+    "name" : "SELLER_ID",
+    "column" : "TEST_KYLIN_FACT.SELLER_ID",
+    "status" : "DIMENSION"
+  }, {
+    "id" : 11,
+    "name" : "PRICE",
+    "column" : "TEST_KYLIN_FACT.PRICE",
+    "status" : "DIMENSION"
+  } ],
+  "all_measures" : [ {
+    "name" : "COUNT_ALL",
+    "function" : {
+      "expression" : "COUNT",
+      "parameters" : [ {
+        "type" : "constant",
+        "value" : "1"
+      } ],
+      "returntype" : "bigint"
+    },
+    "column" : null,
+    "comment" : null,
+    "id" : 100000,
+    "type" : "NORMAL",
+    "internal_ids" : [ ]
+  } ],
+  "recommendations_count" : 0,
+  "computed_columns" : [ ],
+  "canvas" : {
+    "coordinate" : {
+      "TEST_KYLIN_FACT" : {
+        "x" : 419.94439019097223,
+        "y" : 143.38887532552084,
+        "width" : 220.0,
+        "height" : 200.0
+      }
+    },
+    "zoom" : 9.0
+  },
+  "multi_partition_desc" : null,
+  "multi_partition_key_mapping" : null,
+  "fusion_id" : null
+}
\ No newline at end of file
diff --git a/src/spark-project/spark-it/src/test/scala/org/apache/kylin/common/JobSupport.scala b/src/spark-project/spark-it/src/test/scala/org/apache/kylin/common/JobSupport.scala
index ce44815a35..bfd0e8603a 100644
--- a/src/spark-project/spark-it/src/test/scala/org/apache/kylin/common/JobSupport.scala
+++ b/src/spark-project/spark-it/src/test/scala/org/apache/kylin/common/JobSupport.scala
@@ -17,6 +17,9 @@
 
 package org.apache.kylin.common
 
+import java.io.File
+import java.util.Objects
+
 import com.google.common.collect.{Lists, Maps, Sets}
 import org.apache.commons.io.FileUtils
 import org.apache.hadoop.conf.Configuration
@@ -31,22 +34,19 @@ import org.apache.kylin.engine.spark.utils.{FileNames, HDFSUtils}
 import org.apache.kylin.job.engine.JobEngineConfig
 import org.apache.kylin.job.execution.{AbstractExecutable, ExecutableState, NExecutableManager}
 import org.apache.kylin.job.impl.threadpool.NDefaultScheduler
-import org.apache.kylin.metadata.cube.model.{LayoutEntity, NDataSegment, NDataflow, NDataflowManager, NDataflowUpdate}
+import org.apache.kylin.metadata.cube.model._
 import org.apache.kylin.metadata.model.SegmentRange
 import org.apache.kylin.metadata.realization.RealizationStatusEnum
 import org.apache.kylin.query.runtime.plan.TableScanPlan
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.common.SparderQueryTest
+import org.apache.spark.sql.functions._
 import org.junit.Assert
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, Suite}
-import org.apache.spark.sql.functions._
 
 import scala.collection.JavaConverters._
 
-import java.io.File
-import java.util.Objects
-
 trait JobSupport
   extends BeforeAndAfterAll
     with BeforeAndAfterEach
@@ -145,6 +145,45 @@ trait JobSupport
     })
   }
 
+  @throws[Exception]
+  protected def buildSegment(cubeName: String,
+                           segmentRange: SegmentRange[_ <: Comparable[_]],
+                           toBuildLayouts: java.util.Set[LayoutEntity],
+                           prj: String): NDataSegment = {
+    val config: KylinConfig = KylinConfig.getInstanceFromEnv
+    val dsMgr: NDataflowManager = NDataflowManager.getInstance(config, prj)
+    val execMgr: NExecutableManager =
+      NExecutableManager.getInstance(config, prj)
+    val df: NDataflow = dsMgr.getDataflow(cubeName)
+    // ready dataflow, segment, cuboid layout
+    val oneSeg: NDataSegment = dsMgr.appendSegment(df, segmentRange)
+    // create job, and the job type is `inc_build`
+    val job: NSparkCubingJob = NSparkCubingJob.createIncBuildJob(Sets.newHashSet(oneSeg), toBuildLayouts, "ADMIN", null)
+    val sparkStep: NSparkCubingStep = job.getSparkCubingStep
+    val distMetaUrl: StorageURL = StorageURL.valueOf(sparkStep.getDistMetaUrl)
+    Assert.assertEquals("hdfs", distMetaUrl.getScheme)
+    Assert.assertTrue(
+      distMetaUrl
+        .getParameter("path")
+        .startsWith(config.getHdfsWorkingDirectory))
+    // launch the job
+    execMgr.addJob(job)
+    if (!Objects.equals(wait(job), ExecutableState.SUCCEED)) {
+      val message = job.getTasks.asScala
+        .find(executable => Objects.equals(executable.getStatus, ExecutableState.ERROR))
+        .map(task => execMgr.getOutputFromHDFSByJobId(job.getId, task.getId, Integer.MAX_VALUE).getVerboseMsg)
+        .getOrElse("Unknown Error")
+      throw new IllegalStateException(message);
+    }
+
+    val buildStore: ResourceStore = ExecutableUtils.getRemoteStore(config, job.getSparkCubingStep)
+    val merger: AfterBuildResourceMerger = new AfterBuildResourceMerger(config, prj)
+    val layoutIds: java.util.Set[java.lang.Long] = toBuildLayouts.asScala.map(c => new java.lang.Long(c.getId)).asJava
+    merger.mergeAfterIncrement(df.getUuid, oneSeg.getId, layoutIds, buildStore)
+    checkSnapshotTable(df.getId, oneSeg.getId, oneSeg.getProject)
+    oneSeg
+  }
+
   @throws[Exception]
   protected def builCuboid(cubeName: String,
                            segmentRange: SegmentRange[_ <: Comparable[_]],
@@ -195,6 +234,37 @@ trait JobSupport
     null
   }
 
+  @throws[Exception]
+  def buildOneSegementForCubePlanner(dfName: String,
+                                     prj: String = DEFAULT_PROJECT): Unit = {
+    val config = KylinConfig.getInstanceFromEnv
+    val dsMgr = NDataflowManager.getInstance(config, DEFAULT_PROJECT)
+    var df = dsMgr.getDataflow(dfName)
+    Assert.assertTrue(config.getHdfsWorkingDirectory.startsWith("file:"))
+
+    // cleanup all segments first
+    val update = new NDataflowUpdate(df.getUuid)
+    update.setToRemoveSegsWithArray(df.getSegments.asScala.toArray)
+    dsMgr.updateDataflow(update)
+
+    // build one segment for cube planner
+    val layouts = df.getIndexPlan.getAllLayouts
+    var start = SegmentRange.dateToLong("2010-01-01")
+    var end = SegmentRange.dateToLong("2023-01-01")
+    var segment = buildSegment(dfName,
+      new SegmentRange.TimePartitionedSegmentRange(start, end),
+      Sets.newLinkedHashSet(layouts),
+      prj)
+    logInfo(s"build cube planner segment: ${segment}")
+
+    // validate the first segment for build
+    val firstSegment = dsMgr.getDataflow(dfName).getSegments().get(0)
+    Assert.assertEquals(new SegmentRange.TimePartitionedSegmentRange(
+      SegmentRange.dateToLong("2010-01-01"),
+      SegmentRange.dateToLong("2023-01-01")),
+      firstSegment.getSegRange)
+  }
+
   @throws[Exception]
   def buildFourSegementAndMerge(dfName: String,
                                 prj: String = DEFAULT_PROJECT): Unit = {
diff --git a/src/spark-project/spark-it/src/test/scala/org/apache/kylin/common/SSSource.scala b/src/spark-project/spark-it/src/test/scala/org/apache/kylin/common/SSSource.scala
index b23641993c..dd25fd05f5 100644
--- a/src/spark-project/spark-it/src/test/scala/org/apache/kylin/common/SSSource.scala
+++ b/src/spark-project/spark-it/src/test/scala/org/apache/kylin/common/SSSource.scala
@@ -17,17 +17,24 @@
 
 package org.apache.kylin.common
 
+import java.io.{DataInputStream, File}
+import java.nio.charset.Charset
+import java.nio.file.Files
+import java.util.Locale
+
 import com.google.common.base.Preconditions
+import org.apache.commons.io.IOUtils
+import org.apache.commons.lang.StringUtils
+import org.apache.kylin.common.persistence.{JsonSerializer, RootPersistentEntity}
 import org.apache.kylin.common.util.TempMetadataBuilder
-import org.apache.kylin.metadata.model.NTableMetadataManager
+import org.apache.kylin.metadata.cube.model.{IndexPlan, NDataflowManager, NIndexPlanManager}
+import org.apache.kylin.metadata.model.{NDataModel, NDataModelManager, NTableMetadataManager}
 import org.apache.kylin.metadata.project.NProjectManager
 import org.apache.kylin.query.util.{QueryParams, QueryUtil}
 import org.apache.spark.sql.common.{LocalMetadata, SharedSparkSession}
 import org.apache.spark.sql.execution.utils.SchemaProcessor
 import org.scalatest.Suite
 
-import java.util.Locale
-
 trait SSSource extends SharedSparkSession with LocalMetadata {
   self: Suite =>
 
@@ -79,4 +86,25 @@ trait SSSource extends SharedSparkSession with LocalMetadata {
     queryParams.setKylinConfig(NProjectManager.getProjectConfig("default"))
     QueryUtil.massagePushDownSql(queryParams)
   }
+
+  def addModels(resourcePath: String, modelIds: Seq[String]): Unit = {
+    val modelMgr = NDataModelManager.getInstance(KylinConfig.getInstanceFromEnv, getProject)
+    val indexPlanMgr = NIndexPlanManager.getInstance(KylinConfig.getInstanceFromEnv, getProject)
+    val dfMgr = NDataflowManager.getInstance(KylinConfig.getInstanceFromEnv, getProject)
+    modelIds.foreach { id =>
+      val model = read(classOf[NDataModel], resourcePath, s"model_desc/$id.json")
+      model.setProject(getProject)
+      modelMgr.createDataModelDesc(model, "ADMIN")
+      dfMgr.createDataflow(indexPlanMgr.createIndexPlan(
+        read(classOf[IndexPlan], resourcePath, s"index_plan/$id.json")), "ADMIN")
+    }
+  }
+
+  private def read[T <: RootPersistentEntity](clz: Class[T], prePath: String, subPath: String): T = {
+    val path = prePath + subPath;
+    // val path = "src/test/resources/view/" + subPath
+    val contents = StringUtils.join(Files.readAllLines(new File(path).toPath, Charset.defaultCharset), "\n")
+    val bais = IOUtils.toInputStream(contents, Charset.defaultCharset)
+    new JsonSerializer[T](clz).deserialize(new DataInputStream(bais))
+  }
 }
diff --git a/src/spark-project/spark-it/src/test/scala/org/apache/kylin/it/TestCubePlanner.scala b/src/spark-project/spark-it/src/test/scala/org/apache/kylin/it/TestCubePlanner.scala
new file mode 100644
index 0000000000..926e99ad15
--- /dev/null
+++ b/src/spark-project/spark-it/src/test/scala/org/apache/kylin/it/TestCubePlanner.scala
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.it
+
+import java.util
+import java.util.TimeZone
+
+import org.apache.kylin.common._
+import org.apache.kylin.metadata.cube.model.NDataflowManager.NDataflowUpdater
+import org.apache.kylin.metadata.cube.model._
+import org.apache.kylin.metadata.realization.RealizationStatusEnum
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparderEnv
+import org.apache.spark.sql.common.{LocalMetadata, SparderBaseFunSuite}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.execution.utils.SchemaProcessor
+
+class TestCubePlanner extends SparderBaseFunSuite
+  with LocalMetadata
+  with JobSupport
+  with QuerySupport
+  with CompareSupport
+  with SSSource
+  with AdaptiveSparkPlanHelper
+  with Logging {
+
+  override val DEFAULT_PROJECT = "default"
+  override protected def getProject: String = DEFAULT_PROJECT
+
+  private val DF_NAME = "d863b37c-e1a9-717f-7df7-74991815b1eb"
... 178 lines suppressed ...