You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2017/11/23 05:53:03 UTC

[01/18] kylin git commit: APACHE-KYLIN-2734 Introduce hot cuboids export & import

Repository: kylin
Updated Branches:
  refs/heads/ci-dong [created] 74bad59ba


APACHE-KYLIN-2734 Introduce hot cuboids export & import


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/74bad59b
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/74bad59b
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/74bad59b

Branch: refs/heads/ci-dong
Commit: 74bad59ba21ecd5c3a8ad58634894369d5077686
Parents: 1fce193
Author: liapan <li...@ebay.com>
Authored: Thu Sep 28 11:39:53 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 webapp/app/js/controllers/cubeAdvanceSetting.js | 54 +++++++++++++++
 webapp/app/js/model/cubeDescModel.js            |  4 +-
 .../cubeDesigner/advanced_settings.html         | 69 ++++++++++++++++++++
 webapp/app/partials/cubes/cube_detail.html      | 11 ++++
 4 files changed, 136 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/74bad59b/webapp/app/js/controllers/cubeAdvanceSetting.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeAdvanceSetting.js b/webapp/app/js/controllers/cubeAdvanceSetting.js
index 73616f2..8704ed8 100755
--- a/webapp/app/js/controllers/cubeAdvanceSetting.js
+++ b/webapp/app/js/controllers/cubeAdvanceSetting.js
@@ -382,4 +382,58 @@ KylinApp.controller('CubeAdvanceSettingCtrl', function ($scope, $modal,cubeConfi
     }
   };
 
+  $scope.mandatoryDimensionSet = {
+    select: []
+  };
+
+  $scope.uploadMandatoryDimensionSetList = function() {
+    var file = document.getElementById('cuboids').files[0];
+    if (file) {
+      var reader = new FileReader();
+      reader.onload = function(event) {
+        var dimensionSetList = JSON.parse(event.target.result);
+        $scope.cubeMetaFrame.mandatory_dimension_set_list = dimensionSetList;
+        $scope.$apply();
+        // TODO add verify dimension set
+      };
+      reader.readAsText(file);
+    } else {
+      swal('Oops...', 'Please choose your file first.', 'warning');
+    }
+  };
+
+  $scope.removeDimensionSet = function(index) {
+    $scope.cubeMetaFrame.mandatory_dimension_set_list.splice(index, 1);
+  };
+
+  $scope.addDimensionSet = function() {
+    if ($scope.mandatoryDimensionSet.select.length) {
+      // validate the dimension set existed
+      var existed = false;
+      var selectedDimension = _.clone($scope.mandatoryDimensionSet.select).sort(function (dimensionA, dimensionB) {
+        if (dimensionA < dimensionB) return 1;
+        if (dimensionB < dimensionA) return -1;
+        return 0;
+      });
+      angular.forEach($scope.cubeMetaFrame.mandatory_dimension_set_list, function(dimensionSet, index) {
+        var dimensionSetSorted = _.clone(dimensionSet).sort(function (dimensionA, dimensionB) {
+          if (dimensionA < dimensionB) return 1;
+          if (dimensionB < dimensionA) return -1;
+          return 0;
+        });
+        if (angular.equals(dimensionSet, selectedDimension)) {
+          existed = true;
+        };
+      });
+      if (!existed) {
+        $scope.cubeMetaFrame.mandatory_dimension_set_list.push($scope.mandatoryDimensionSet.select);
+        $scope.mandatoryDimensionSet.select = [];
+      } else {
+        swal('Oops...', 'Dimension set already existed', 'warning');
+      }
+    } else {
+      swal('Oops...', 'Dimension set should not be empty', 'warning');
+    }
+  };
+
 });

http://git-wip-us.apache.org/repos/asf/kylin/blob/74bad59b/webapp/app/js/model/cubeDescModel.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/cubeDescModel.js b/webapp/app/js/model/cubeDescModel.js
index cf44a25..9bf969f 100644
--- a/webapp/app/js/model/cubeDescModel.js
+++ b/webapp/app/js/model/cubeDescModel.js
@@ -44,8 +44,8 @@ KylinApp.service('CubeDescModel', function (kylinConfig) {
       "rowkey": {
         "rowkey_columns": []
       },
-      "aggregation_groups": []
-      ,
+      "aggregation_groups": [],
+      "mandatory_dimension_set_list": [],
       "partition_date_start":0,
       "partition_date_end":undefined,
       "notify_list": [],

http://git-wip-us.apache.org/repos/asf/kylin/blob/74bad59b/webapp/app/partials/cubeDesigner/advanced_settings.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubeDesigner/advanced_settings.html b/webapp/app/partials/cubeDesigner/advanced_settings.html
index 5d53075..c423097 100755
--- a/webapp/app/partials/cubeDesigner/advanced_settings.html
+++ b/webapp/app/partials/cubeDesigner/advanced_settings.html
@@ -289,6 +289,75 @@
               </div>
           </div>
         </div>
+        <!--Eidt Mandatory Cuboids-->
+        <div class="form-group large-popover" ng-if="state.mode=='edit' || (cubeMetaFrame.mandatory_dimension_set_list.length > 0 && state.mode!=='edit')">
+          <h3 style="margin-left:42px">Mandatory Cuboids</h3>
+          <div style="margin-left:42px;">
+            <div class="box-body">
+              <!-- view mode-->
+              <table ng-if="cubeMetaFrame.mandatory_dimension_set_list.length > 0 && state.mode!=='edit'" class="table table-hover table-bordered list" style="table-layout: fixed;margin-left:42px;width:92%;">
+                <thead>
+                  <tr>
+                    <th>Cuboids</th>
+                  </tr>
+                </thead>
+                <tbody>
+                  <tr ng-repeat="dimensionSet in cubeMetaFrame.mandatory_dimension_set_list">
+                    <td>
+                      <p>{{dimensionSet}}</p>
+                    </td>
+                  </tr>
+                </tbody>
+              </table>
+              <!-- edit mode-->
+              <table ng-if="state.mode=='edit'" class="table table-hover table-bordered list" style="table-layout: fixed;margin-left:42px;width:92%;">
+                <thead>
+                  <tr>
+                    <th>Cuboids</th>
+                    <th style="width:5%"></th>
+                  </tr>
+                </thead>
+                <tbody>
+                  <tr ng-repeat="dimensionSet in cubeMetaFrame.mandatory_dimension_set_list">
+                    <td>
+                      <ui-select multiple ng-model="dimensionSet" theme="bootstrap" sortable="true" close-on-select="false" class="form-control">
+                        <ui-select-match placeholder="Select Dimension...">{{$item.column}}</ui-select-match>
+                        <ui-select-choices repeat="dimension.column as dimension in cubeMetaFrame.rowkey.rowkey_columns | filter:$select.search">
+                          <div ng-bind-html="dimension.column | highlight: $select.search"></div>
+                        </ui-select-choices>
+                      </ui-select>
+                    </td>
+                    <td>
+                      <button class="btn btn-xs btn-info" ng-click="removeDimensionSet($index)"><i class="fa fa-minus"></i>
+                      </button>
+                    </td>
+                  </tr>
+                  <tr>
+                    <td>
+                      <ui-select multiple ng-model="mandatoryDimensionSet.select" theme="bootstrap" sortable="true" close-on-select="false" class="form-control">
+                        <ui-select-match placeholder="Select Dimension...">{{$item.column}}</ui-select-match>
+                        <ui-select-choices repeat="dimension.column as dimension in cubeMetaFrame.rowkey.rowkey_columns | filter:$select.search">
+                          <div ng-bind-html="dimension.column | highlight: $select.search"></div>
+                        </ui-select-choices>
+                      </ui-select>
+                    </td>
+                    <td>
+                      <button class="btn btn-xs btn-info" ng-click="addDimensionSet()"><i class="fa fa-plus"></i>
+                      </button>
+                    </td>
+                  </tr>
+                  <tr>
+                    <td colspan="2">
+                      <label>Import cuboids from file:</label>
+                      <input type="file" name="cuboids" id="cuboids" style="display: inline;"/>
+                      <button class="btn btn-xs btn-info" ng-click="uploadMandatoryDimensionSetList()">Upload</button>
+                    </td>
+                  </tr>
+                </tbody>
+              </table>
+            </div>
+          </div>
+        </div>
         <!--Cube Engine-->
         <div class="form-group large-popover" style="margin-bottom:30px;">
           <h3 style="margin-left:42px;margin-bottom:30px;">Cube Engine  <i kylinpopover placement="right" title="Cube Engine" template="CubeEngineTip.html" class="fa fa-info-circle"></i></h3>

http://git-wip-us.apache.org/repos/asf/kylin/blob/74bad59b/webapp/app/partials/cubes/cube_detail.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubes/cube_detail.html b/webapp/app/partials/cubes/cube_detail.html
index e80bb09..cc41b63 100755
--- a/webapp/app/partials/cubes/cube_detail.html
+++ b/webapp/app/partials/cubes/cube_detail.html
@@ -154,6 +154,17 @@
             </div>
             <div class="row">
                 <div class="col-sm-12">
+                    <div class="dropup pull-right" style="margin-left: 10px;" ng-if="recommendData">
+                        <button class="btn btn-default dropdown-toggle" type="button" id="export" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">
+                            Export
+                            <span class="caret"></span>
+                        </button>
+                        <ul class="dropdown-menu" aria-labelledby="export">
+                            <li><a ng-href="{{config.service.url}}cubes/{{cube.name}}/cuboids/export?top=10" target="_blank" >Top 10</a></li>
+                            <li><a ng-href="{{config.service.url}}cubes/{{cube.name}}/cuboids/export?top=50" target="_blank" >Top 50</a></li>
+                            <li><a ng-href="{{config.service.url}}cubes/{{cube.name}}/cuboids/export?top=100" target="_blank" >Top 100</a></li>
+                        </ul>
+                    </div>
                     <button class="btn btn-success btn-next pull-right" ng-click="optimizeCuboids(cube)" ng-if="recommendData">
                         Optimize
                     </button>


[02/18] kylin git commit: APACHE-KYLIN-2735: Introduce an option to make job scheduler consider job priority

Posted by li...@apache.org.
APACHE-KYLIN-2735: Introduce an option to make job scheduler consider job priority


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/12fefdc2
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/12fefdc2
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/12fefdc2

Branch: refs/heads/ci-dong
Commit: 12fefdc2b957b36389c371443e1e84f774df87ab
Parents: 950e3db
Author: Zhong <nj...@apache.org>
Authored: Thu Aug 31 14:56:19 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/KylinConfigBase.java    |   8 ++
 .../kylin/job/engine/JobEngineConfig.java       |  14 +++
 .../kylin/job/execution/AbstractExecutable.java |   9 ++
 .../job/execution/CheckpointExecutable.java     |   7 ++
 .../job/execution/DefaultChainedExecutable.java |   7 ++
 .../job/impl/threadpool/DefaultScheduler.java   | 111 ++++++++++++++++++-
 .../org/apache/kylin/engine/mr/CubingJob.java   |  19 +++-
 7 files changed, 171 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/12fefdc2/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 3d67ee3..04680ed 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -577,6 +577,14 @@ abstract public class KylinConfigBase implements Serializable {
         return Integer.parseInt(getOptional("kylin.job.scheduler.default", "0"));
     }
 
+    public boolean getSchedulerPriorityConsidered() {
+        return Boolean.parseBoolean(getOptional("kylin.job.scheduler.priority-considered", "false"));
+    }
+
+    public Integer getSchedulerPriorityBarFetchFromQueue() {
+        return Integer.parseInt(getOptional("kylin.job.scheduler.priority-bar-fetch-from-queue", "20"));
+    }
+
     public Integer getSchedulerPollIntervalSecond() {
         return Integer.parseInt(getOptional("kylin.job.scheduler.poll-interval-second", "30"));
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/12fefdc2/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java b/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
index 6890557..9ba602f 100644
--- a/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
+++ b/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
@@ -105,6 +105,20 @@ public class JobEngineConfig {
         return config;
     }
 
+    /**
+     * @return if consider job priority when scheduling jobs
+     * */
+    public boolean getJobPriorityConsidered() {
+        return config.getSchedulerPriorityConsidered();
+    }
+
+    /**
+     * @return the priority bar for fetching jobs from job priority queue
+     */
+    public int getFetchQueuePriorityBar() {
+        return config.getSchedulerPriorityBarFetchFromQueue();
+    }
+
     public String getHdfsWorkingDirectory() {
         return config.getHdfsWorkingDirectory();
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/12fefdc2/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java b/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
index 30b6421..a37cdc9 100644
--- a/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
@@ -44,6 +44,8 @@ import com.google.common.collect.Maps;
  */
 public abstract class AbstractExecutable implements Executable, Idempotent {
 
+    public static final Integer DEFAULT_PRIORITY = 10;
+
     protected static final String SUBMITTER = "submitter";
     protected static final String NOTIFY_LIST = "notify_list";
     protected static final String START_TIME = "startTime";
@@ -389,6 +391,13 @@ public abstract class AbstractExecutable implements Executable, Idempotent {
         return output.getState() == ExecutableState.READY;
     }
 
+    /**
+     * The larger the value, the higher priority
+     * */
+    public int getDefaultPriority() {
+        return DEFAULT_PRIORITY;
+    }
+
     /*
     * discarded is triggered by JobService, the Scheduler is not awake of that
     *

http://git-wip-us.apache.org/repos/asf/kylin/blob/12fefdc2/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java b/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java
index 9864400..db477cb 100644
--- a/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java
@@ -29,6 +29,8 @@ public class CheckpointExecutable extends DefaultChainedExecutable {
 
     private static final Logger logger = LoggerFactory.getLogger(CheckpointExecutable.class);
 
+    public static final Integer DEFAULT_PRIORITY = 30;
+
     private static final String DEPLOY_ENV_NAME = "envName";
     private static final String PROJECT_INSTANCE_NAME = "projectName";
 
@@ -75,4 +77,9 @@ public class CheckpointExecutable extends DefaultChainedExecutable {
     public void setProjectName(String name) {
         setParam(PROJECT_INSTANCE_NAME, name);
     }
+
+    @Override
+    public int getDefaultPriority() {
+        return DEFAULT_PRIORITY;
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/12fefdc2/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
index 54b3b0a..ff8dfee 100644
--- a/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
@@ -31,6 +31,8 @@ import com.google.common.collect.Maps;
  */
 public class DefaultChainedExecutable extends AbstractExecutable implements ChainedExecutable {
 
+    public static final Integer DEFAULT_PRIORITY = 10;
+
     private final List<AbstractExecutable> subTasks = Lists.newArrayList();
 
     public DefaultChainedExecutable() {
@@ -167,4 +169,9 @@ public class DefaultChainedExecutable extends AbstractExecutable implements Chai
         executable.setId(getId() + "-" + String.format("%02d", subTasks.size()));
         this.subTasks.add(executable);
     }
+
+    @Override
+    public int getDefaultPriority() {
+        return DEFAULT_PRIORITY;
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/12fefdc2/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
index 6ef9c81..287f215 100644
--- a/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
+++ b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
@@ -18,7 +18,9 @@
 
 package org.apache.kylin.job.impl.threadpool;
 
+import java.util.Comparator;
 import java.util.Map;
+import java.util.PriorityQueue;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
@@ -29,6 +31,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.common.util.SetThreadName;
 import org.apache.kylin.job.Scheduler;
 import org.apache.kylin.job.engine.JobEngineConfig;
@@ -51,7 +54,7 @@ public class DefaultScheduler implements Scheduler<AbstractExecutable>, Connecti
 
     private JobLock jobLock;
     private ExecutableManager executableManager;
-    private FetcherRunner fetcher;
+    private Runnable fetcher;
     private ScheduledExecutorService fetcherPool;
     private ExecutorService jobPool;
     private DefaultContext context;
@@ -69,6 +72,110 @@ public class DefaultScheduler implements Scheduler<AbstractExecutable>, Connecti
         }
     }
 
+    private class FetcherRunnerWithPriority implements Runnable {
+        volatile PriorityQueue<Pair<AbstractExecutable, Integer>> jobPriorityQueue = new PriorityQueue<>(1,
+                new Comparator<Pair<AbstractExecutable, Integer>>() {
+                    @Override
+                    public int compare(Pair<AbstractExecutable, Integer> o1, Pair<AbstractExecutable, Integer> o2) {
+                        return o1.getSecond() > o2.getSecond() ? -1 : 1;
+                    }
+                });
+
+        private void addToJobPool(AbstractExecutable executable, int priority) {
+            String jobDesc = executable.toString();
+            logger.info(jobDesc + " prepare to schedule and its priority is " + priority);
+            try {
+                context.addRunningJob(executable);
+                jobPool.execute(new JobRunner(executable));
+                logger.info(jobDesc + " scheduled");
+            } catch (Exception ex) {
+                context.removeRunningJob(executable);
+                logger.warn(jobDesc + " fail to schedule", ex);
+            }
+        }
+
+        @Override
+        synchronized public void run() {
+            try {
+                // logger.debug("Job Fetcher is running...");
+                Map<String, Executable> runningJobs = context.getRunningJobs();
+
+                // fetch job from jobPriorityQueue first to reduce chance to scan job list
+                Map<String, Integer> leftJobPriorities = Maps.newHashMap();
+                Pair<AbstractExecutable, Integer> executableWithPriority;
+                while ((executableWithPriority = jobPriorityQueue.peek()) != null
+                        // the priority of jobs in pendingJobPriorities should be above a threshold
+                        && executableWithPriority.getSecond() >= jobEngineConfig.getFetchQueuePriorityBar()) {
+                    executableWithPriority = jobPriorityQueue.poll();
+                    AbstractExecutable executable = executableWithPriority.getFirst();
+                    int curPriority = executableWithPriority.getSecond();
+                    // the job should wait more than one time
+                    if (curPriority > executable.getDefaultPriority() + 1) {
+                        addToJobPool(executable, curPriority);
+                    } else {
+                        leftJobPriorities.put(executable.getId(), curPriority + 1);
+                    }
+                }
+
+                if (runningJobs.size() >= jobEngineConfig.getMaxConcurrentJobLimit()) {
+                    logger.warn("There are too many jobs running, Job Fetch will wait until next schedule time");
+                    return;
+                }
+
+                while ((executableWithPriority = jobPriorityQueue.poll()) != null) {
+                    leftJobPriorities.put(executableWithPriority.getFirst().getId(),
+                            executableWithPriority.getSecond() + 1);
+                }
+
+                int nRunning = 0, nReady = 0, nStopped = 0, nOthers = 0, nError = 0, nDiscarded = 0, nSUCCEED = 0;
+                for (final String id : executableManager.getAllJobIds()) {
+                    if (runningJobs.containsKey(id)) {
+                        // logger.debug("Job id:" + id + " is already running");
+                        nRunning++;
+                        continue;
+                    }
+
+                    AbstractExecutable executable = executableManager.getJob(id);
+                    if (!executable.isReady()) {
+                        final Output output = executableManager.getOutput(id);
+                        // logger.debug("Job id:" + id + " not runnable");
+                        if (output.getState() == ExecutableState.DISCARDED) {
+                            nDiscarded++;
+                        } else if (output.getState() == ExecutableState.ERROR) {
+                            nError++;
+                        } else if (output.getState() == ExecutableState.SUCCEED) {
+                            nSUCCEED++;
+                        } else if (output.getState() == ExecutableState.STOPPED) {
+                            nStopped++;
+                        } else {
+                            nOthers++;
+                        }
+                        continue;
+                    }
+
+                    nReady++;
+                    Integer priority = leftJobPriorities.get(id);
+                    if (priority == null) {
+                        priority = executable.getDefaultPriority();
+                    }
+                    jobPriorityQueue.add(new Pair<>(executable, priority));
+                }
+
+                while (runningJobs.size() < jobEngineConfig.getMaxConcurrentJobLimit()
+                        && (executableWithPriority = jobPriorityQueue.poll()) != null) {
+                    addToJobPool(executableWithPriority.getFirst(), executableWithPriority.getSecond());
+                }
+
+                logger.info("Job Fetcher: " + nRunning + " running, " + runningJobs.size() + " actual running, "
+                        + nStopped + " stopped, " + nReady + " ready, " + jobPriorityQueue.size() + " waiting, " //
+                        + nSUCCEED + " already succeed, " + nError + " error, " + nDiscarded + " discarded, " + nOthers
+                        + " others");
+            } catch (Exception e) {
+                logger.warn("Job Fetcher caught a exception " + e);
+            }
+        }
+    }
+
     private class FetcherRunner implements Runnable {
 
         @Override
@@ -222,7 +329,7 @@ public class DefaultScheduler implements Scheduler<AbstractExecutable>, Connecti
 
         int pollSecond = jobEngineConfig.getPollIntervalSecond();
         logger.info("Fetching jobs every {} seconds", pollSecond);
-        fetcher = new FetcherRunner();
+        fetcher = jobEngineConfig.getJobPriorityConsidered() ? new FetcherRunnerWithPriority() : new FetcherRunner();
         fetcherPool.scheduleAtFixedRate(fetcher, pollSecond / 10, pollSecond, TimeUnit.SECONDS);
         hasStarted = true;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/12fefdc2/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
index 71b62a0..abf7e02 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
@@ -65,12 +65,18 @@ public class CubingJob extends DefaultChainedExecutable {
     }
 
     public enum CubingJobTypeEnum {
-        BUILD("BUILD"), OPTIMIZE("OPTIMIZE"), MERGE("MERGE");
+        BUILD("BUILD", 20), OPTIMIZE("OPTIMIZE", 5), MERGE("MERGE", 25);
 
         private final String name;
+        private final int defaultPriority;
 
-        CubingJobTypeEnum(String name) {
+        CubingJobTypeEnum(String name, int priority) {
             this.name = name;
+            this.defaultPriority = priority;
+        }
+
+        public int getDefaultPriority() {
+            return defaultPriority;
         }
 
         public String toString() {
@@ -151,6 +157,15 @@ public class CubingJob extends DefaultChainedExecutable {
         super();
     }
 
+    @Override
+    public int getDefaultPriority() {
+        CubingJobTypeEnum jobType = CubingJobTypeEnum.getByName(getJobType());
+        if (jobType == null) {
+            return super.getDefaultPriority();
+        }
+        return jobType.getDefaultPriority();
+    }
+
     protected void setDeployEnvName(String name) {
         setParam(DEPLOY_ENV_NAME, name);
     }


[07/18] kylin git commit: APACHE-KYLIN-2733: Introduce optimize job for adjusting cuboid set

Posted by li...@apache.org.
APACHE-KYLIN-2733: Introduce optimize job for adjusting cuboid set


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/020c4e78
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/020c4e78
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/020c4e78

Branch: refs/heads/ci-dong
Commit: 020c4e785191a4850cac8df19efcb49c80121d36
Parents: fbfbee4
Author: Zhong <nj...@apache.org>
Authored: Wed Aug 30 11:17:43 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/cube/CubeInstance.java     |  12 +-
 .../java/org/apache/kylin/cube/CubeManager.java |  84 ++++++-
 .../java/org/apache/kylin/cube/CubeUpdate.java  |  14 +-
 .../kylin/cube/common/RowKeySplitter.java       |   2 +-
 .../org/apache/kylin/cube/cuboid/Cuboid.java    |   5 +
 .../apache/kylin/cube/cuboid/CuboidUtil.java    |  48 ++++
 .../cube/cuboid/TreeCuboidSchedulerManager.java |   6 +-
 .../cube/inmemcubing/InMemCubeBuilder.java      |   2 +-
 .../org/apache/kylin/cube/kv/RowKeyDecoder.java |   4 +-
 .../org/apache/kylin/engine/EngineFactory.java  |   4 +
 .../apache/kylin/engine/IBatchCubingEngine.java |   3 +
 .../kylin/job/constant/ExecutableConstants.java |   5 +
 .../org/apache/kylin/job/dao/ExecutableDao.java |  18 +-
 .../org/apache/kylin/job/dao/ExecutablePO.java  |  11 +
 .../kylin/job/execution/ExecutableManager.java  |  41 +++-
 .../job/impl/threadpool/DefaultScheduler.java   |   7 +-
 .../engine/mr/BatchOptimizeJobBuilder2.java     | 227 +++++++++++++++++
 .../mr/BatchOptimizeJobCheckpointBuilder.java   |  89 +++++++
 .../org/apache/kylin/engine/mr/CubingJob.java   |   7 +-
 .../org/apache/kylin/engine/mr/IMROutput2.java  |  29 ++-
 .../kylin/engine/mr/JobBuilderSupport.java      |  54 ++++-
 .../kylin/engine/mr/MRBatchCubingEngine.java    |   5 +
 .../kylin/engine/mr/MRBatchCubingEngine2.java   |   5 +
 .../java/org/apache/kylin/engine/mr/MRUtil.java |   4 +
 .../engine/mr/common/AbstractHadoopJob.java     |  24 +-
 .../kylin/engine/mr/common/BatchConstants.java  |   1 +
 .../kylin/engine/mr/common/CubeStatsReader.java | 126 ++++++----
 .../engine/mr/common/CuboidRecommenderUtil.java |  14 +-
 .../engine/mr/common/CuboidSchedulerUtil.java   |  54 +++++
 .../engine/mr/common/CuboidStatsReaderUtil.java |   5 +-
 .../kylin/engine/mr/common/MapReduceUtil.java   | 117 +++++++++
 .../mr/common/StatisticsDecisionUtil.java       |   2 +-
 .../engine/mr/steps/BaseCuboidMapperBase.java   |   2 +-
 .../steps/CalculateStatsFromBaseCuboidJob.java  | 116 +++++++++
 .../CalculateStatsFromBaseCuboidMapper.java     | 201 ++++++++++++++++
 .../CalculateStatsFromBaseCuboidReducer.java    | 112 +++++++++
 .../engine/mr/steps/CopyDictionaryStep.java     |  70 ++++++
 .../engine/mr/steps/CubingExecutableUtil.java   |   9 +
 .../apache/kylin/engine/mr/steps/CuboidJob.java |  28 ++-
 .../mr/steps/FactDistinctColumnsMapper.java     |  37 +--
 .../mr/steps/FactDistinctColumnsReducer.java    |   1 +
 .../mr/steps/FilterRecommendCuboidDataJob.java  | 103 ++++++++
 .../steps/FilterRecommendCuboidDataMapper.java  | 107 ++++++++
 .../mr/steps/InMemCuboidFromBaseCuboidJob.java  | 154 ++++++++++++
 .../steps/InMemCuboidFromBaseCuboidMapper.java  |  96 ++++++++
 .../steps/InMemCuboidFromBaseCuboidReducer.java |  23 ++
 .../kylin/engine/mr/steps/InMemCuboidJob.java   |   4 +-
 .../engine/mr/steps/InMemCuboidMapperBase.java  |  10 +-
 .../kylin/engine/mr/steps/KVGTRecordWriter.java |   2 +-
 .../engine/mr/steps/MergeCuboidMapper.java      |   2 +-
 .../mr/steps/MergeStatisticsWithOldStep.java    | 144 +++++++++++
 .../kylin/engine/mr/steps/NDCuboidMapper.java   |  12 +-
 .../kylin/engine/mr/steps/ReducerNumSizing.java | 106 --------
 .../UpdateCubeInfoAfterCheckpointStep.java      |  69 ++++++
 .../steps/UpdateCubeInfoAfterOptimizeStep.java  |  72 ++++++
 .../mr/steps/UpdateOldCuboidShardJob.java       | 105 ++++++++
 .../mr/steps/UpdateOldCuboidShardMapper.java    | 142 +++++++++++
 .../kylin/engine/spark/SparkCubingByLayer.java  |   2 +-
 .../kylin/rest/controller/CubeController.java   |  63 +++++
 .../kylin/rest/request/JobOptimizeRequest.java  |  34 +++
 .../apache/kylin/rest/service/CubeService.java  |   4 +
 .../apache/kylin/rest/service/JobService.java   | 241 ++++++++++++++++++-
 .../storage/hbase/steps/CreateHTableJob.java    |  19 +-
 .../hbase/steps/HBaseMROutput2Transition.java   |  41 +++-
 .../kylin/storage/hbase/steps/HBaseMRSteps.java |  82 +++++++
 65 files changed, 2987 insertions(+), 255 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
index f6eceb6..462223a 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
@@ -148,6 +148,16 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
         return segments.getMergingSegments(mergedSegment);
     }
 
+    public CubeSegment getOriginalSegmentToOptimize(CubeSegment optimizedSegment) {
+        for (CubeSegment segment : this.getSegments(SegmentStatusEnum.READY)) {
+            if (!optimizedSegment.equals(segment) //
+                    && optimizedSegment.getSegRange().equals(segment.getSegRange())) {
+                return segment;
+            }
+        }
+        return null;
+    }
+
     public CubeDesc getDescriptor() {
         return CubeDescManager.getInstance(config).getCubeDesc(descName);
     }
@@ -353,7 +363,7 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
             return cuboidsRecommend;
         case RECOMMEND_MISSING_WITH_BASE:
             cuboidsRecommend.removeAll(currentCuboids);
-            currentCuboids.add(getCuboidScheduler().getBaseCuboidId());
+            cuboidsRecommend.add(getCuboidScheduler().getBaseCuboidId());
             return cuboidsRecommend;
         default:
             return null;

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
index 388c840..5e72721 100755
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -30,6 +30,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -76,6 +77,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
 /**
  * @author yangli9
@@ -406,6 +408,10 @@ public class CubeManager implements IRealizationProvider {
             cube.setCuboids(update.getCuboids());
         }
 
+        if (update.getCuboidsRecommend() != null) {
+            cube.setCuboidsRecommend(update.getCuboidsRecommend());
+        }
+
         try {
             getStore().putResource(cube.getResourcePath(), cube, CUBE_SERIALIZER);
         } catch (IllegalStateException ise) {
@@ -483,8 +489,7 @@ public class CubeManager implements IRealizationProvider {
         return newSegment;
     }
 
-    public CubeSegment refreshSegment(CubeInstance cube, TSRange tsRange, SegmentRange segRange)
-            throws IOException {
+    public CubeSegment refreshSegment(CubeInstance cube, TSRange tsRange, SegmentRange segRange) throws IOException {
         checkInputRanges(tsRange, segRange);
         checkBuildingSegment(cube);
 
@@ -519,6 +524,27 @@ public class CubeManager implements IRealizationProvider {
         return newSegment;
     }
 
+    public CubeSegment[] optimizeSegments(CubeInstance cube, Set<Long> cuboidsRecommend) throws IOException {
+        checkReadyForOptimize(cube);
+
+        List<CubeSegment> readySegments = cube.getSegments(SegmentStatusEnum.READY);
+        CubeSegment[] optimizeSegments = new CubeSegment[readySegments.size()];
+        int i = 0;
+        for (CubeSegment segment : readySegments) {
+            CubeSegment newSegment = newSegment(cube, segment.getTSRange(), null);
+            validateNewSegments(cube, newSegment);
+
+            optimizeSegments[i++] = newSegment;
+        }
+
+        CubeUpdate cubeBuilder = new CubeUpdate(cube);
+        cubeBuilder.setCuboidsRecommend(cuboidsRecommend);
+        cubeBuilder.setToAddSegs(optimizeSegments);
+        updateCube(cubeBuilder);
+
+        return optimizeSegments;
+    }
+
     public CubeSegment mergeSegments(CubeInstance cube, TSRange tsRange, SegmentRange segRange, boolean force)
             throws IOException {
         if (cube.getSegments().isEmpty())
@@ -594,8 +620,15 @@ public class CubeManager implements IRealizationProvider {
     }
 
     private void checkBuildingSegment(CubeInstance cube) {
-        int maxBuldingSeg = cube.getConfig().getMaxBuildingSegments();
-        if (cube.getBuildingSegments().size() >= maxBuldingSeg) {
+        checkBuildingSegment(cube, cube.getConfig().getMaxBuildingSegments());
+    }
+
+    public void checkReadyForOptimize(CubeInstance cube) {
+        checkBuildingSegment(cube, 1);
+    }
+
+    private void checkBuildingSegment(CubeInstance cube, int maxBuildingSeg) {
+        if (cube.getBuildingSegments().size() >= maxBuildingSeg) {
             throw new IllegalStateException(
                     "There is already " + cube.getBuildingSegments().size() + " building segment; ");
         }
@@ -725,6 +758,49 @@ public class CubeManager implements IRealizationProvider {
         updateCube(cubeBuilder);
     }
 
+    public void promoteNewlyOptimizeSegments(CubeInstance cube, CubeSegment... optimizedSegments) throws IOException {
+        for (CubeSegment seg : optimizedSegments) {
+            seg.setStatus(SegmentStatusEnum.READY_PENDING);
+        }
+
+        CubeUpdate cubeBuilder = new CubeUpdate(cube);
+        cubeBuilder.setToUpdateSegs(optimizedSegments);
+        updateCube(cubeBuilder);
+    }
+
+    public void promoteCheckpointOptimizeSegments(CubeInstance cube, Map<Long, Long> recommendCuboids,
+            CubeSegment... optimizedSegments) throws IOException {
+        if (cube.getSegments().size() != optimizedSegments.length * 2) {
+            throw new IllegalStateException("For cube " + cube
+                    + ", every READY segment should be optimized and all segments should be READY before optimizing");
+        }
+        CubeSegment[] originalSegments = new CubeSegment[optimizedSegments.length];
+        int i = 0;
+        for (CubeSegment seg : optimizedSegments) {
+            originalSegments[i++] = cube.getOriginalSegmentToOptimize(seg);
+
+            if (StringUtils.isBlank(seg.getStorageLocationIdentifier()))
+                throw new IllegalStateException(
+                        "For cube " + cube + ", segment " + seg + " missing StorageLocationIdentifier");
+
+            if (StringUtils.isBlank(seg.getLastBuildJobID()))
+                throw new IllegalStateException("For cube " + cube + ", segment " + seg + " missing LastBuildJobID");
+
+            seg.setStatus(SegmentStatusEnum.READY);
+        }
+
+        logger.info("Promoting cube " + cube + ", new segments " + Arrays.toString(optimizedSegments)
+                + ", to remove segments " + originalSegments);
+
+        CubeUpdate cubeBuilder = new CubeUpdate(cube);
+        cubeBuilder.setToRemoveSegs(originalSegments) //
+                .setToUpdateSegs(optimizedSegments) //
+                .setStatus(RealizationStatusEnum.READY) //
+                .setCuboids(recommendCuboids) //
+                .setCuboidsRecommend(Sets.<Long> newHashSet());
+        updateCube(cubeBuilder);
+    }
+
     public void validateNewSegments(CubeInstance cube, CubeSegment newSegments) {
         List<CubeSegment> tobe = cube.calculateToBeSegments(newSegments);
         List<CubeSegment> newList = Arrays.asList(newSegments);

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-cube/src/main/java/org/apache/kylin/cube/CubeUpdate.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeUpdate.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeUpdate.java
index fae20dc..2e1d652 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeUpdate.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeUpdate.java
@@ -19,6 +19,7 @@
 package org.apache.kylin.cube;
 
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 
@@ -34,6 +35,7 @@ public class CubeUpdate {
     private String owner;
     private int cost = -1;
     private Map<Long, Long> cuboids = null;
+    private Set<Long> cuboidsRecommend = null;
 
     public CubeUpdate(CubeInstance cubeInstance) {
         this.cubeInstance = cubeInstance;
@@ -106,7 +108,17 @@ public class CubeUpdate {
         return cuboids;
     }
 
-    public void setCuboids(Map<Long, Long> cuboids) {
+    public CubeUpdate setCuboids(Map<Long, Long> cuboids) {
         this.cuboids = cuboids;
+        return this;
+    }
+
+    public Set<Long> getCuboidsRecommend() {
+        return cuboidsRecommend;
+    }
+
+    public CubeUpdate setCuboidsRecommend(Set<Long> cuboidsRecommend) {
+        this.cuboidsRecommend = cuboidsRecommend;
+        return this;
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-cube/src/main/java/org/apache/kylin/cube/common/RowKeySplitter.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/common/RowKeySplitter.java b/core-cube/src/main/java/org/apache/kylin/cube/common/RowKeySplitter.java
index cd26347..0c54ecf 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/common/RowKeySplitter.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/common/RowKeySplitter.java
@@ -115,7 +115,7 @@ public class RowKeySplitter implements java.io.Serializable {
         offset += RowConstants.ROWKEY_CUBOIDID_LEN;
 
         long lastSplittedCuboidId = Bytes.toLong(cuboidIdSplit.value, 0, cuboidIdSplit.length);
-        Cuboid cuboid = Cuboid.findById(cubeSegment, lastSplittedCuboidId);
+        Cuboid cuboid = Cuboid.findForMandatory(cubeDesc, lastSplittedCuboidId);
 
         // rowkey columns
         for (int i = 0; i < cuboid.getColumns().size(); i++) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
index efd2e2e..3c4fceb 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
@@ -88,6 +88,11 @@ public class Cuboid implements Comparable<Cuboid>, Serializable {
         return cuboidID;
     }
 
+    // for mandatory cuboid, no need to translate cuboid
+    public static Cuboid findForMandatory(CubeDesc cube, long cuboidID) {
+        return new Cuboid(cube, cuboidID, cuboidID);
+    }
+
     public static Cuboid findById(CuboidScheduler cuboidScheduler, byte[] cuboidID) {
         return findById(cuboidScheduler, Bytes.toLong(cuboidID));
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidUtil.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidUtil.java b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidUtil.java
new file mode 100644
index 0000000..a84f153
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidUtil.java
@@ -0,0 +1,48 @@
+/*
+ * 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.cube.cuboid;
+
+import com.google.common.base.Preconditions;
+
+public class CuboidUtil {
+
+    public static Integer[][] getCuboidBitSet(Long[] cuboidIds, int nRowKey) {
+        Preconditions.checkArgument(nRowKey < Long.SIZE,
+                "the size of row key could not be large than " + (Long.SIZE - 1));
+
+        Integer[][] allCuboidsBitSet = new Integer[cuboidIds.length][];
+
+        for (int j = 0; j < cuboidIds.length; j++) {
+            Long cuboidId = cuboidIds[j];
+
+            allCuboidsBitSet[j] = new Integer[Long.bitCount(cuboidId)];
+
+            long mask = 1L << (nRowKey - 1);
+            int position = 0;
+            for (int i = 0; i < nRowKey; i++) {
+                if ((mask & cuboidId) > 0) {
+                    allCuboidsBitSet[j][position] = i;
+                    position++;
+                }
+                mask = mask >> 1;
+            }
+        }
+        return allCuboidsBitSet;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java
index 5e8d965..22e636b 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java
@@ -64,7 +64,7 @@ public class TreeCuboidSchedulerManager {
      * @param cubeName
      * @return null if the cube has no pre-built cuboids
      */
-    public static TreeCuboidScheduler getTreeCuboidScheduler(String cubeName) {
+    public TreeCuboidScheduler getTreeCuboidScheduler(String cubeName) {
         TreeCuboidScheduler result = cache.get(cubeName);
         if (result == null) {
             CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
@@ -83,14 +83,14 @@ public class TreeCuboidSchedulerManager {
         return result;
     }
 
-    public static TreeCuboidScheduler getTreeCuboidScheduler(CubeDesc cubeDesc, Map<Long, Long> cuboidsWithRowCnt) {
+    public TreeCuboidScheduler getTreeCuboidScheduler(CubeDesc cubeDesc, Map<Long, Long> cuboidsWithRowCnt) {
         if (cuboidsWithRowCnt == null || cuboidsWithRowCnt.isEmpty()) {
             return null;
         }
         return getTreeCuboidScheduler(cubeDesc, Lists.newArrayList(cuboidsWithRowCnt.keySet()), cuboidsWithRowCnt);
     }
 
-    public static TreeCuboidScheduler getTreeCuboidScheduler(CubeDesc cubeDesc, List<Long> cuboidIds,
+    public TreeCuboidScheduler getTreeCuboidScheduler(CubeDesc cubeDesc, List<Long> cuboidIds,
             Map<Long, Long> cuboidsWithRowCnt) {
         if (cuboidIds == null || cuboidsWithRowCnt == null) {
             return null;

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
index f63b53f..97bb1de 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
@@ -108,7 +108,7 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
     }
 
     private GridTable newGridTableByCuboidID(long cuboidID) throws IOException {
-        GTInfo info = CubeGridTable.newGTInfo(Cuboid.findById(cuboidScheduler, cuboidID),
+        GTInfo info = CubeGridTable.newGTInfo(Cuboid.findForMandatory(cubeDesc, cuboidID),
                 new CubeDimEncMap(cubeDesc, dictionaryMap)
         );
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyDecoder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyDecoder.java b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyDecoder.java
index 5a1f668..bb03c4c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyDecoder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyDecoder.java
@@ -36,7 +36,6 @@ import org.apache.kylin.metadata.model.TblColRef;
  */
 public class RowKeyDecoder {
 
-    private final CubeSegment cubeSegment;
     private final CubeDesc cubeDesc;
     private final RowKeyColumnIO colIO;
     private final RowKeySplitter rowKeySplitter;
@@ -45,7 +44,6 @@ public class RowKeyDecoder {
     private List<String> values;
 
     public RowKeyDecoder(CubeSegment cubeSegment) {
-        this.cubeSegment = cubeSegment;
         this.cubeDesc = cubeSegment.getCubeDesc();
         this.rowKeySplitter = new RowKeySplitter(cubeSegment, 65, 255);
         this.colIO = new RowKeyColumnIO(cubeSegment.getDimensionEncodingMap());
@@ -75,7 +73,7 @@ public class RowKeyDecoder {
         if (this.cuboid != null && this.cuboid.getId() == cuboidID) {
             return;
         }
-        this.cuboid = Cuboid.findById(cubeSegment, cuboidID);
+        this.cuboid = Cuboid.findForMandatory(cubeDesc, cuboidID);
     }
 
     private void collectValue(TblColRef col, byte[] valueBytes, int length) throws IOException {

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-job/src/main/java/org/apache/kylin/engine/EngineFactory.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/engine/EngineFactory.java b/core-job/src/main/java/org/apache/kylin/engine/EngineFactory.java
index 78b1efe..03d986b 100644
--- a/core-job/src/main/java/org/apache/kylin/engine/EngineFactory.java
+++ b/core-job/src/main/java/org/apache/kylin/engine/EngineFactory.java
@@ -61,4 +61,8 @@ public class EngineFactory {
         return batchEngine(mergeSegment).createBatchMergeJob(mergeSegment, submitter);
     }
 
+    /** Optimize a segment based on the cuboid recommend list produced by the cube planner. */
+    public static DefaultChainedExecutable createBatchOptimizeJob(CubeSegment optimizeSegment, String submitter) {
+        return batchEngine(optimizeSegment).createBatchOptimizeJob(optimizeSegment, submitter);
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java b/core-job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java
index 754dbde..a618eac 100644
--- a/core-job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java
+++ b/core-job/src/main/java/org/apache/kylin/engine/IBatchCubingEngine.java
@@ -37,6 +37,9 @@ public interface IBatchCubingEngine {
     /** Merge multiple small segments into a big one. */
     public DefaultChainedExecutable createBatchMergeJob(CubeSegment mergeSegment, String submitter);
 
+    /** Optimize a segment based on the cuboid recommend list produced by the cube planner. */
+    public DefaultChainedExecutable createBatchOptimizeJob(CubeSegment optimizeSegment, String submitter);
+
     public Class<?> getSourceInterface();
 
     public Class<?> getStorageInterface();

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java b/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
index 36496fe..f3caf3b 100644
--- a/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
+++ b/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
@@ -37,6 +37,9 @@ public final class ExecutableConstants {
     public static final String STEP_NAME_CREATE_FLAT_HIVE_TABLE = "Create Intermediate Flat Hive Table";
     public static final String STEP_NAME_MATERIALIZE_HIVE_VIEW_IN_LOOKUP = "Materialize Hive View in Lookup Tables";
     public static final String STEP_NAME_FACT_DISTINCT_COLUMNS = "Extract Fact Table Distinct Columns";
+    public static final String STEP_NAME_CALCULATE_STATS_FROM_BASE_CUBOID = "Calculate Statistics from Base Cuboid";
+    public static final String STEP_NAME_FILTER_RECOMMEND_CUBOID_DATA_FOR_OPTIMIZATION = "Filter Recommend Cuboid Data for Optimization";
+    public static final String STEP_NAME_UPDATE_OLD_CUBOID_SHARD = "Update Old Cuboid Shard for Optimization";
     public static final String STEP_NAME_BUILD_BASE_CUBOID = "Build Base Cuboid";
     public static final String STEP_NAME_BUILD_IN_MEM_CUBE = "Build Cube In-Mem";
     public static final String STEP_NAME_BUILD_SPARK_CUBE = "Build Cube with Spark";
@@ -45,8 +48,10 @@ public final class ExecutableConstants {
     public static final String STEP_NAME_CREATE_HBASE_TABLE = "Create HTable";
     public static final String STEP_NAME_CONVERT_CUBOID_TO_HFILE = "Convert Cuboid Data to HFile";
     public static final String STEP_NAME_BULK_LOAD_HFILE = "Load HFile to HBase Table";
+    public static final String STEP_NAME_COPY_DICTIONARY = "Copy dictionary from Old Segment";
     public static final String STEP_NAME_MERGE_DICTIONARY = "Merge Cuboid Dictionary";
     public static final String STEP_NAME_MERGE_STATISTICS = "Merge Cuboid Statistics";
+    public static final String STEP_NAME_MERGE_STATISTICS_WITH_OLD = "Merge Cuboid Statistics with Old for Optimization";
     public static final String STEP_NAME_SAVE_STATISTICS = "Save Cuboid Statistics";
     public static final String STEP_NAME_MERGE_CUBOID = "Merge Cuboid Data";
     public static final String STEP_NAME_UPDATE_CUBE_INFO = "Update Cube Info";

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
index 16875b1..9cfc61a 100644
--- a/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
+++ b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
@@ -92,8 +92,8 @@ public class ExecutableDao {
         return store.getResource(path, ExecutablePO.class, JOB_SERIALIZER);
     }
 
-    private void writeJobResource(String path, ExecutablePO job) throws IOException {
-        store.putResource(path, job, JOB_SERIALIZER);
+    private long writeJobResource(String path, ExecutablePO job) throws IOException {
+        return store.putResource(path, job, JOB_SERIALIZER);
     }
 
     private ExecutableOutputPO readJobOutputResource(String path) throws IOException {
@@ -179,6 +179,20 @@ public class ExecutableDao {
         }
     }
 
+    public ExecutablePO updateJob(ExecutablePO job) throws PersistentException {
+        try {
+            if (getJob(job.getUuid()) == null) {
+                throw new IllegalArgumentException("job id:" + job.getUuid() + " does not exist");
+            }
+            final long ts = writeJobResource(pathOfJob(job), job);
+            job.setLastModified(ts);
+            return job;
+        } catch (IOException e) {
+            logger.error("error update job:" + job.getUuid(), e);
+            throw new PersistentException(e);
+        }
+    }
+
     public void deleteJob(String uuid) throws PersistentException {
         try {
             store.deleteResource(pathOfJob(uuid));

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java
index 75717e0..f48c876 100644
--- a/core-job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java
+++ b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutablePO.java
@@ -39,6 +39,9 @@ public class ExecutablePO extends RootPersistentEntity {
     @JsonProperty("tasks")
     private List<ExecutablePO> tasks;
 
+    @JsonProperty("tasks_check")
+    private List<ExecutablePO> tasksForCheck;
+
     @JsonProperty("type")
     private String type;
 
@@ -61,6 +64,14 @@ public class ExecutablePO extends RootPersistentEntity {
         this.tasks = tasks;
     }
 
+    public List<ExecutablePO> getTasksForCheck() {
+        return tasksForCheck;
+    }
+
+    public void setTasksForCheck(List<ExecutablePO> tasksForCheck) {
+        this.tasksForCheck = tasksForCheck;
+    }
+
     public String getType() {
         return type;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java b/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java
index bab8c30..bc38fff 100644
--- a/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java
@@ -96,6 +96,13 @@ public class ExecutableManager {
             }
             result.setTasks(tasks);
         }
+        if (executable instanceof CheckpointExecutable) {
+            List<ExecutablePO> tasksForCheck = Lists.newArrayList();
+            for (AbstractExecutable taskForCheck : ((CheckpointExecutable) executable).getSubTasksForCheck()) {
+                tasksForCheck.add(parse(taskForCheck));
+            }
+            result.setTasksForCheck(tasksForCheck);
+        }
         return result;
     }
 
@@ -121,6 +128,23 @@ public class ExecutableManager {
         }
     }
 
+    public void updateCheckpointJob(String jobId, List<AbstractExecutable> subTasksForCheck) {
+        try {
+            final ExecutablePO job = executableDao.getJob(jobId);
+            Preconditions.checkArgument(job != null, "there is no related job for job id:" + jobId);
+
+            List<ExecutablePO> tasksForCheck = Lists.newArrayListWithExpectedSize(subTasksForCheck.size());
+            for (AbstractExecutable taskForCheck : subTasksForCheck) {
+                tasksForCheck.add(parse(taskForCheck));
+            }
+            job.setTasksForCheck(tasksForCheck);
+            executableDao.updateJob(job);
+        } catch (PersistentException e) {
+            logger.error("fail to update checkpoint job:" + jobId, e);
+            throw new RuntimeException(e);
+        }
+    }
+
     //for ut
     public void deleteJob(String jobId) {
         try {
@@ -349,7 +373,15 @@ public class ExecutableManager {
         if (job == null) {
             return;
         }
-
+        if (job.getStatus().isFinalState()) {
+            if (job.getStatus() != ExecutableState.DISCARDED) {
+                logger.warn("The status of job " + jobId + " is " + job.getStatus().toString()
+                        + ". It's final state and cannot be transfer to be discarded!!!");
+            } else {
+                logger.warn("The job " + jobId + " has been discarded.");
+            }
+            return;
+        }
         if (job instanceof DefaultChainedExecutable) {
             List<AbstractExecutable> tasks = ((DefaultChainedExecutable) job).getTasks();
             for (AbstractExecutable task : tasks) {
@@ -501,6 +533,13 @@ public class ExecutableManager {
                     ((ChainedExecutable) result).addTask(parseTo(subTask));
                 }
             }
+            List<ExecutablePO> tasksForCheck = executablePO.getTasksForCheck();
+            if (tasksForCheck != null && !tasksForCheck.isEmpty()) {
+                Preconditions.checkArgument(result instanceof CheckpointExecutable);
+                for (ExecutablePO subTaskForCheck : tasksForCheck) {
+                    ((CheckpointExecutable) result).addTaskForCheck(parseTo(subTaskForCheck));
+                }
+            }
             return result;
         } catch (ReflectiveOperationException e) {
             throw new IllegalStateException("cannot parse this job:" + executablePO.getId(), e);

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
index 64a7db7..6ef9c81 100644
--- a/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
+++ b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
@@ -88,8 +88,9 @@ public class DefaultScheduler implements Scheduler<AbstractExecutable>, Connecti
                         nRunning++;
                         continue;
                     }
-                    final Output output = executableManager.getOutput(id);
-                    if ((output.getState() != ExecutableState.READY)) {
+                    final AbstractExecutable executable = executableManager.getJob(id);
+                    if (!executable.isReady()) {
+                        final Output output = executableManager.getOutput(id);
                         // logger.debug("Job id:" + id + " not runnable");
                         if (output.getState() == ExecutableState.DISCARDED) {
                             nDiscarded++;
@@ -105,10 +106,8 @@ public class DefaultScheduler implements Scheduler<AbstractExecutable>, Connecti
                         continue;
                     }
                     nReady++;
-                    AbstractExecutable executable = null;
                     String jobDesc = null;
                     try {
-                        executable = executableManager.getJob(id);
                         jobDesc = executable.toString();
                         logger.info(jobDesc + " prepare to schedule");
                         context.addRunningJob(executable);

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobBuilder2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobBuilder2.java
new file mode 100644
index 0000000..a8127cc
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobBuilder2.java
@@ -0,0 +1,227 @@
+/*
+ * 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.mr;
+
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidModeEnum;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+import org.apache.kylin.engine.mr.steps.CopyDictionaryStep;
+import org.apache.kylin.engine.mr.steps.CubingExecutableUtil;
+import org.apache.kylin.engine.mr.steps.FilterRecommendCuboidDataJob;
+import org.apache.kylin.engine.mr.steps.InMemCuboidFromBaseCuboidJob;
+import org.apache.kylin.engine.mr.steps.MergeStatisticsWithOldStep;
+import org.apache.kylin.engine.mr.steps.NDCuboidJob;
+import org.apache.kylin.engine.mr.steps.UpdateCubeInfoAfterOptimizeStep;
+import org.apache.kylin.engine.mr.steps.UpdateOldCuboidShardJob;
+import org.apache.kylin.job.constant.ExecutableConstants;
+import org.apache.kylin.job.engine.JobEngineConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+public class BatchOptimizeJobBuilder2 extends JobBuilderSupport {
+    private static final Logger logger = LoggerFactory.getLogger(BatchOptimizeJobBuilder2.class);
+
+    private final IMROutput2.IMRBatchOptimizeOutputSide2 outputSide;
+
+    public BatchOptimizeJobBuilder2(CubeSegment optimizeSegment, String submitter) {
+        super(optimizeSegment, submitter);
+        this.outputSide = MRUtil.getBatchOptimizeOutputSide2(optimizeSegment);
+    }
+
+    public CubingJob build() {
+        logger.info("MR_V2 new job to OPTIMIZE a segment " + seg);
+
+        final CubingJob result = CubingJob.createOptimizeJob(seg, submitter, config);
+        CubingExecutableUtil.setCubeName(seg.getRealization().getName(), result.getParams());
+
+        final String jobId = result.getId();
+        final String cuboidRootPath = getCuboidRootPath(jobId);
+        final String optimizeCuboidRootPath = getOptimizationCuboidPath(jobId);
+
+        CubeSegment oldSegment = seg.getCubeInstance().getOriginalSegmentToOptimize(seg);
+        Preconditions.checkNotNull(oldSegment, "cannot find the original segment to be optimized by " + seg);
+
+        // Phase 1: Prepare base cuboid data from old segment
+        String oldcuboidRootPath = getCuboidRootPath(oldSegment) + "*";
+        result.addTask(createFilterRecommendCuboidDataStep(oldcuboidRootPath, optimizeCuboidRootPath));
+
+        // Phase 2: Prepare dictionary and statistics for new segment
+        result.addTask(createCopyDictionaryStep());
+        String optStatsSourcePath = getBaseCuboidPath(optimizeCuboidRootPath);
+        String optStatsDstPath = getOptimizationStatisticsPath(jobId);
+        result.addTask(createCalculateStatsFromBaseCuboid(optStatsSourcePath, optStatsDstPath,
+                CuboidModeEnum.RECOMMEND_MISSING));
+        result.addTask(createMergeStatisticsWithOldStep(jobId, optStatsDstPath, getStatisticsPath(jobId)));
+        outputSide.addStepPhase2_CreateHTable(result);
+
+        result.addTask(createUpdateShardForOldCuboidDataStep(optimizeCuboidRootPath + "*", cuboidRootPath));
+
+        // Phase 3: Build Cube for Missing Cuboid Data
+        addLayerCubingSteps(result, jobId, CuboidModeEnum.RECOMMEND_MISSING_WITH_BASE, cuboidRootPath); // layer cubing
+        result.addTask(createInMemCubingStep(jobId, CuboidModeEnum.RECOMMEND_MISSING_WITH_BASE, cuboidRootPath));
+
+        outputSide.addStepPhase3_BuildCube(result);
+
+        // Phase 4: Update Metadata & Cleanup
+        result.addTask(createUpdateCubeInfoAfterOptimizeStep(jobId));
+        outputSide.addStepPhase4_Cleanup(result);
+
+        return result;
+    }
+
+    public MapReduceExecutable createFilterRecommendCuboidDataStep(String inputPath, String outputPath) {
+        MapReduceExecutable result = new MapReduceExecutable();
+        result.setName(ExecutableConstants.STEP_NAME_FILTER_RECOMMEND_CUBOID_DATA_FOR_OPTIMIZATION);
+
+        StringBuilder cmd = new StringBuilder();
+        appendMapReduceParameters(cmd);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, seg.getRealization().getName());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, inputPath);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, outputPath);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME,
+                "Kylin_Filter_Recommend_Cuboid_Data_" + seg.getRealization().getName());
+
+        result.setMapReduceParams(cmd.toString());
+        result.setMapReduceJobClass(FilterRecommendCuboidDataJob.class);
+        return result;
+    }
+
+    public CopyDictionaryStep createCopyDictionaryStep() {
+        CopyDictionaryStep result = new CopyDictionaryStep();
+        result.setName(ExecutableConstants.STEP_NAME_COPY_DICTIONARY);
+
+        CubingExecutableUtil.setCubeName(seg.getRealization().getName(), result.getParams());
+        CubingExecutableUtil.setSegmentId(seg.getUuid(), result.getParams());
+        return result;
+    }
+
+    private MapReduceExecutable createUpdateShardForOldCuboidDataStep(String inputPath, String outputPath) {
+        MapReduceExecutable result = new MapReduceExecutable();
+        result.setName(ExecutableConstants.STEP_NAME_UPDATE_OLD_CUBOID_SHARD);
+
+        StringBuilder cmd = new StringBuilder();
+        appendMapReduceParameters(cmd);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, seg.getRealization().getName());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, inputPath);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, outputPath);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME,
+                "Kylin_Update_Old_Cuboid_Shard_" + seg.getRealization().getName());
+
+        result.setMapReduceParams(cmd.toString());
+        result.setMapReduceJobClass(UpdateOldCuboidShardJob.class);
+        return result;
+    }
+
+    private MergeStatisticsWithOldStep createMergeStatisticsWithOldStep(final String jobId, final String optStatsPath,
+            final String mergedStatisticsFolder) {
+        MergeStatisticsWithOldStep result = new MergeStatisticsWithOldStep();
+        result.setName(ExecutableConstants.STEP_NAME_MERGE_STATISTICS_WITH_OLD);
+
+        CubingExecutableUtil.setCubingJobId(jobId, result.getParams());
+        CubingExecutableUtil.setCubeName(seg.getRealization().getName(), result.getParams());
+        CubingExecutableUtil.setSegmentId(seg.getUuid(), result.getParams());
+        CubingExecutableUtil.setStatisticsPath(optStatsPath, result.getParams());
+        CubingExecutableUtil.setMergedStatisticsPath(mergedStatisticsFolder, result.getParams());
+
+        return result;
+    }
+
+    private void addLayerCubingSteps(final CubingJob result, final String jobId, final CuboidModeEnum cuboidMode,
+            final String cuboidRootPath) {
+        // Don't know statistics so that tree cuboid scheduler is not determined. Determine the maxLevel at runtime
+        final int maxLevel = seg.getCubeDesc().getRowkey().getRowKeyColumns().length;
+        // Don't need to build base cuboid
+        // n dim cuboid steps
+        for (int i = 1; i <= maxLevel; i++) {
+            String parentCuboidPath = i == 1 ? getBaseCuboidPath(cuboidRootPath)
+                    : getCuboidOutputPathsByLevel(cuboidRootPath, i - 1);
+            result.addTask(createNDimensionCuboidStep(parentCuboidPath,
+                    getCuboidOutputPathsByLevel(cuboidRootPath, i), i, jobId, cuboidMode));
+        }
+    }
+
+    private MapReduceExecutable createNDimensionCuboidStep(String parentPath, String outputPath, int level,
+            String jobId, CuboidModeEnum cuboidMode) {
+        // ND cuboid job
+        MapReduceExecutable ndCuboidStep = new MapReduceExecutable();
+
+        ndCuboidStep.setName(ExecutableConstants.STEP_NAME_BUILD_N_D_CUBOID + " : level " + level);
+        StringBuilder cmd = new StringBuilder();
+
+        appendMapReduceParameters(cmd);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, seg.getRealization().getName());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, parentPath);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, outputPath);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME,
+                "Kylin_ND-Cuboid_Builder_" + seg.getRealization().getName() + "_Step");
+        appendExecCmdParameters(cmd, BatchConstants.ARG_LEVEL, "" + level);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBING_JOB_ID, jobId);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBOID_MODE, cuboidMode.toString());
+
+        ndCuboidStep.setMapReduceParams(cmd.toString());
+        ndCuboidStep.setMapReduceJobClass(getNDCuboidJob());
+        return ndCuboidStep;
+    }
+
+    private MapReduceExecutable createInMemCubingStep(String jobId, CuboidModeEnum cuboidMode, String cuboidRootPath) {
+        MapReduceExecutable cubeStep = new MapReduceExecutable();
+
+        StringBuilder cmd = new StringBuilder();
+        appendMapReduceParameters(cmd, JobEngineConfig.IN_MEM_JOB_CONF_SUFFIX);
+
+        cubeStep.setName(ExecutableConstants.STEP_NAME_BUILD_IN_MEM_CUBE);
+
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, seg.getRealization().getName());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, getBaseCuboidPath(cuboidRootPath));
+        appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, getInMemCuboidPath(cuboidRootPath));
+        appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME,
+                "Kylin_Cube_Builder_" + seg.getRealization().getName());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBING_JOB_ID, jobId);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBOID_MODE, cuboidMode.toString());
+
+        cubeStep.setMapReduceParams(cmd.toString());
+        cubeStep.setMapReduceJobClass(InMemCuboidFromBaseCuboidJob.class);
+        cubeStep.setCounterSaveAs(
+                CubingJob.SOURCE_RECORD_COUNT + "," + CubingJob.SOURCE_SIZE_BYTES + "," + CubingJob.CUBE_SIZE_BYTES);
+        return cubeStep;
+    }
+
+    public UpdateCubeInfoAfterOptimizeStep createUpdateCubeInfoAfterOptimizeStep(String jobId) {
+        final UpdateCubeInfoAfterOptimizeStep result = new UpdateCubeInfoAfterOptimizeStep();
+        result.setName(ExecutableConstants.STEP_NAME_UPDATE_CUBE_INFO);
+
+        CubingExecutableUtil.setCubeName(seg.getRealization().getName(), result.getParams());
+        CubingExecutableUtil.setSegmentId(seg.getUuid(), result.getParams());
+        CubingExecutableUtil.setCubingJobId(jobId, result.getParams());
+
+        return result;
+    }
+
+    protected Class<? extends AbstractHadoopJob> getNDCuboidJob() {
+        return NDCuboidJob.class;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobCheckpointBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobCheckpointBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobCheckpointBuilder.java
new file mode 100644
index 0000000..1a779d2
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobCheckpointBuilder.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.engine.mr;
+
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.engine.mr.steps.CubingExecutableUtil;
+import org.apache.kylin.engine.mr.steps.UpdateCubeInfoAfterCheckpointStep;
+import org.apache.kylin.job.constant.ExecutableConstants;
+import org.apache.kylin.job.execution.CheckpointExecutable;
+import org.apache.kylin.metadata.project.ProjectInstance;
+import org.apache.kylin.metadata.project.ProjectManager;
+
+import com.google.common.base.Preconditions;
+
+public class BatchOptimizeJobCheckpointBuilder {
+
+    protected static SimpleDateFormat format = new SimpleDateFormat("z yyyy-MM-dd HH:mm:ss");
+
+    final protected CubeInstance cube;
+    final protected String submitter;
+
+    private final IMROutput2.IMRBatchOptimizeOutputSide2 outputSide;
+
+    public BatchOptimizeJobCheckpointBuilder(CubeInstance cube, String submitter) {
+        this.cube = cube;
+        this.submitter = submitter;
+
+        Preconditions.checkNotNull(cube.getFirstSegment(), "Cube " + cube + " is empty!!!");
+        this.outputSide = MRUtil.getBatchOptimizeOutputSide2(cube.getFirstSegment());
+    }
+
+    public CheckpointExecutable build() {
+        KylinConfig kylinConfig = cube.getConfig();
+        List<ProjectInstance> projList = ProjectManager.getInstance(kylinConfig).findProjects(cube.getType(),
+                cube.getName());
+        if (projList == null || projList.size() == 0) {
+            throw new RuntimeException("Cannot find the project containing the cube " + cube.getName() + "!!!");
+        } else if (projList.size() >= 2) {
+            throw new RuntimeException("Find more than one project containing the cube " + cube.getName()
+                    + ". It does't meet the uniqueness requirement!!! ");
+        }
+
+        CheckpointExecutable checkpointJob = new CheckpointExecutable();
+        checkpointJob.setSubmitter(submitter);
+        CubingExecutableUtil.setCubeName(cube.getName(), checkpointJob.getParams());
+        checkpointJob.setName(
+                cube.getName() + " - OPTIMIZE CHECKPOINT - " + format.format(new Date(System.currentTimeMillis())));
+        checkpointJob.setDeployEnvName(kylinConfig.getDeployEnv());
+        checkpointJob.setProjectName(projList.get(0).getName());
+
+        // Phase 1: Update cube information
+        checkpointJob.addTask(createUpdateCubeInfoAfterCheckpointStep());
+
+        // Phase 2: Garbage collection
+        outputSide.addStepPhase5_Cleanup(checkpointJob);
+
+        return checkpointJob;
+    }
+
+    private UpdateCubeInfoAfterCheckpointStep createUpdateCubeInfoAfterCheckpointStep() {
+        UpdateCubeInfoAfterCheckpointStep result = new UpdateCubeInfoAfterCheckpointStep();
+        result.setName(ExecutableConstants.STEP_NAME_UPDATE_CUBE_INFO);
+
+        CubingExecutableUtil.setCubeName(cube.getName(), result.getParams());
+        return result;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
index 4e7bcdd..71b62a0 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
@@ -65,7 +65,7 @@ public class CubingJob extends DefaultChainedExecutable {
     }
 
     public enum CubingJobTypeEnum {
-        BUILD("BUILD"), MERGE("MERGE");
+        BUILD("BUILD"), OPTIMIZE("OPTIMIZE"), MERGE("MERGE");
 
         private final String name;
 
@@ -106,6 +106,10 @@ public class CubingJob extends DefaultChainedExecutable {
         return initCubingJob(seg, CubingJobTypeEnum.BUILD.toString(), submitter, config);
     }
 
+    public static CubingJob createOptimizeJob(CubeSegment seg, String submitter, JobEngineConfig config) {
+        return initCubingJob(seg, CubingJobTypeEnum.OPTIMIZE.toString(), submitter, config);
+    }
+
     public static CubingJob createMergeJob(CubeSegment seg, String submitter, JobEngineConfig config) {
         return initCubingJob(seg, CubingJobTypeEnum.MERGE.toString(), submitter, config);
     }
@@ -135,6 +139,7 @@ public class CubingJob extends DefaultChainedExecutable {
         result.setJobType(jobType);
         CubingExecutableUtil.setCubeName(seg.getCubeInstance().getName(), result.getParams());
         CubingExecutableUtil.setSegmentId(seg.getUuid(), result.getParams());
+        CubingExecutableUtil.setSegmentName(seg.getName(), result.getParams());
         result.setName(jobType + " CUBE - " + seg.getCubeInstance().getName() + " - " + seg.getName() + " - "
                 + format.format(new Date(System.currentTimeMillis())));
         result.setSubmitter(submitter);

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java
index 69bba0a..e70b497 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMROutput2.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
 
 public interface IMROutput2 {
@@ -67,7 +68,7 @@ public interface IMROutput2 {
         public void configureJobInput(Job job, String input) throws Exception;
 
         /** Configure the OutputFormat of given job. */
-        public void configureJobOutput(Job job, String output, CubeSegment segment, int level) throws Exception;
+        public void configureJobOutput(Job job, String output, CubeSegment segment, CuboidScheduler cuboidScheduler, int level) throws Exception;
 
     }
 
@@ -113,4 +114,30 @@ public interface IMROutput2 {
         public CubeSegment findSourceSegment(FileSplit fileSplit, CubeInstance cube);
     }
 
+    public IMRBatchOptimizeOutputSide2 getBatchOptimizeOutputSide(CubeSegment seg);
+
+    /**
+     * Participate the batch cubing flow as the output side. Responsible for saving
+     * the cuboid output to storage at the end of Phase 3.
+     *
+     * - Phase 1: Filter Recommended Cuboid Data
+     * - Phase 2: Copy Dictionary & Calculate Statistics & Update Reused Cuboid Shard
+     * - Phase 3: Build Cube
+     * - Phase 4: Cleanup Optimize
+     * - Phase 5: Update Metadata & Cleanup
+     */
+    public interface IMRBatchOptimizeOutputSide2 {
+
+        /** Create HTable based on recommended cuboids & statistics*/
+        public void addStepPhase2_CreateHTable(DefaultChainedExecutable jobFlow);
+
+        /** Build only missing cuboids*/
+        public void addStepPhase3_BuildCube(DefaultChainedExecutable jobFlow);
+
+        /** Cleanup intermediate cuboid data on HDFS*/
+        public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow);
+
+        /** Invoked by Checkpoint job & Cleanup old segments' HTables and related working directory*/
+        public void addStepPhase5_Cleanup(DefaultChainedExecutable jobFlow);
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
index 2a51c89..694c936 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
@@ -22,14 +22,16 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidModeEnum;
 import org.apache.kylin.engine.mr.common.BatchConstants;
 import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
 import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+import org.apache.kylin.engine.mr.steps.CalculateStatsFromBaseCuboidJob;
 import org.apache.kylin.engine.mr.steps.CreateDictionaryJob;
-import org.apache.kylin.engine.mr.steps.UHCDictionaryJob;
 import org.apache.kylin.engine.mr.steps.CubingExecutableUtil;
 import org.apache.kylin.engine.mr.steps.FactDistinctColumnsJob;
 import org.apache.kylin.engine.mr.steps.MergeDictionaryStep;
+import org.apache.kylin.engine.mr.steps.UHCDictionaryJob;
 import org.apache.kylin.engine.mr.steps.UpdateCubeInfoAfterBuildStep;
 import org.apache.kylin.engine.mr.steps.UpdateCubeInfoAfterMergeStep;
 import org.apache.kylin.job.constant.ExecutableConstants;
@@ -48,6 +50,10 @@ public class JobBuilderSupport {
 
     final public static String LayeredCuboidFolderPrefix = "level_";
 
+    final public static String PathNameCuboidBase = "base_cuboid";
+    final public static String PathNameCuboidOld = "old";
+    final public static String PathNameCuboidInMem = "in_memory";
+
     public JobBuilderSupport(CubeSegment seg, String submitter) {
         Preconditions.checkNotNull(seg, "segment cannot be null");
         this.config = new JobEngineConfig(seg.getConfig());
@@ -98,6 +104,31 @@ public class JobBuilderSupport {
         return result;
     }
 
+    public MapReduceExecutable createCalculateStatsFromBaseCuboid(String inputPath, String outputPath) {
+        return createCalculateStatsFromBaseCuboid(inputPath, outputPath, CuboidModeEnum.CURRENT);
+    }
+
+    public MapReduceExecutable createCalculateStatsFromBaseCuboid(String inputPath, String outputPath,
+            CuboidModeEnum cuboidMode) {
+        MapReduceExecutable result = new MapReduceExecutable();
+        result.setName(ExecutableConstants.STEP_NAME_CALCULATE_STATS_FROM_BASE_CUBOID);
+        result.setMapReduceJobClass(CalculateStatsFromBaseCuboidJob.class);
+        StringBuilder cmd = new StringBuilder();
+        appendMapReduceParameters(cmd);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, seg.getRealization().getName());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, inputPath);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, outputPath);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_STATS_SAMPLING_PERCENT,
+                String.valueOf(config.getConfig().getCubingInMemSamplingPercent()));
+        appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME,
+                "Calculate_Stats_For_Segment_" + seg.getRealization().getName() + "_Step");
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBOID_MODE, cuboidMode.toString());
+
+        result.setMapReduceParams(cmd.toString());
+        return result;
+    }
+
     public HadoopShellExecutable createBuildDictionaryStep(String jobId) {
         // base cuboid job
         HadoopShellExecutable buildDictionaryStep = new HadoopShellExecutable();
@@ -197,6 +228,18 @@ public class JobBuilderSupport {
         return getRealizationRootPath(jobId) + "/dict";
     }
 
+    public String getOptimizationRootPath(String jobId) {
+        return getRealizationRootPath(jobId) + "/optimize";
+    }
+
+    public String getOptimizationStatisticsPath(String jobId) {
+        return getOptimizationRootPath(jobId) + "/statistics";
+    }
+
+    public String getOptimizationCuboidPath(String jobId) {
+        return getOptimizationRootPath(jobId) + "/cuboid/";
+    }
+
     // ============================================================================
     // static methods also shared by other job flow participant
     // ----------------------------------------------------------------------------
@@ -218,10 +261,17 @@ public class JobBuilderSupport {
 
     public static String getCuboidOutputPathsByLevel(String cuboidRootPath, int level) {
         if (level == 0) {
-            return cuboidRootPath + LayeredCuboidFolderPrefix + "base_cuboid";
+            return cuboidRootPath + LayeredCuboidFolderPrefix + PathNameCuboidBase;
         } else {
             return cuboidRootPath + LayeredCuboidFolderPrefix + level + "_cuboid";
         }
     }
 
+    public static String getBaseCuboidPath(String cuboidRootPath) {
+        return cuboidRootPath + PathNameCuboidBase;
+    }
+
+    public static String getInMemCuboidPath(String cuboidRootPath) {
+        return cuboidRootPath + PathNameCuboidInMem;
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java
index 681c545..74c9b6d 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine.java
@@ -48,6 +48,11 @@ public class MRBatchCubingEngine implements IBatchCubingEngine {
     }
 
     @Override
+    public DefaultChainedExecutable createBatchOptimizeJob(CubeSegment optimizeSegment, String submitter) {
+        return new BatchOptimizeJobBuilder2(optimizeSegment, submitter).build();
+    }
+
+    @Override
     public Class<?> getSourceInterface() {
         return IMRInput.class;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java
index d9fdcb9..665e791 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRBatchCubingEngine2.java
@@ -48,6 +48,11 @@ public class MRBatchCubingEngine2 implements IBatchCubingEngine {
     }
 
     @Override
+    public DefaultChainedExecutable createBatchOptimizeJob(CubeSegment optimizeSegment, String submitter) {
+        return new BatchOptimizeJobBuilder2(optimizeSegment, submitter).build();
+    }
+
+    @Override
     public Class<?> getSourceInterface() {
         return IMRInput.class;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
index b2a2ea3..124e5e7 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
@@ -76,6 +76,10 @@ public class MRUtil {
         return SourceFactory.createEngineAdapter(seg, IMRInput.class).getBatchMergeInputSide(seg);
     }
 
+    public static IMROutput2.IMRBatchOptimizeOutputSide2 getBatchOptimizeOutputSide2(CubeSegment seg) {
+        return StorageFactory.createEngineAdapter(seg, IMROutput2.class).getBatchOptimizeOutputSide(seg);
+    }
+    
     // use this method instead of ToolRunner.run() because ToolRunner.run() is not thread-sale
     // Refer to: http://stackoverflow.com/questions/22462665/is-hadoops-toorunner-thread-safe
     public static int runMRJob(Tool tool, String[] args) throws Exception {

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
index fd212be..f5cee9f 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
@@ -24,12 +24,14 @@ package org.apache.kylin.engine.mr.common;
  */
 
 import static org.apache.hadoop.util.StringUtils.formatTime;
+import static org.apache.kylin.engine.mr.common.JobRelatedMetaUtil.collectCubeMetadata;
 
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.HashMap;
 import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
@@ -114,6 +116,8 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
     protected static final Option OPTION_STATISTICS_SAMPLING_PERCENT = OptionBuilder
             .withArgName(BatchConstants.ARG_STATS_SAMPLING_PERCENT).hasArg().isRequired(false)
             .withDescription("Statistics sampling percentage").create(BatchConstants.ARG_STATS_SAMPLING_PERCENT);
+    protected static final Option OPTION_CUBOID_MODE = OptionBuilder.withArgName(BatchConstants.ARG_CUBOID_MODE)
+            .hasArg().isRequired(false).withDescription("Cuboid Mode").create(BatchConstants.ARG_CUBOID_MODE);
 
     private static final String MAP_REDUCE_CLASSPATH = "mapreduce.application.classpath";
 
@@ -492,27 +496,41 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
     }
 
     protected void attachCubeMetadata(CubeInstance cube, Configuration conf) throws IOException {
-        dumpKylinPropsAndMetadata(cube.getProject(), JobRelatedMetaUtil.collectCubeMetadata(cube), cube.getConfig(),
+        dumpKylinPropsAndMetadata(cube.getProject(), collectCubeMetadata(cube), cube.getConfig(),
                 conf);
     }
 
     protected void attachCubeMetadataWithDict(CubeInstance cube, Configuration conf) throws IOException {
         Set<String> dumpList = new LinkedHashSet<>();
-        dumpList.addAll(JobRelatedMetaUtil.collectCubeMetadata(cube));
+        dumpList.addAll(collectCubeMetadata(cube));
         for (CubeSegment segment : cube.getSegments()) {
             dumpList.addAll(segment.getDictionaryPaths());
         }
         dumpKylinPropsAndMetadata(cube.getProject(), dumpList, cube.getConfig(), conf);
     }
 
+    protected void attachSegmentsMetadataWithDict(List<CubeSegment> segments, Configuration conf) throws IOException {
+        Set<String> dumpList = new LinkedHashSet<>();
+        CubeInstance cube = segments.get(0).getCubeInstance();
+        dumpList.addAll(collectCubeMetadata(cube));
+        for (CubeSegment segment : segments) {
+            dumpList.addAll(segment.getDictionaryPaths());
+        }
+        dumpKylinPropsAndMetadata(cube.getProject(), dumpList, cube.getConfig(), conf);
+    }
+
     protected void attachSegmentMetadataWithDict(CubeSegment segment, Configuration conf) throws IOException {
+        attachSegmentMetadata(segment, conf, true, false);
+    }
+
+    protected void attachSegmentMetadataWithAll(CubeSegment segment, Configuration conf) throws IOException {
         attachSegmentMetadata(segment, conf, true, true);
     }
 
     protected void attachSegmentMetadata(CubeSegment segment, Configuration conf, boolean ifDictIncluded,
             boolean ifStatsIncluded) throws IOException {
         Set<String> dumpList = new LinkedHashSet<>();
-        dumpList.addAll(JobRelatedMetaUtil.collectCubeMetadata(segment.getCubeInstance()));
+        dumpList.addAll(collectCubeMetadata(segment.getCubeInstance()));
         if (ifDictIncluded) {
             dumpList.addAll(segment.getDictionaryPaths());
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
index aaf2654..1d6a582 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
@@ -75,6 +75,7 @@ public interface BatchConstants {
     String ARG_INPUT = "input";
     String ARG_OUTPUT = "output";
     String ARG_PROJECT = "project";
+    String ARG_CUBOID_MODE = "cuboidMode";
     String ARG_JOB_NAME = "jobname";
     String ARG_CUBING_JOB_ID = "cubingJobId";
     String ARG_CUBE_NAME = "cubename";

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
index c82d797..3d7d542 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
@@ -80,51 +80,52 @@ public class CubeStatsReader {
     final CuboidScheduler cuboidScheduler;
 
     public CubeStatsReader(CubeSegment cubeSegment, KylinConfig kylinConfig) throws IOException {
+        this(cubeSegment, cubeSegment.getCuboidScheduler(), kylinConfig);
+    }
+
+    /**
+     * @param cuboidScheduler if it's null, part of it's functions will not be supported
+     */
+    public CubeStatsReader(CubeSegment cubeSegment, CuboidScheduler cuboidScheduler, KylinConfig kylinConfig)
+            throws IOException {
         ResourceStore store = ResourceStore.getStore(kylinConfig);
-        cuboidScheduler = cubeSegment.getCuboidScheduler();
         String statsKey = cubeSegment.getStatisticsResourcePath();
         File tmpSeqFile = writeTmpSeqFile(store.getResource(statsKey).inputStream);
-        Reader reader = null;
-
-        try {
-            Configuration hadoopConf = HadoopUtil.getCurrentConfiguration();
-
-            Path path = new Path(HadoopUtil.fixWindowsPath("file://" + tmpSeqFile.getAbsolutePath()));
-            Option seqInput = SequenceFile.Reader.file(path);
-            reader = new SequenceFile.Reader(hadoopConf, seqInput);
-
-            int percentage = 100;
-            int mapperNumber = 0;
-            double mapperOverlapRatio = 0;
-            Map<Long, HLLCounter> counterMap = Maps.newHashMap();
-
-            LongWritable key = (LongWritable) ReflectionUtils.newInstance(reader.getKeyClass(), hadoopConf);
-            BytesWritable value = (BytesWritable) ReflectionUtils.newInstance(reader.getValueClass(), hadoopConf);
-            while (reader.next(key, value)) {
-                if (key.get() == 0L) {
-                    percentage = Bytes.toInt(value.getBytes());
-                } else if (key.get() == -1) {
-                    mapperOverlapRatio = Bytes.toDouble(value.getBytes());
-                } else if (key.get() == -2) {
-                    mapperNumber = Bytes.toInt(value.getBytes());
-                } else if (key.get() > 0) {
-                    HLLCounter hll = new HLLCounter(kylinConfig.getCubeStatsHLLPrecision());
-                    ByteArray byteArray = new ByteArray(value.getBytes());
-                    hll.readRegisters(byteArray.asBuffer());
-                    counterMap.put(key.get(), hll);
-                }
-            }
-
-            this.seg = cubeSegment;
-            this.samplingPercentage = percentage;
-            this.mapperNumberOfFirstBuild = mapperNumber;
-            this.mapperOverlapRatioOfFirstBuild = mapperOverlapRatio;
-            this.cuboidRowEstimatesHLL = counterMap;
+        Path path = new Path(HadoopUtil.fixWindowsPath("file://" + tmpSeqFile.getAbsolutePath()));
+
+        CubeStatsResult cubeStatsResult = new CubeStatsResult();
+        cubeStatsResult.initialize(path, kylinConfig.getCubeStatsHLLPrecision());
+        tmpSeqFile.delete();
+
+        this.seg = cubeSegment;
+        this.cuboidScheduler = cuboidScheduler;
+        this.samplingPercentage = cubeStatsResult.percentage;
+        this.mapperNumberOfFirstBuild = cubeStatsResult.mapperNumber;
+        this.mapperOverlapRatioOfFirstBuild = cubeStatsResult.mapperOverlapRatio;
+        this.cuboidRowEstimatesHLL = cubeStatsResult.counterMap;
+    }
 
-        } finally {
-            IOUtils.closeStream(reader);
-            tmpSeqFile.delete();
-        }
+    /**
+     * Read statistics from
+     * @param path
+     * rather than
+     * @param cubeSegment
+     *
+     * Since the statistics are from
+     * @param path
+     * cuboid scheduler should be provided by default
+     */
+    public CubeStatsReader(CubeSegment cubeSegment, CuboidScheduler cuboidScheduler, KylinConfig kylinConfig, Path path)
+            throws IOException {
+        CubeStatsResult cubeStatsResult = new CubeStatsResult();
+        cubeStatsResult.initialize(path, kylinConfig.getCubeStatsHLLPrecision());
+
+        this.seg = cubeSegment;
+        this.cuboidScheduler = cuboidScheduler;
+        this.samplingPercentage = cubeStatsResult.percentage;
+        this.mapperNumberOfFirstBuild = cubeStatsResult.mapperNumber;
+        this.mapperOverlapRatioOfFirstBuild = cubeStatsResult.mapperOverlapRatio;
+        this.cuboidRowEstimatesHLL = cubeStatsResult.counterMap;
     }
 
     private File writeTmpSeqFile(InputStream inputStream) throws IOException {
@@ -144,6 +145,10 @@ public class CubeStatsReader {
         return this.cuboidRowEstimatesHLL;
     }
 
+    public int getSamplingPercentage() {
+        return samplingPercentage;
+    }
+
     public Map<Long, Long> getCuboidRowEstimatesHLL() {
         return getCuboidRowCountMapFromSampling(cuboidRowEstimatesHLL, samplingPercentage);
     }
@@ -253,6 +258,9 @@ public class CubeStatsReader {
 
     //return MB
     public double estimateLayerSize(int level) {
+        if (cuboidScheduler == null) {
+            throw new UnsupportedOperationException("cuboid scheduler is null");
+        }
         List<List<Long>> layeredCuboids = cuboidScheduler.getCuboidsByLayer();
         Map<Long, Double> cuboidSizeMap = getCuboidSizeMap();
         double ret = 0;
@@ -265,11 +273,17 @@ public class CubeStatsReader {
     }
 
     public List<Long> getCuboidsByLayer(int level) {
+        if (cuboidScheduler == null) {
+            throw new UnsupportedOperationException("cuboid scheduler is null");
+        }
         List<List<Long>> layeredCuboids = cuboidScheduler.getCuboidsByLayer();
         return layeredCuboids.get(level);
     }
 
     private void printCuboidInfoTreeEntry(Map<Long, Long> cuboidRows, Map<Long, Double> cuboidSizes, PrintWriter out) {
+        if (cuboidScheduler == null) {
+            throw new UnsupportedOperationException("cuboid scheduler is null");
+        }
         long baseCuboid = Cuboid.getBaseCuboidId(seg.getCubeDesc());
         int dimensionCount = Long.bitCount(baseCuboid);
         printCuboidInfoTree(-1L, baseCuboid, cuboidScheduler, cuboidRows, cuboidSizes, dimensionCount, 0, out);
@@ -317,6 +331,36 @@ public class CubeStatsReader {
         return new DecimalFormat("#.##").format(input);
     }
 
+    private class CubeStatsResult {
+        private int percentage = 100;
+        private double mapperOverlapRatio = 0;
+        private int mapperNumber = 0;
+        Map<Long, HLLCounter> counterMap = Maps.newHashMap();
+
+        void initialize(Path path, int precision) throws IOException {
+            Configuration hadoopConf = HadoopUtil.getCurrentConfiguration();
+            Option seqInput = SequenceFile.Reader.file(path);
+            try (Reader reader = new SequenceFile.Reader(hadoopConf, seqInput)) {
+                LongWritable key = (LongWritable) ReflectionUtils.newInstance(reader.getKeyClass(), hadoopConf);
+                BytesWritable value = (BytesWritable) ReflectionUtils.newInstance(reader.getValueClass(), hadoopConf);
+                while (reader.next(key, value)) {
+                    if (key.get() == 0L) {
+                        percentage = Bytes.toInt(value.getBytes());
+                    } else if (key.get() == -1) {
+                        mapperOverlapRatio = Bytes.toDouble(value.getBytes());
+                    } else if (key.get() == -2) {
+                        mapperNumber = Bytes.toInt(value.getBytes());
+                    } else if (key.get() > 0) {
+                        HLLCounter hll = new HLLCounter(precision);
+                        ByteArray byteArray = new ByteArray(value.getBytes());
+                        hll.readRegisters(byteArray.asBuffer());
+                        counterMap.put(key.get(), hll);
+                    }
+                }
+            }
+        }
+    }
+
     public static void main(String[] args) throws IOException {
         System.out.println("CubeStatsReader is used to read cube statistic saved in metadata store");
         KylinConfig config = KylinConfig.getInstanceFromEnv();

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidRecommenderUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidRecommenderUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidRecommenderUtil.java
index ba3f023..649eeb6 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidRecommenderUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidRecommenderUtil.java
@@ -39,20 +39,21 @@ public class CuboidRecommenderUtil {
             return null;
         }
 
-        CubeStatsReader cubeStatsReader = new CubeStatsReader(segment, segment.getConfig());
+        CubeStatsReader cubeStatsReader = new CubeStatsReader(segment, null, segment.getConfig());
         if (cubeStatsReader.getCuboidRowEstimatesHLL() == null
                 || cubeStatsReader.getCuboidRowEstimatesHLL().isEmpty()) {
             logger.info("Cuboid Statistics is not enabled.");
             return null;
         }
-        long baseCuboid = segment.getCuboidScheduler().getBaseCuboidId();
+        CubeInstance cube = segment.getCubeInstance();
+        long baseCuboid = cube.getCuboidScheduler().getBaseCuboidId();
         if (cubeStatsReader.getCuboidRowEstimatesHLL().get(baseCuboid) == null
                 || cubeStatsReader.getCuboidRowEstimatesHLL().get(baseCuboid) == 0L) {
             logger.info("Base cuboid count in cuboid statistics is 0.");
             return null;
         }
 
-        String key = segment.getCubeInstance().getName();
+        String key = cube.getName();
         CuboidStats cuboidStats = new CuboidStats.Builder(key, baseCuboid, cubeStatsReader.getCuboidRowEstimatesHLL(),
                 cubeStatsReader.getCuboidSizeMap()).build();
         return CuboidRecommender.getInstance().getRecommendCuboidList(cuboidStats, segment.getConfig(), false);
@@ -81,20 +82,21 @@ public class CuboidRecommenderUtil {
             return null;
         }
 
-        CubeStatsReader cubeStatsReader = new CubeStatsReader(segment, segment.getConfig());
+        CubeStatsReader cubeStatsReader = new CubeStatsReader(segment, null, segment.getConfig());
         if (cubeStatsReader.getCuboidRowEstimatesHLL() == null
                 || cubeStatsReader.getCuboidRowEstimatesHLL().isEmpty()) {
             logger.info("Cuboid Statistics is not enabled.");
             return null;
         }
-        long baseCuboid = segment.getCuboidScheduler().getBaseCuboidId();
+        CubeInstance cube = segment.getCubeInstance();
+        long baseCuboid = cube.getCuboidScheduler().getBaseCuboidId();
         if (cubeStatsReader.getCuboidRowEstimatesHLL().get(baseCuboid) == null
                 || cubeStatsReader.getCuboidRowEstimatesHLL().get(baseCuboid) == 0L) {
             logger.info("Base cuboid count in cuboid statistics is 0.");
             return null;
         }
 
-        String key = segment.getCubeInstance().getName() + "-" + segment.getName();
+        String key = cube.getName() + "-" + segment.getName();
         CuboidStats cuboidStats = new CuboidStats.Builder(key, baseCuboid, cubeStatsReader.getCuboidRowEstimatesHLL(),
                 cubeStatsReader.getCuboidSizeMap()).setHitFrequencyMap(hitFrequencyMap)
                         .setRollingUpCountSourceMap(rollingUpCountSourceMap,

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidSchedulerUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidSchedulerUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidSchedulerUtil.java
new file mode 100644
index 0000000..d684c04
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidSchedulerUtil.java
@@ -0,0 +1,54 @@
+/*
+ * 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.mr.common;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidModeEnum;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.cuboid.DefaultCuboidScheduler;
+import org.apache.kylin.cube.cuboid.TreeCuboidSchedulerManager;
+
+public class CuboidSchedulerUtil {
+
+    public static CuboidScheduler getCuboidSchedulerByMode(CubeSegment segment, String cuboidModeName) {
+        return getCuboidSchedulerByMode(segment, segment.getCubeInstance().getCuboidsByMode(cuboidModeName));
+    }
+
+    public static CuboidScheduler getCuboidSchedulerByMode(CubeSegment segment, CuboidModeEnum cuboidMode) {
+        return getCuboidSchedulerByMode(segment, segment.getCubeInstance().getCuboidsByMode(cuboidMode));
+    }
+
+    public static CuboidScheduler getCuboidSchedulerByMode(CubeSegment segment, Set<Long> cuboidSet) {
+        CuboidScheduler cuboidScheduler;
+        try {
+            cuboidScheduler = TreeCuboidSchedulerManager.getInstance().getTreeCuboidScheduler(segment.getCubeDesc(), //
+                    CuboidStatsReaderUtil.readCuboidStatsFromSegment(cuboidSet, segment));
+        } catch (IOException e) {
+            throw new RuntimeException("Fail to cube stats for segment" + segment + " due to " + e);
+        }
+
+        if (cuboidScheduler == null) {
+            cuboidScheduler = new DefaultCuboidScheduler(segment.getCubeDesc());
+        }
+        return cuboidScheduler;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsReaderUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsReaderUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsReaderUtil.java
index 68380f3..56ab504 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsReaderUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsReaderUtil.java
@@ -74,7 +74,7 @@ public class CuboidStatsReaderUtil {
         Map<Long, HLLCounter> cuboidHLLMapMerged = Maps.newHashMapWithExpectedSize(cuboidSet.size());
         Map<Long, Double> sizeMapMerged = Maps.newHashMapWithExpectedSize(cuboidSet.size());
         for (CubeSegment pSegment : segmentList) {
-            CubeStatsReader pReader = new CubeStatsReader(pSegment, pSegment.getConfig());
+            CubeStatsReader pReader = new CubeStatsReader(pSegment, null, pSegment.getConfig());
             Map<Long, HLLCounter> pHLLMap = pReader.getCuboidRowHLLCounters();
             if (pHLLMap == null || pHLLMap.isEmpty()) {
                 logger.info("Cuboid Statistics for segment " + pSegment.getName() + " is not enabled.");
@@ -113,7 +113,7 @@ public class CuboidStatsReaderUtil {
             return null;
         }
 
-        CubeStatsReader cubeStatsReader = new CubeStatsReader(cubeSegment, cubeSegment.getConfig());
+        CubeStatsReader cubeStatsReader = new CubeStatsReader(cubeSegment, null, cubeSegment.getConfig());
         if (cubeStatsReader.getCuboidRowEstimatesHLL() == null
                 || cubeStatsReader.getCuboidRowEstimatesHLL().isEmpty()) {
             logger.info("Cuboid Statistics is not enabled.");
@@ -132,4 +132,5 @@ public class CuboidStatsReaderUtil {
         }
         return cuboidsWithStats;
     }
+
 }


[09/18] kylin git commit: APACHE-KYLIN-2733: Change findById to findForMandatory during cube building

Posted by li...@apache.org.
APACHE-KYLIN-2733: Change findById to findForMandatory during cube building


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/37f83ed6
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/37f83ed6
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/37f83ed6

Branch: refs/heads/ci-dong
Commit: 37f83ed6dd882712cdcf49c35af5acd1a7855611
Parents: 020c4e7
Author: Zhong <nj...@apache.org>
Authored: Tue Nov 7 02:36:16 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/cube/cuboid/Cuboid.java     |  9 +--------
 .../java/org/apache/kylin/cube/CubeInstanceTest.java  | 14 +++++++++-----
 .../org/apache/kylin/cube/kv/RowKeyDecoderTest.java   |  2 +-
 .../org/apache/kylin/cube/kv/RowKeyEncoderTest.java   |  6 +++---
 .../kylin/engine/mr/steps/MergeCuboidMapper.java      |  2 +-
 .../apache/kylin/engine/mr/steps/NDCuboidMapper.java  |  2 +-
 .../apache/kylin/engine/spark/SparkCubingByLayer.java |  6 +++---
 .../kylin/storage/hbase/steps/HBaseCuboidWriter.java  |  5 +++--
 8 files changed, 22 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/37f83ed6/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
index 3c4fceb..d9c7803 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
@@ -93,18 +93,11 @@ public class Cuboid implements Comparable<Cuboid>, Serializable {
         return new Cuboid(cube, cuboidID, cuboidID);
     }
 
-    public static Cuboid findById(CuboidScheduler cuboidScheduler, byte[] cuboidID) {
-        return findById(cuboidScheduler, Bytes.toLong(cuboidID));
-    }
-
+    @Deprecated
     public static Cuboid findById(CubeSegment cubeSegment, long cuboidID) {
         return findById(cubeSegment.getCuboidScheduler(), cuboidID);
     }
 
-    public static Cuboid findById(CubeInstance cubeInstance, long cuboidID) {
-        return findById(cubeInstance.getCuboidScheduler(), cuboidID);
-    }
-
     @VisibleForTesting
     static Cuboid findById(CubeDesc cubeDesc, long cuboidID) {
         return findById(cubeDesc.getInitialCuboidScheduler(), cuboidID);

http://git-wip-us.apache.org/repos/asf/kylin/blob/37f83ed6/core-cube/src/test/java/org/apache/kylin/cube/CubeInstanceTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/CubeInstanceTest.java b/core-cube/src/test/java/org/apache/kylin/cube/CubeInstanceTest.java
index 0af5f98..9e517b9 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/CubeInstanceTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/CubeInstanceTest.java
@@ -18,9 +18,13 @@
 
 package org.apache.kylin.cube;
 
+import static org.junit.Assert.assertNotNull;
+
 import java.io.DataInputStream;
 import java.io.FileInputStream;
 import java.io.InputStream;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.kylin.common.persistence.JsonSerializer;
@@ -46,12 +50,12 @@ public class CubeInstanceTest {
         Map<Long, Long> cuboidsWithRowCnt = cubeInstance.getCuboids();
         TreeCuboidScheduler.CuboidTree.createFromCuboids(Lists.newArrayList(cuboidsWithRowCnt.keySet()),
                 new TreeCuboidScheduler.CuboidCostComparator(cuboidsWithRowCnt));
-    }
 
-    public void printMap(Map<Long, Long> map) {
-        System.out.println("size: " + map.size());
-        for (Map.Entry<Long, Long> entry : map.entrySet()) {
-            System.out.println(entry.getKey() + ":" + entry.getValue());
+        List<Long> cuboids = Lists.newArrayList(cuboidsWithRowCnt.keySet());
+        Collections.sort(cuboids);
+        for (Long cuboid : cuboids) {
+            System.out.println(cuboid + ":" + cuboidsWithRowCnt.get(cuboid));
         }
+        assertNotNull(cuboidsWithRowCnt.get(255L));
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/37f83ed6/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyDecoderTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyDecoderTest.java b/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyDecoderTest.java
index 459e734..d24baf6 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyDecoderTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyDecoderTest.java
@@ -91,7 +91,7 @@ public class RowKeyDecoderTest extends LocalFileMetadataTestCase {
         data[7] = "15";
 
         long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        Cuboid baseCuboid = Cuboid.findById(cube, baseCuboidId);
+        Cuboid baseCuboid = Cuboid.findForMandatory(cubeDesc, baseCuboidId);
         RowKeyEncoder rowKeyEncoder = new RowKeyEncoder(cube.getFirstSegment(), baseCuboid);
 
         byte[] encodedKey = rowKeyEncoder.encode(data);

http://git-wip-us.apache.org/repos/asf/kylin/blob/37f83ed6/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyEncoderTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyEncoderTest.java b/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyEncoderTest.java
index dcd883e..d638d8c 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyEncoderTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyEncoderTest.java
@@ -66,7 +66,7 @@ public class RowKeyEncoderTest extends LocalFileMetadataTestCase {
         data[7] = "15";
 
         long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        Cuboid baseCuboid = Cuboid.findById(cube, baseCuboidId);
+        Cuboid baseCuboid = Cuboid.findForMandatory(cubeDesc, baseCuboidId);
         RowKeyEncoder rowKeyEncoder = new RowKeyEncoder(cube.getFirstSegment(), baseCuboid);
 
         byte[] encodedKey = rowKeyEncoder.encode(data);
@@ -97,7 +97,7 @@ public class RowKeyEncoderTest extends LocalFileMetadataTestCase {
         data[8] = "15";
 
         long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        Cuboid baseCuboid = Cuboid.findById(cube, baseCuboidId);
+        Cuboid baseCuboid = Cuboid.findForMandatory(cubeDesc, baseCuboidId);
         RowKeyEncoder rowKeyEncoder = new RowKeyEncoder(cube.getFirstSegment(), baseCuboid);
 
         byte[] encodedKey = rowKeyEncoder.encode(data);
@@ -133,7 +133,7 @@ public class RowKeyEncoderTest extends LocalFileMetadataTestCase {
         data[8] = null;
 
         long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        Cuboid baseCuboid = Cuboid.findById(cube, baseCuboidId);
+        Cuboid baseCuboid = Cuboid.findForMandatory(cubeDesc, baseCuboidId);
         RowKeyEncoder rowKeyEncoder = new RowKeyEncoder(cube.getFirstSegment(), baseCuboid);
 
         byte[] encodedKey = rowKeyEncoder.encode(data);

http://git-wip-us.apache.org/repos/asf/kylin/blob/37f83ed6/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
index 4f68293..2c85a28 100755
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
@@ -147,7 +147,7 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
     @Override
     public void doMap(Text key, Text value, Context context) throws IOException, InterruptedException {
         long cuboidID = rowKeySplitter.split(key.getBytes());
-        Cuboid cuboid = Cuboid.findById(cube, cuboidID);
+        Cuboid cuboid = Cuboid.findForMandatory(cubeDesc, cuboidID);
         RowKeyEncoder rowkeyEncoder = rowKeyEncoderProvider.getRowkeyEncoder(cuboid);
 
         SplittedBytes[] splittedByteses = rowKeySplitter.getSplitBuffers();

http://git-wip-us.apache.org/repos/asf/kylin/blob/37f83ed6/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
index 359bd4c..f936393 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
@@ -86,7 +86,7 @@ public class NDCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
     @Override
     public void doMap(Text key, Text value, Context context) throws IOException, InterruptedException {
         long cuboidId = rowKeySplitter.split(key.getBytes());
-        Cuboid parentCuboid = Cuboid.findById(cuboidScheduler, cuboidId);
+        Cuboid parentCuboid = Cuboid.findForMandatory(cubeDesc, cuboidId);
 
         Collection<Long> myChildren = cuboidScheduler.getSpanningCuboid(cuboidId);
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/37f83ed6/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
index bdf0633..8d75070 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
@@ -290,7 +290,7 @@ public class SparkCubingByLayer extends AbstractApplication implements Serializa
                         CubeJoinedFlatTableEnrich interDesc = new CubeJoinedFlatTableEnrich(
                                 EngineFactory.getJoinedFlatTableDesc(cubeSegment), cubeDesc);
                         long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-                        Cuboid baseCuboid = Cuboid.findById(cubeInstance, baseCuboidId);
+                        Cuboid baseCuboid = Cuboid.findForMandatory(cubeDesc, baseCuboidId);
                         baseCuboidBuilder = new BaseCuboidBuilder(kConfig, cubeDesc, cubeSegment, interDesc,
                                 AbstractRowKeyEncoder.createInstance(cubeSegment, baseCuboid),
                                 MeasureIngester.create(cubeDesc.getMeasures()), cubeSegment.buildDictionaryMap());
@@ -423,7 +423,7 @@ public class SparkCubingByLayer extends AbstractApplication implements Serializa
 
             byte[] key = tuple2._1().array();
             long cuboidId = rowKeySplitter.split(key);
-            Cuboid parentCuboid = Cuboid.findById(cubeSegment, cuboidId);
+            Cuboid parentCuboid = Cuboid.findForMandatory(cubeDesc, cuboidId);
 
             Collection<Long> myChildren = cubeSegment.getCuboidScheduler().getSpanningCuboid(cuboidId);
 
@@ -434,7 +434,7 @@ public class SparkCubingByLayer extends AbstractApplication implements Serializa
 
             List<Tuple2<ByteArray, Object[]>> tuples = new ArrayList(myChildren.size());
             for (Long child : myChildren) {
-                Cuboid childCuboid = Cuboid.findById(cubeSegment, child);
+                Cuboid childCuboid = Cuboid.findForMandatory(cubeDesc, child);
                 Pair<Integer, ByteArray> result = ndCuboidBuilder.buildKey(parentCuboid, childCuboid,
                         rowKeySplitter.getSplitBuffers());
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/37f83ed6/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
index 59f1349..e04e07d 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
@@ -81,7 +81,8 @@ public class HBaseCuboidWriter implements ICuboidWriter {
     //TODO:shardingonstreaming
     private byte[] createKey(Long cuboidId, GTRecord record) {
         if (rowKeyEncoder == null || rowKeyEncoder.getCuboidID() != cuboidId) {
-            rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment, Cuboid.findById(cubeSegment, cuboidId));
+            rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment,
+                    Cuboid.findForMandatory(cubeDesc, cuboidId));
             keybuf = rowKeyEncoder.createBuf();
         }
         rowKeyEncoder.encode(record, record.getInfo().getPrimaryKey(), keybuf);
@@ -92,7 +93,7 @@ public class HBaseCuboidWriter implements ICuboidWriter {
     @Override
     public void write(long cuboidId, GTRecord record) throws IOException {
         byte[] key = createKey(cuboidId, record);
-        final Cuboid cuboid = Cuboid.findById(cubeSegment, cuboidId);
+        final Cuboid cuboid = Cuboid.findForMandatory(cubeDesc, cuboidId);
         final int nDims = cuboid.getColumns().size();
         final ImmutableBitSet bitSet = new ImmutableBitSet(nDims, nDims + cubeDesc.getMeasures().size());
 


[06/18] kylin git commit: APACHE-KYLIN-2733: Introduce optimize job for adjusting cuboid set

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceUtil.java
new file mode 100644
index 0000000..0379f64
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceUtil.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.engine.mr.common;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.job.exception.JobException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MapReduceUtil {
+
+    private static final Logger logger = LoggerFactory.getLogger(MapReduceUtil.class);
+
+    /**
+     * @param cuboidScheduler specified can provide more flexibility
+     * */
+    public static int getLayeredCubingReduceTaskNum(CubeSegment cubeSegment, CuboidScheduler cuboidScheduler,
+            double totalMapInputMB, int level)
+            throws ClassNotFoundException, IOException, InterruptedException, JobException {
+        CubeDesc cubeDesc = cubeSegment.getCubeDesc();
+        KylinConfig kylinConfig = cubeDesc.getConfig();
+
+        double perReduceInputMB = kylinConfig.getDefaultHadoopJobReducerInputMB();
+        double reduceCountRatio = kylinConfig.getDefaultHadoopJobReducerCountRatio();
+        logger.info("Having per reduce MB " + perReduceInputMB + ", reduce count ratio " + reduceCountRatio + ", level "
+                + level);
+
+        CubeStatsReader cubeStatsReader = new CubeStatsReader(cubeSegment, cuboidScheduler, kylinConfig);
+
+        double parentLayerSizeEst, currentLayerSizeEst, adjustedCurrentLayerSizeEst;
+
+        if (level == -1) {
+            //merge case
+            double estimatedSize = cubeStatsReader.estimateCubeSize();
+            adjustedCurrentLayerSizeEst = estimatedSize > totalMapInputMB ? totalMapInputMB : estimatedSize;
+            logger.debug("estimated size {}, input size {}, adjustedCurrentLayerSizeEst: {}", estimatedSize,
+                    totalMapInputMB, adjustedCurrentLayerSizeEst);
+        } else if (level == 0) {
+            //base cuboid case TODO: the estimation could be very WRONG because it has no correction
+            adjustedCurrentLayerSizeEst = cubeStatsReader.estimateLayerSize(0);
+            logger.debug("adjustedCurrentLayerSizeEst: {}", adjustedCurrentLayerSizeEst);
+        } else {
+            parentLayerSizeEst = cubeStatsReader.estimateLayerSize(level - 1);
+            currentLayerSizeEst = cubeStatsReader.estimateLayerSize(level);
+            adjustedCurrentLayerSizeEst = totalMapInputMB / parentLayerSizeEst * currentLayerSizeEst;
+            logger.debug(
+                    "totalMapInputMB: {}, parentLayerSizeEst: {}, currentLayerSizeEst: {}, adjustedCurrentLayerSizeEst: {}",
+                    totalMapInputMB, parentLayerSizeEst, currentLayerSizeEst, adjustedCurrentLayerSizeEst);
+        }
+
+        // number of reduce tasks
+        int numReduceTasks = (int) Math.round(adjustedCurrentLayerSizeEst / perReduceInputMB * reduceCountRatio + 0.99);
+
+        // adjust reducer number for cube which has DISTINCT_COUNT measures for better performance
+        if (cubeDesc.hasMemoryHungryMeasures()) {
+            logger.debug("Multiply reducer num by 4 to boost performance for memory hungry measures");
+            numReduceTasks = numReduceTasks * 4;
+        }
+
+        // at least 1 reducer by default
+        numReduceTasks = Math.max(kylinConfig.getHadoopJobMinReducerNumber(), numReduceTasks);
+        // no more than 500 reducer by default
+        numReduceTasks = Math.min(kylinConfig.getHadoopJobMaxReducerNumber(), numReduceTasks);
+
+        return numReduceTasks;
+    }
+
+    public static int getInmemCubingReduceTaskNum(CubeSegment cubeSeg, CuboidScheduler cuboidScheduler)
+            throws IOException {
+        KylinConfig kylinConfig = cubeSeg.getConfig();
+
+        Map<Long, Double> cubeSizeMap = new CubeStatsReader(cubeSeg, cuboidScheduler, kylinConfig).getCuboidSizeMap();
+        double totalSizeInM = 0;
+        for (Double cuboidSize : cubeSizeMap.values()) {
+            totalSizeInM += cuboidSize;
+        }
+
+        double perReduceInputMB = kylinConfig.getDefaultHadoopJobReducerInputMB();
+        double reduceCountRatio = kylinConfig.getDefaultHadoopJobReducerCountRatio();
+
+        // number of reduce tasks
+        int numReduceTasks = (int) Math.round(totalSizeInM / perReduceInputMB * reduceCountRatio);
+
+        // at least 1 reducer by default
+        numReduceTasks = Math.max(kylinConfig.getHadoopJobMinReducerNumber(), numReduceTasks);
+        // no more than 500 reducer by default
+        numReduceTasks = Math.min(kylinConfig.getHadoopJobMaxReducerNumber(), numReduceTasks);
+
+        logger.info("Having total map input MB " + Math.round(totalSizeInM));
+        logger.info("Having per reduce MB " + perReduceInputMB);
+        logger.info("Setting " + Reducer.Context.NUM_REDUCES + "=" + numReduceTasks);
+        return numReduceTasks;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/StatisticsDecisionUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/StatisticsDecisionUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/StatisticsDecisionUtil.java
index 9c805a8..a5a1ba8 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/StatisticsDecisionUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/StatisticsDecisionUtil.java
@@ -38,7 +38,7 @@ public class StatisticsDecisionUtil {
     protected static final Logger logger = LoggerFactory.getLogger(StatisticsDecisionUtil.class);
 
     public static void decideCubingAlgorithm(CubingJob cubingJob, CubeSegment seg) throws IOException {
-        CubeStatsReader cubeStats = new CubeStatsReader(seg, seg.getConfig());
+        CubeStatsReader cubeStats = new CubeStatsReader(seg, null, seg.getConfig());
         decideCubingAlgorithm(cubingJob, seg, cubeStats.getMapperOverlapRatioOfFirstBuild(),
                 cubeStats.getMapperNumberOfFirstBuild());
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
index 1ae6cd0..0ad4b9e 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
@@ -60,7 +60,7 @@ abstract public class BaseCuboidMapperBase<KEYIN, VALUEIN> extends KylinMapper<K
     @Override
     protected void doSetup(Context context) throws IOException {
         super.bindCurrentConfiguration(context.getConfiguration());
-        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME);
         segmentID = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_ID);
         final KylinConfig kylinConfig = AbstractHadoopJob.loadKylinPropsAndMetadata();
         cube = CubeManager.getInstance(kylinConfig).getCube(cubeName);

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CalculateStatsFromBaseCuboidJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CalculateStatsFromBaseCuboidJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CalculateStatsFromBaseCuboidJob.java
new file mode 100644
index 0000000..b60076c
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CalculateStatsFromBaseCuboidJob.java
@@ -0,0 +1,116 @@
+/*
+ * 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.mr.steps;
+
+import java.io.IOException;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CalculateStatsFromBaseCuboidJob extends AbstractHadoopJob {
+
+    private static final Logger logger = LoggerFactory.getLogger(CalculateStatsFromBaseCuboidJob.class);
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_SEGMENT_ID);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_STATISTICS_SAMPLING_PERCENT);
+            options.addOption(OPTION_CUBOID_MODE);
+            parseOptions(options, args);
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            String cubeName = getOptionValue(OPTION_CUBE_NAME);
+            String segmentID = getOptionValue(OPTION_SEGMENT_ID);
+            Path input = new Path(getOptionValue(OPTION_INPUT_PATH));
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            String statistics_sampling_percent = getOptionValue(OPTION_STATISTICS_SAMPLING_PERCENT);
+            String cuboidMode = getOptionValue(OPTION_CUBOID_MODE);
+
+            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            CubeSegment cubeSegment = cube.getSegmentById(segmentID);
+
+            job.getConfiguration().set(BatchConstants.CFG_CUBOID_MODE, cuboidMode);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_ID, segmentID);
+            job.getConfiguration().set(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT, statistics_sampling_percent);
+            logger.info("Starting: " + job.getJobName());
+
+            setJobClasspath(job, cube.getConfig());
+
+            setupMapper(input);
+            setupReducer(output, 1);
+
+            attachSegmentMetadataWithDict(cubeSegment, job.getConfiguration());
+
+            return waitForCompletion(job);
+
+        } catch (Exception e) {
+            logger.error("error in CalculateStatsFromBaseCuboidJob", e);
+            printUsage(options);
+            throw e;
+        } finally {
+            if (job != null)
+                cleanupTempConfFile(job.getConfiguration());
+        }
+    }
+
+    private void setupMapper(Path input) throws IOException {
+        FileInputFormat.setInputPaths(job, input);
+        job.setMapperClass(CalculateStatsFromBaseCuboidMapper.class);
+        job.setInputFormatClass(SequenceFileInputFormat.class);
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(Text.class);
+    }
+
+    private void setupReducer(Path output, int numberOfReducers) throws IOException {
+        job.setReducerClass(CalculateStatsFromBaseCuboidReducer.class);
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        job.setOutputKeyClass(NullWritable.class);
+        job.setOutputValueClass(Text.class);
+        job.setNumReduceTasks(numberOfReducers);
+
+        FileOutputFormat.setOutputPath(job, output);
+        job.getConfiguration().set(BatchConstants.CFG_OUTPUT_PATH, output.toString());
+
+        deletePath(job.getConfiguration(), output);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CalculateStatsFromBaseCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CalculateStatsFromBaseCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CalculateStatsFromBaseCuboidMapper.java
new file mode 100644
index 0000000..1b32944
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CalculateStatsFromBaseCuboidMapper.java
@@ -0,0 +1,201 @@
+/*
+ * 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.mr.steps;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.KylinVersion;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidUtil;
+import org.apache.kylin.cube.kv.RowKeyDecoder;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.KylinMapper;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.measure.BufferedMeasureCodec;
+import org.apache.kylin.measure.hllc.HLLCounter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+
+public class CalculateStatsFromBaseCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
+    private static final Logger logger = LoggerFactory.getLogger(CalculateStatsFromBaseCuboidMapper.class);
+
+    protected int nRowKey;
+    protected long baseCuboidId;
+
+    private int samplingPercentage;
+    private int rowCount = 0;
+    private long[] rowHashCodesLong = null;
+    //about details of the new algorithm, please see KYLIN-2518
+    private boolean isUsePutRowKeyToHllNewAlgorithm;
+
+    private HLLCounter[] allCuboidsHLL = null;
+    private Long[] cuboidIds;
+    private Integer[][] allCuboidsBitSet = null;
+    private HashFunction hf = null;
+
+    RowKeyDecoder rowKeyDecoder;
+
+    protected Text outputKey = new Text();
+    protected Text outputValue = new Text();
+
+    @Override
+    protected void doSetup(Context context) throws IOException {
+        Configuration conf = context.getConfiguration();
+        HadoopUtil.setCurrentConfiguration(conf);
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+
+        String cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
+        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
+        CubeDesc cubeDesc = cube.getDescriptor();
+        CubeSegment cubeSegment = cube.getSegmentById(conf.get(BatchConstants.CFG_CUBE_SEGMENT_ID));
+
+        baseCuboidId = cube.getCuboidScheduler().getBaseCuboidId();
+        nRowKey = cubeDesc.getRowkey().getRowKeyColumns().length;
+
+        String cuboidModeName = conf.get(BatchConstants.CFG_CUBOID_MODE);
+        Set<Long> cuboidIdSet = cube.getCuboidsByMode(cuboidModeName);
+
+        cuboidIds = cuboidIdSet.toArray(new Long[cuboidIdSet.size()]);
+        allCuboidsBitSet = CuboidUtil.getCuboidBitSet(cuboidIds, nRowKey);
+
+        samplingPercentage = Integer
+                .parseInt(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT));
+
+        allCuboidsHLL = new HLLCounter[cuboidIds.length];
+        for (int i = 0; i < cuboidIds.length; i++) {
+            allCuboidsHLL[i] = new HLLCounter(cubeDesc.getConfig().getCubeStatsHLLPrecision());
+        }
+
+        //for KYLIN-2518 backward compatibility
+        if (KylinVersion.isBefore200(cubeDesc.getVersion())) {
+            isUsePutRowKeyToHllNewAlgorithm = false;
+            hf = Hashing.murmur3_32();
+            logger.info("Found KylinVersion : {}. Use old algorithm for cuboid sampling.", cubeDesc.getVersion());
+        } else {
+            isUsePutRowKeyToHllNewAlgorithm = true;
+            rowHashCodesLong = new long[nRowKey];
+            hf = Hashing.murmur3_128();
+            logger.info(
+                    "Found KylinVersion : {}. Use new algorithm for cuboid sampling. About the details of the new algorithm, please refer to KYLIN-2518",
+                    cubeDesc.getVersion());
+        }
+
+        rowKeyDecoder = new RowKeyDecoder(cubeSegment);
+    }
+
+    @Override
+    public void doMap(Text key, Text value, Context context) throws InterruptedException, IOException {
+        long cuboidID = rowKeyDecoder.decode(key.getBytes());
+        if (cuboidID != baseCuboidId) {
+            return; // Skip data from cuboids which are not the base cuboid
+        }
+
+        List<String> keyValues = rowKeyDecoder.getValues();
+
+        if (rowCount < samplingPercentage) {
+            Preconditions.checkArgument(nRowKey == keyValues.size());
+
+            String[] row = keyValues.toArray(new String[keyValues.size()]);
+            if (isUsePutRowKeyToHllNewAlgorithm) {
+                putRowKeyToHLLNew(row);
+            } else {
+                putRowKeyToHLLOld(row);
+            }
+        }
+
+        if (++rowCount == 100)
+            rowCount = 0;
+    }
+
+    public 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[i];
+            if (colValue != null) {
+                rowHashCodes[i] = hc.putString(colValue).hash().asBytes();
+            } else {
+                rowHashCodes[i] = hc.putInt(0).hash().asBytes();
+            }
+        }
+
+        // use the row key column hash to get a consolidated hash for each cuboid
+        for (int i = 0; i < cuboidIds.length; i++) {
+            Hasher hc = hf.newHasher();
+            for (int position = 0; position < allCuboidsBitSet[i].length; position++) {
+                hc.putBytes(rowHashCodes[allCuboidsBitSet[i][position]]);
+            }
+
+            allCuboidsHLL[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[i];
+            if (colValue == null)
+                colValue = "0";
+            byte[] bytes = hc.putString(colValue).hash().asBytes();
+            rowHashCodesLong[i] = (Bytes.toLong(bytes) + i);//add column ordinal to the hash value to distinguish between (a,b) and (b,a)
+        }
+
+        // user the row key column hash to get a consolidated hash for each cuboid
+        for (int i = 0, n = allCuboidsBitSet.length; i < n; i++) {
+            long value = 0;
+            for (int position = 0; position < allCuboidsBitSet[i].length; position++) {
+                value += rowHashCodesLong[allCuboidsBitSet[i][position]];
+            }
+            allCuboidsHLL[i].addHashDirectly(value);
+        }
+    }
+
+    @Override
+    protected void doCleanup(Context context) throws IOException, InterruptedException {
+        ByteBuffer hllBuf = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);
+        HLLCounter hll;
+        for (int i = 0; i < cuboidIds.length; i++) {
+            hll = allCuboidsHLL[i];
+
+            outputKey.set(Bytes.toBytes(cuboidIds[i]));
+            hllBuf.clear();
+            hll.writeRegisters(hllBuf);
+            outputValue.set(hllBuf.array(), 0, hllBuf.position());
+            context.write(outputKey, outputValue);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CalculateStatsFromBaseCuboidReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CalculateStatsFromBaseCuboidReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CalculateStatsFromBaseCuboidReducer.java
new file mode 100644
index 0000000..489dac4
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CalculateStatsFromBaseCuboidReducer.java
@@ -0,0 +1,112 @@
+/*
+ * 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.mr.steps;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.engine.mr.KylinReducer;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.CubeStatsWriter;
+import org.apache.kylin.measure.hllc.HLLCounter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class CalculateStatsFromBaseCuboidReducer extends KylinReducer<Text, Text, NullWritable, Text> {
+
+    private static final Logger logger = LoggerFactory.getLogger(CalculateStatsFromBaseCuboidReducer.class);
+
+    private KylinConfig cubeConfig;
+    protected long baseCuboidId;
+    protected Map<Long, HLLCounter> cuboidHLLMap = null;
+    private List<Long> baseCuboidRowCountInMappers;
+    private long totalRowsBeforeMerge = 0;
+
+    private String output = null;
+    private int samplingPercentage;
+
+    @Override
+    protected void doSetup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+
+        Configuration conf = context.getConfiguration();
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+        String cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
+        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
+        cubeConfig = cube.getConfig();
+
+        baseCuboidId = cube.getCuboidScheduler().getBaseCuboidId();
+        baseCuboidRowCountInMappers = Lists.newLinkedList();
+
+        output = conf.get(BatchConstants.CFG_OUTPUT_PATH);
+        samplingPercentage = Integer
+                .parseInt(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT));
+
+        cuboidHLLMap = Maps.newHashMap();
+    }
+
+    @Override
+    public void doReduce(Text key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+        long cuboidId = Bytes.toLong(key.getBytes());
+        logger.info("Cuboid id to be processed: " + cuboidId);
+        for (Text value : values) {
+            HLLCounter hll = new HLLCounter(cubeConfig.getCubeStatsHLLPrecision());
+            ByteBuffer bf = ByteBuffer.wrap(value.getBytes(), 0, value.getLength());
+            hll.readRegisters(bf);
+
+            if (cuboidId == baseCuboidId) {
+                baseCuboidRowCountInMappers.add(hll.getCountEstimate());
+            }
+
+            totalRowsBeforeMerge += hll.getCountEstimate();
+
+            if (cuboidHLLMap.get(cuboidId) != null) {
+                cuboidHLLMap.get(cuboidId).merge(hll);
+            } else {
+                cuboidHLLMap.put(cuboidId, hll);
+            }
+        }
+    }
+
+    @Override
+    protected void doCleanup(Context context) throws IOException, InterruptedException {
+        long grandTotal = 0;
+        for (HLLCounter hll : cuboidHLLMap.values()) {
+            grandTotal += hll.getCountEstimate();
+        }
+        double mapperOverlapRatio = grandTotal == 0 ? 0 : (double) totalRowsBeforeMerge / grandTotal;
+
+        CubeStatsWriter.writeCuboidStatistics(context.getConfiguration(), new Path(output), //
+                cuboidHLLMap, samplingPercentage, baseCuboidRowCountInMappers.size(), mapperOverlapRatio);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CopyDictionaryStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CopyDictionaryStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CopyDictionaryStep.java
new file mode 100644
index 0000000..3341be9
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CopyDictionaryStep.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr.steps;
+
+import java.io.IOException;
+
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.CubeUpdate;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+public class CopyDictionaryStep extends AbstractExecutable {
+
+    private static final Logger logger = LoggerFactory.getLogger(CopyDictionaryStep.class);
+
+    public CopyDictionaryStep() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final CubeManager mgr = CubeManager.getInstance(context.getConfig());
+        final CubeInstance cube = mgr.getCube(CubingExecutableUtil.getCubeName(this.getParams()));
+        final CubeSegment optimizeSegment = cube.getSegmentById(CubingExecutableUtil.getSegmentId(this.getParams()));
+
+        CubeSegment oldSegment = optimizeSegment.getCubeInstance().getOriginalSegmentToOptimize(optimizeSegment);
+        Preconditions.checkNotNull(oldSegment,
+                "cannot find the original segment to be optimized by " + optimizeSegment);
+
+        // --- Copy dictionary
+        optimizeSegment.getDictionaries().putAll(oldSegment.getDictionaries());
+        optimizeSegment.getSnapshots().putAll(oldSegment.getSnapshots());
+        optimizeSegment.getRowkeyStats().addAll(oldSegment.getRowkeyStats());
+
+        try {
+            CubeUpdate cubeBuilder = new CubeUpdate(cube);
+            cubeBuilder.setToUpdateSegs(optimizeSegment);
+            mgr.updateCube(cubeBuilder);
+        } catch (IOException e) {
+            logger.error("fail to merge dictionary or lookup snapshots", e);
+            return ExecuteResult.createError(e);
+        }
+
+        return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CubingExecutableUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CubingExecutableUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CubingExecutableUtil.java
index 65c5869..e06077a 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CubingExecutableUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CubingExecutableUtil.java
@@ -38,6 +38,7 @@ import javax.annotation.Nullable;
 public class CubingExecutableUtil {
 
     public static final String CUBE_NAME = "cubeName";
+    public static final String SEGMENT_NAME = "segmentName";
     public static final String SEGMENT_ID = "segmentId";
     public static final String MERGING_SEGMENT_IDS = "mergingSegmentIds";
     public static final String STATISTICS_PATH = "statisticsPath";
@@ -61,6 +62,14 @@ public class CubingExecutableUtil {
         return params.get(CUBE_NAME);
     }
 
+    public static void setSegmentName(String segmentName, Map<String, String> params) {
+        params.put(SEGMENT_NAME, segmentName);
+    }
+
+    public static String getSegmentName(Map<String, String> params) {
+        return params.get(SEGMENT_NAME);
+    }
+
     public static void setSegmentId(String segmentId, Map<String, String> params) {
         params.put(SEGMENT_ID, segmentId);
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
index 6a8ba4c..b49b639 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
@@ -28,12 +28,15 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidModeEnum;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
 import org.apache.kylin.engine.mr.CubingJob;
 import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
 import org.apache.kylin.engine.mr.IMROutput2;
 import org.apache.kylin.engine.mr.MRUtil;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.CuboidSchedulerUtil;
 import org.apache.kylin.job.execution.ExecutableManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -50,18 +53,27 @@ public class CuboidJob extends AbstractHadoopJob {
 
     private boolean skipped = false;
 
+    private CuboidScheduler cuboidScheduler;
+
     @Override
     public boolean isSkipped() {
         return skipped;
     }
 
-    private boolean checkSkip(String cubingJobId) {
+    private boolean checkSkip(String cubingJobId, int level) {
         if (cubingJobId == null)
             return false;
 
         ExecutableManager execMgr = ExecutableManager.getInstance(KylinConfig.getInstanceFromEnv());
         CubingJob cubingJob = (CubingJob) execMgr.getJob(cubingJobId);
         skipped = cubingJob.isLayerCubing() == false;
+        if (!skipped) {
+            skipped = (level > cuboidScheduler.getBuildLevel());
+            if (skipped) {
+                logger.info("Job level: " + level + " for " + cubingJobId + "[" + cubingJobId
+                        + "] exceeds real cuboid tree levels : " + cuboidScheduler.getBuildLevel());
+            }
+        }
         return skipped;
     }
 
@@ -80,6 +92,7 @@ public class CuboidJob extends AbstractHadoopJob {
             options.addOption(OPTION_OUTPUT_PATH);
             options.addOption(OPTION_NCUBOID_LEVEL);
             options.addOption(OPTION_CUBING_JOB_ID);
+            options.addOption(OPTION_CUBOID_MODE);
             parseOptions(options, args);
 
             String output = getOptionValue(OPTION_OUTPUT_PATH);
@@ -87,12 +100,18 @@ public class CuboidJob extends AbstractHadoopJob {
             int nCuboidLevel = Integer.parseInt(getOptionValue(OPTION_NCUBOID_LEVEL));
             String segmentID = getOptionValue(OPTION_SEGMENT_ID);
             String cubingJobId = getOptionValue(OPTION_CUBING_JOB_ID);
+            String cuboidModeName = getOptionValue(OPTION_CUBOID_MODE);
+            if (cuboidModeName == null) {
+                cuboidModeName = CuboidModeEnum.CURRENT.toString();
+            }
 
             CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
             CubeInstance cube = cubeMgr.getCube(cubeName);
             CubeSegment segment = cube.getSegmentById(segmentID);
 
-            if (checkSkip(cubingJobId)) {
+            cuboidScheduler = CuboidSchedulerUtil.getCuboidSchedulerByMode(segment, cuboidModeName);
+
+            if (checkSkip(cubingJobId, nCuboidLevel)) {
                 logger.info("Skip job " + getOptionValue(OPTION_JOB_NAME) + " for " + segmentID + "[" + segmentID + "]");
                 return 0;
             }
@@ -104,7 +123,7 @@ public class CuboidJob extends AbstractHadoopJob {
             setJobClasspath(job, cube.getConfig());
 
             // add metadata to distributed cache
-            attachSegmentMetadataWithDict(segment, job.getConfiguration());
+            attachSegmentMetadataWithAll(segment, job.getConfiguration());
 
             // Mapper
             job.setMapperClass(this.mapperClass);
@@ -122,12 +141,13 @@ public class CuboidJob extends AbstractHadoopJob {
 
             // set output
             IMROutput2.IMROutputFormat outputFormat = MRUtil.getBatchCubingOutputSide2(segment).getOuputFormat();
-            outputFormat.configureJobOutput(job, output, segment, nCuboidLevel);
+            outputFormat.configureJobOutput(job, output, segment, cuboidScheduler, nCuboidLevel);
 
             // set job configuration
             job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
             job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_ID, segmentID);
             job.getConfiguration().setInt(BatchConstants.CFG_CUBE_CUBOID_LEVEL, nCuboidLevel);
+            job.getConfiguration().set(BatchConstants.CFG_CUBOID_MODE, cuboidModeName);
 
             return waitForCompletion(job);
         } finally {

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java
index 2965e2f..e9fd3bd 100755
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java
@@ -21,13 +21,13 @@ package org.apache.kylin.engine.mr.steps;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collection;
-import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.io.Text;
 import org.apache.kylin.common.KylinVersion;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.StringUtil;
-import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.cuboid.CuboidUtil;
 import org.apache.kylin.engine.mr.common.BatchConstants;
 import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.hllc.HLLCounter;
@@ -37,7 +37,6 @@ import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Lists;
 import com.google.common.hash.HashFunction;
 import com.google.common.hash.Hasher;
 import com.google.common.hash.Hashing;
@@ -56,7 +55,6 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB
 
 
     protected boolean collectStatistics = false;
-    protected CuboidScheduler cuboidScheduler = null;
     protected int nRowKey;
     private Integer[][] allCuboidsBitSet = null;
     private HLLCounter[] allCuboidsHLL = null;
@@ -86,15 +84,11 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB
         collectStatistics = Boolean.parseBoolean(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_ENABLED));
         if (collectStatistics) {
             samplingPercentage = Integer.parseInt(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT));
-            cuboidScheduler = cubeDesc.getInitialCuboidScheduler();
             nRowKey = cubeDesc.getRowkey().getRowKeyColumns().length;
 
-            List<Long> cuboidIdList = Lists.newArrayList();
-            List<Integer[]> allCuboidsBitSetList = Lists.newArrayList();
-            addCuboidBitSet(baseCuboidId, allCuboidsBitSetList, cuboidIdList);
-
-            allCuboidsBitSet = allCuboidsBitSetList.toArray(new Integer[cuboidIdList.size()][]);
-            cuboidIds = cuboidIdList.toArray(new Long[cuboidIdList.size()]);
+            Set<Long> cuboidIdSet = cubeSeg.getCuboidScheduler().getAllCuboidIds();
+            cuboidIds = cuboidIdSet.toArray(new Long[cuboidIdSet.size()]);
+            allCuboidsBitSet = CuboidUtil.getCuboidBitSet(cuboidIds, nRowKey);
 
             allCuboidsHLL = new HLLCounter[cuboidIds.length];
             for (int i = 0; i < cuboidIds.length; i++) {
@@ -129,27 +123,6 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB
 
     }
 
-    private void addCuboidBitSet(long cuboidId, List<Integer[]> allCuboidsBitSet, List<Long> allCuboids) {
-        allCuboids.add(cuboidId);
-        Integer[] indice = new Integer[Long.bitCount(cuboidId)];
-
-        long mask = Long.highestOneBit(baseCuboidId);
-        int position = 0;
-        for (int i = 0; i < nRowKey; i++) {
-            if ((mask & cuboidId) > 0) {
-                indice[position] = i;
-                position++;
-            }
-            mask = mask >> 1;
-        }
-
-        allCuboidsBitSet.add(indice);
-        Collection<Long> children = cuboidScheduler.getSpanningCuboid(cuboidId);
-        for (Long childId : children) {
-            addCuboidBitSet(childId, allCuboidsBitSet, allCuboids);
-        }
-    }
-
     @Override
     public void doMap(KEYIN key, Object record, Context context) throws IOException, InterruptedException {
         Collection<String[]> rowCollection = flatTableInputFormat.parseMapperInput(record);

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
index 74c8c2c..0f65a3e 100755
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
@@ -110,6 +110,7 @@ public class FactDistinctColumnsReducer extends KylinReducer<SelfDefineSortableK
         if (collectStatistics && (taskId == numberOfTasks - 1)) {
             // hll
             isStatistics = true;
+            baseCuboidId = cube.getCuboidScheduler().getBaseCuboidId();
             baseCuboidRowCountInMappers = Lists.newArrayList();
             cuboidHLLMap = Maps.newHashMap();
             samplingPercentage = Integer.parseInt(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT));

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FilterRecommendCuboidDataJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FilterRecommendCuboidDataJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FilterRecommendCuboidDataJob.java
new file mode 100644
index 0000000..97f9dc1
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FilterRecommendCuboidDataJob.java
@@ -0,0 +1,103 @@
+/*
+ * 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.mr.steps;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FilterRecommendCuboidDataJob extends AbstractHadoopJob {
+
+    private static final Logger logger = LoggerFactory.getLogger(FilterRecommendCuboidDataJob.class);
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_SEGMENT_ID);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            parseOptions(options, args);
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            String segmentID = getOptionValue(OPTION_SEGMENT_ID);
+            Path input = new Path(getOptionValue(OPTION_INPUT_PATH));
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+
+            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            CubeSegment optSegment = cube.getSegmentById(segmentID);
+            CubeSegment originalSegment = cube.getOriginalSegmentToOptimize(optSegment);
+
+            logger.info("Starting: " + job.getJobName());
+
+            setJobClasspath(job, cube.getConfig());
+
+            // Mapper
+            job.setMapperClass(FilterRecommendCuboidDataMapper.class);
+
+            // Reducer
+            job.setNumReduceTasks(0);
+
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(Text.class);
+
+            // Input
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+            FileInputFormat.setInputPaths(job, input);
+            // Output
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            FileOutputFormat.setOutputPath(job, output);
+
+            // set job configuration
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_ID, segmentID);
+            // add metadata to distributed cache
+            attachSegmentMetadataWithDict(originalSegment, job.getConfiguration());
+
+            this.deletePath(job.getConfiguration(), output);
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            logger.error("error in CuboidJob", e);
+            printUsage(options);
+            throw e;
+        } finally {
+            if (job != null)
+                cleanupTempConfFile(job.getConfiguration());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FilterRecommendCuboidDataMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FilterRecommendCuboidDataMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FilterRecommendCuboidDataMapper.java
new file mode 100644
index 0000000..d8ae3ab
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FilterRecommendCuboidDataMapper.java
@@ -0,0 +1,107 @@
+/*
+ * 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.mr.steps;
+
+import static org.apache.kylin.engine.mr.JobBuilderSupport.PathNameCuboidBase;
+import static org.apache.kylin.engine.mr.JobBuilderSupport.PathNameCuboidOld;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.MultipleOutputs;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.common.RowKeySplitter;
+import org.apache.kylin.engine.mr.KylinMapper;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+
+import com.google.common.base.Preconditions;
+
+public class FilterRecommendCuboidDataMapper extends KylinMapper<Text, Text, Text, Text> {
+
+    private MultipleOutputs mos;
+
+    private RowKeySplitter rowKeySplitter;
+    private long baseCuboid;
+    private Set<Long> recommendCuboids;
+
+    @Override
+    protected void doSetup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+        mos = new MultipleOutputs(context);
+
+        String cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME);
+        String segmentID = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_ID);
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+
+        CubeManager cubeManager = CubeManager.getInstance(config);
+        CubeInstance cube = cubeManager.getCube(cubeName);
+        CubeSegment optSegment = cube.getSegmentById(segmentID);
+        CubeSegment originalSegment = cube.getOriginalSegmentToOptimize(optSegment);
+
+        rowKeySplitter = new RowKeySplitter(originalSegment, 65, 255);
+        baseCuboid = cube.getCuboidScheduler().getBaseCuboidId();
+
+        recommendCuboids = cube.getCuboidsRecommend();
+        Preconditions.checkNotNull(recommendCuboids, "The recommend cuboid map could not be null");
+    }
+
+    @Override
+    public void doMap(Text key, Text value, Context context) throws IOException, InterruptedException {
+        long cuboidID = rowKeySplitter.split(key.getBytes());
+        if (cuboidID != baseCuboid && !recommendCuboids.contains(cuboidID)) {
+            return;
+        }
+
+        String baseOutputPath = PathNameCuboidOld;
+        if (cuboidID == baseCuboid) {
+            baseOutputPath = PathNameCuboidBase;
+        }
+        mos.write(key, value, generateFileName(baseOutputPath));
+    }
+
+    @Override
+    public void doCleanup(Context context) throws IOException, InterruptedException {
+        mos.close();
+
+        Path outputDirBase = new Path(context.getConfiguration().get(FileOutputFormat.OUTDIR), PathNameCuboidBase);
+        FileSystem fs = FileSystem.get(context.getConfiguration());
+        if (!fs.exists(outputDirBase)) {
+            fs.mkdirs(outputDirBase);
+            SequenceFile
+                    .createWriter(context.getConfiguration(),
+                            SequenceFile.Writer.file(new Path(outputDirBase, "part-m-00000")),
+                            SequenceFile.Writer.keyClass(Text.class), SequenceFile.Writer.valueClass(Text.class))
+                    .close();
+        }
+    }
+
+    private String generateFileName(String subDir) {
+        return subDir + "/part";
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidJob.java
new file mode 100644
index 0000000..62109f4
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidJob.java
@@ -0,0 +1,154 @@
+/*
+ * 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.mr.steps;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidModeEnum;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.engine.mr.ByteArrayWritable;
+import org.apache.kylin.engine.mr.CubingJob;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.CuboidSchedulerUtil;
+import org.apache.kylin.engine.mr.common.MapReduceUtil;
+import org.apache.kylin.job.execution.ExecutableManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class InMemCuboidFromBaseCuboidJob extends AbstractHadoopJob {
+    protected static final Logger logger = LoggerFactory.getLogger(InMemCuboidFromBaseCuboidJob.class);
+
+    private boolean skipped = false;
+
+    @Override
+    public boolean isSkipped() {
+        return skipped;
+    }
+
+    private boolean checkSkip(String cubingJobId) {
+        if (cubingJobId == null)
+            return false;
+
+        ExecutableManager execMgr = ExecutableManager.getInstance(KylinConfig.getInstanceFromEnv());
+        CubingJob cubingJob = (CubingJob) execMgr.getJob(cubingJobId);
+        skipped = !cubingJob.isInMemCubing();
+        return skipped;
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_SEGMENT_ID);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_CUBING_JOB_ID);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_CUBOID_MODE);
+            parseOptions(options, args);
+
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            String segmentID = getOptionValue(OPTION_SEGMENT_ID);
+            String output = getOptionValue(OPTION_OUTPUT_PATH);
+
+            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            CubeSegment cubeSeg = cube.getSegmentById(segmentID);
+            String cubingJobId = getOptionValue(OPTION_CUBING_JOB_ID);
+
+            String cuboidModeName = getOptionValue(OPTION_CUBOID_MODE);
+            if (cuboidModeName == null) {
+                cuboidModeName = CuboidModeEnum.CURRENT.toString();
+            }
+
+            CuboidScheduler cuboidScheduler = CuboidSchedulerUtil.getCuboidSchedulerByMode(cubeSeg, cuboidModeName);
+
+            if (checkSkip(cubingJobId)) {
+                logger.info("Skip job " + getOptionValue(OPTION_JOB_NAME) + " for " + cubeSeg);
+                return 0;
+            }
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            logger.info("Starting: " + job.getJobName());
+
+            setJobClasspath(job, cube.getConfig());
+
+            // add metadata to distributed cache
+            attachSegmentMetadataWithAll(cubeSeg, job.getConfiguration());
+
+            // set job configuration
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_ID, segmentID);
+            job.getConfiguration().set(BatchConstants.CFG_CUBOID_MODE, cuboidModeName);
+
+            String input = getOptionValue(OPTION_INPUT_PATH);
+            FileInputFormat.setInputPaths(job, new Path(input));
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+
+            // set mapper
+            job.setMapperClass(InMemCuboidFromBaseCuboidMapper.class);
+            job.setMapOutputKeyClass(ByteArrayWritable.class);
+            job.setMapOutputValueClass(ByteArrayWritable.class);
+
+            // set output
+            job.setReducerClass(InMemCuboidFromBaseCuboidReducer.class);
+            job.setNumReduceTasks(MapReduceUtil.getInmemCubingReduceTaskNum(cubeSeg, cuboidScheduler));
+
+            // the cuboid file and KV class must be compatible with 0.7 version for smooth upgrade
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(Text.class);
+
+            Path outputPath = new Path(output);
+            FileOutputFormat.setOutputPath(job, outputPath);
+
+            HadoopUtil.deletePath(job.getConfiguration(), outputPath);
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            logger.error("error in CuboidJob", e);
+            printUsage(options);
+            throw e;
+        } finally {
+            if (job != null)
+                cleanupTempConfFile(job.getConfiguration());
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        InMemCuboidFromBaseCuboidJob job = new InMemCuboidFromBaseCuboidJob();
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidMapper.java
new file mode 100644
index 0000000..73a39d6
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidMapper.java
@@ -0,0 +1,96 @@
+/*
+ * 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.mr.steps;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.gridtable.CubeGridTable;
+import org.apache.kylin.cube.inmemcubing.AbstractInMemCubeBuilder;
+import org.apache.kylin.cube.inmemcubing.DoggedCubeBuilder;
+import org.apache.kylin.cube.inmemcubing.InputConverterUnit;
+import org.apache.kylin.cube.inmemcubing.InputConverterUnitForBaseCuboid;
+import org.apache.kylin.cube.kv.CubeDimEncMap;
+import org.apache.kylin.engine.mr.ByteArrayWritable;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public class InMemCuboidFromBaseCuboidMapper
+        extends InMemCuboidMapperBase<Text, Text, ByteArrayWritable, ByteArrayWritable, ByteArray> {
+    private static final Log logger = LogFactory.getLog(InMemCuboidFromBaseCuboidMapper.class);
+
+    private ByteBuffer keyValueBuffer;
+    private int keyOffset;
+
+    @Override
+    protected void doSetup(Mapper.Context context) throws IOException {
+        super.doSetup(context);
+
+        long baseCuboid = Cuboid.getBaseCuboidId(cubeDesc);
+        GTInfo gtInfo = CubeGridTable.newGTInfo(Cuboid.findForMandatory(cubeDesc, baseCuboid),
+                new CubeDimEncMap(cubeDesc, dictionaryMap));
+        keyValueBuffer = ByteBuffer.allocate(gtInfo.getMaxRecordLength());
+        keyOffset = cubeSegment.getRowKeyPreambleSize();
+    }
+
+    @Override
+    protected InputConverterUnit<ByteArray> getInputConverterUnit() {
+        return new InputConverterUnitForBaseCuboid();
+    }
+
+    @Override
+    protected Future getCubingThreadFuture(Context context, Map<TblColRef, Dictionary<String>> dictionaryMap,
+            int reserveMemoryMB, CuboidScheduler cuboidScheduler, InputConverterUnit<ByteArray> inputConverterUnit) {
+        AbstractInMemCubeBuilder cubeBuilder = new DoggedCubeBuilder(cuboidScheduler, flatDesc, dictionaryMap);
+        cubeBuilder.setReserveMemoryMB(reserveMemoryMB);
+        cubeBuilder.setConcurrentThreads(taskThreadCount);
+
+        ExecutorService executorService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder().setDaemon(true)
+                .setNameFormat("inmemory-cube-building-from-base-cuboid-mapper-%d").build());
+        return executorService.submit(cubeBuilder.buildAsRunnable(queue, inputConverterUnit,
+                new MapContextGTRecordWriter(context, cubeDesc, cubeSegment)));
+    }
+
+    @Override
+    protected ByteArray getRecordFromKeyValue(Text key, Text value) {
+        keyValueBuffer.clear();
+        keyValueBuffer.put(key.getBytes(), keyOffset, key.getBytes().length - keyOffset);
+        keyValueBuffer.put(value.getBytes());
+
+        byte[] keyValue = new byte[keyValueBuffer.position()];
+        System.arraycopy(keyValueBuffer.array(), 0, keyValue, 0, keyValueBuffer.position());
+
+        return new ByteArray(keyValue);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidReducer.java
new file mode 100644
index 0000000..fbc45d9
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidReducer.java
@@ -0,0 +1,23 @@
+/*
+ * 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.mr.steps;
+
+public class InMemCuboidFromBaseCuboidReducer extends InMemCuboidReducer {
+    //pass
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java
index 73a2eb9..b0ea7b7 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidJob.java
@@ -93,7 +93,7 @@ public class InMemCuboidJob extends AbstractHadoopJob {
             setJobClasspath(job, cube.getConfig());
 
             // add metadata to distributed cache
-            attachSegmentMetadataWithDict(segment, job.getConfiguration());
+            attachSegmentMetadataWithAll(segment, job.getConfiguration());
 
             // set job configuration
             job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
@@ -116,7 +116,7 @@ public class InMemCuboidJob extends AbstractHadoopJob {
 
             // set output
             IMROutput2.IMROutputFormat outputFormat = MRUtil.getBatchCubingOutputSide2(segment).getOuputFormat();
-            outputFormat.configureJobOutput(job, output, segment, 0);
+            outputFormat.configureJobOutput(job, output, segment, segment.getCuboidScheduler(), 0);
 
             return waitForCompletion(job);
         } finally {

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java
index 43f95e5..7b4738b 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java
@@ -33,8 +33,6 @@ import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.CuboidScheduler;
-import org.apache.kylin.cube.cuboid.DefaultCuboidScheduler;
-import org.apache.kylin.cube.cuboid.TreeCuboidSchedulerManager;
 import org.apache.kylin.cube.inmemcubing.ConsumeBlockingQueueController;
 import org.apache.kylin.cube.inmemcubing.InputConverterUnit;
 import org.apache.kylin.cube.model.CubeDesc;
@@ -43,7 +41,7 @@ import org.apache.kylin.engine.EngineFactory;
 import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.engine.mr.common.CuboidStatsReaderUtil;
+import org.apache.kylin.engine.mr.common.CuboidSchedulerUtil;
 import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
@@ -114,11 +112,7 @@ public abstract class InMemCuboidMapperBase<KEYIN, VALUEIN, KEYOUT, VALUEOUT, T>
         }
 
         String cuboidModeName = conf.get(BatchConstants.CFG_CUBOID_MODE);
-        CuboidScheduler cuboidScheduler = TreeCuboidSchedulerManager.getTreeCuboidScheduler(cubeDesc, //
-                CuboidStatsReaderUtil.readCuboidStatsFromSegment(cube.getCuboidsByMode(cuboidModeName), cubeSegment));
-        if (cuboidScheduler == null) {
-            cuboidScheduler = new DefaultCuboidScheduler(cubeDesc);
-        }
+        CuboidScheduler cuboidScheduler = CuboidSchedulerUtil.getCuboidSchedulerByMode(cubeSegment, cuboidModeName);
 
         taskThreadCount = config.getCubeAlgorithmInMemConcurrentThreads();
         reserveMemoryMB = calculateReserveMB(conf);

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KVGTRecordWriter.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KVGTRecordWriter.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KVGTRecordWriter.java
index 2058bc9..60d0870 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KVGTRecordWriter.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KVGTRecordWriter.java
@@ -95,7 +95,7 @@ public abstract class KVGTRecordWriter implements ICuboidWriter {
     protected abstract void writeAsKeyValue(ByteArrayWritable key, ByteArrayWritable value) throws IOException;
 
     private void initVariables(Long cuboidId) {
-        rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment, Cuboid.findById(cubeSegment, cuboidId));
+        rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment, Cuboid.findForMandatory(cubeDesc, cuboidId));
         keyBuf = rowKeyEncoder.createBuf();
 
         dimensions = Long.bitCount(cuboidId);

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
index 5a72faf..4f68293 100755
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
@@ -91,7 +91,7 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
     protected void doSetup(Context context) throws IOException, InterruptedException {
         super.bindCurrentConfiguration(context.getConfiguration());
 
-        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME);
         segmentID = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_ID);
 
         config = AbstractHadoopJob.loadKylinPropsAndMetadata();

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsWithOldStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsWithOldStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsWithOldStep.java
new file mode 100644
index 0000000..e97c6bb
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsWithOldStep.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr.steps;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.CubingJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.CubeStatsReader;
+import org.apache.kylin.engine.mr.common.CubeStatsWriter;
+import org.apache.kylin.engine.mr.common.StatisticsDecisionUtil;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.apache.kylin.measure.hllc.HLLCounter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+
+public class MergeStatisticsWithOldStep extends AbstractExecutable {
+
+    private static final Logger logger = LoggerFactory.getLogger(MergeStatisticsWithOldStep.class);
+
+    protected Map<Long, HLLCounter> cuboidHLLMap = Maps.newHashMap();
+
+    public MergeStatisticsWithOldStep() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final CubeManager mgr = CubeManager.getInstance(context.getConfig());
+        final CubeInstance cube = mgr.getCube(CubingExecutableUtil.getCubeName(this.getParams()));
+        final CubeSegment optimizeSegment = cube.getSegmentById(CubingExecutableUtil.getSegmentId(this.getParams()));
+        final String statsInputPath = CubingExecutableUtil.getStatisticsPath(this.getParams());
+
+        CubeSegment oldSegment = optimizeSegment.getCubeInstance().getOriginalSegmentToOptimize(optimizeSegment);
+        Preconditions.checkNotNull(oldSegment,
+                "cannot find the original segment to be optimized by " + optimizeSegment);
+
+        KylinConfig kylinConf = cube.getConfig();
+        Configuration conf = HadoopUtil.getCurrentConfiguration();
+        ResourceStore rs = ResourceStore.getStore(kylinConf);
+        int averageSamplingPercentage = 0;
+
+        try {
+            //1. Add statistics from optimized segment
+            Path statisticsFilePath = new Path(statsInputPath,
+                    BatchConstants.CFG_STATISTICS_CUBOID_ESTIMATION_FILENAME);
+            FileSystem hdfs = FileSystem.get(conf);
+            if (!hdfs.exists(statisticsFilePath))
+                throw new IOException("File " + statisticsFilePath + " does not exists");
+
+            CubeStatsReader optimizeSegmentStatsReader = new CubeStatsReader(optimizeSegment, null,
+                    optimizeSegment.getConfig(), statisticsFilePath);
+            averageSamplingPercentage += optimizeSegmentStatsReader.getSamplingPercentage();
+            addFromCubeStatsReader(optimizeSegmentStatsReader);
+
+            //2. Add statistics from old segment
+            CubeStatsReader oldSegmentStatsReader = new CubeStatsReader(oldSegment, null, oldSegment.getConfig());
+            averageSamplingPercentage += oldSegmentStatsReader.getSamplingPercentage();
+            addFromCubeStatsReader(oldSegmentStatsReader);
+
+            logger.info("Cuboid set with stats info: " + cuboidHLLMap.keySet().toString());
+            //3. Store merged statistics for recommend cuboids
+            averageSamplingPercentage = averageSamplingPercentage / 2;
+            Set<Long> cuboidsRecommend = cube.getCuboidsRecommend();
+
+            Map<Long, HLLCounter> resultCuboidHLLMap = Maps.newHashMapWithExpectedSize(cuboidsRecommend.size());
+            for (Long cuboid : cuboidsRecommend) {
+                HLLCounter hll = cuboidHLLMap.get(cuboid);
+                if (hll == null) {
+                    logger.warn("Cannot get the row count stats for cuboid " + cuboid);
+                } else {
+                    resultCuboidHLLMap.put(cuboid, hll);
+                }
+            }
+
+            String resultDir = CubingExecutableUtil.getMergedStatisticsPath(this.getParams());
+            CubeStatsWriter.writeCuboidStatistics(conf, new Path(resultDir), resultCuboidHLLMap,
+                    averageSamplingPercentage);
+
+            try (FSDataInputStream mergedStats = hdfs
+                    .open(new Path(resultDir, BatchConstants.CFG_STATISTICS_CUBOID_ESTIMATION_FILENAME))) {
+                // put the statistics to metadata store
+                String statisticsFileName = optimizeSegment.getStatisticsResourcePath();
+                rs.putResource(statisticsFileName, mergedStats, System.currentTimeMillis());
+            }
+
+            //By default, the cube optimization will use in-memory cubing
+            CubingJob cubingJob = (CubingJob) getManager()
+                    .getJob(CubingExecutableUtil.getCubingJobId(this.getParams()));
+            StatisticsDecisionUtil.decideCubingAlgorithm(cubingJob, optimizeSegment);
+
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+        } catch (IOException e) {
+            logger.error("fail to merge cuboid statistics", e);
+            return ExecuteResult.createError(e);
+        }
+
+    }
+
+    private void addFromCubeStatsReader(CubeStatsReader cubeStatsReader) {
+        for (Map.Entry<Long, HLLCounter> entry : cubeStatsReader.getCuboidRowHLLCounters().entrySet()) {
+            if (cuboidHLLMap.get(entry.getKey()) != null) {
+                cuboidHLLMap.get(entry.getKey()).merge(entry.getValue());
+            } else {
+                cuboidHLLMap.put(entry.getKey(), entry.getValue());
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
index 9f2a6ad..359bd4c 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
@@ -35,6 +35,7 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.CuboidSchedulerUtil;
 import org.apache.kylin.engine.mr.common.NDCuboidBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -50,8 +51,8 @@ public class NDCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
     private Text outputKey = new Text();
     private String cubeName;
     private String segmentID;
-    private CubeSegment cubeSegment;
     private CubeDesc cubeDesc;
+    private CubeSegment cubeSegment;
     private CuboidScheduler cuboidScheduler;
 
     private int handleCounter;
@@ -65,17 +66,18 @@ public class NDCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
     protected void doSetup(Context context) throws IOException {
         super.bindCurrentConfiguration(context.getConfiguration());
 
-        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME);
         segmentID = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_ID);
+        String cuboidModeName = context.getConfiguration().get(BatchConstants.CFG_CUBOID_MODE);
 
         KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
 
         CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
-        cubeSegment = cube.getSegmentById(segmentID);
         cubeDesc = cube.getDescriptor();
+        cubeSegment = cube.getSegmentById(segmentID);
         ndCuboidBuilder = new NDCuboidBuilder(cubeSegment);
         // initialize CubiodScheduler
-        cuboidScheduler = cubeSegment.getCuboidScheduler();
+        cuboidScheduler = CuboidSchedulerUtil.getCuboidSchedulerByMode(cubeSegment, cuboidModeName);
         rowKeySplitter = new RowKeySplitter(cubeSegment, 65, 256);
     }
 
@@ -104,7 +106,7 @@ public class NDCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
         }
 
         for (Long child : myChildren) {
-            Cuboid childCuboid = Cuboid.findById(cuboidScheduler, child);
+            Cuboid childCuboid = Cuboid.findForMandatory(cubeDesc, child);
             Pair<Integer, ByteArray> result = ndCuboidBuilder.buildKey(parentCuboid, childCuboid, rowKeySplitter.getSplitBuffers());
             outputKey.set(result.getSecond().array(), 0, result.getFirst());
             context.write(outputKey, value);


[14/18] kylin git commit: APACHE-KYLIN-2822: backend support for sunburst chart to show cuboid tree

Posted by li...@apache.org.
APACHE-KYLIN-2822: backend support for sunburst chart to show cuboid tree


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/c6e80936
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/c6e80936
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/c6e80936

Branch: refs/heads/ci-dong
Commit: c6e80936a764ebf4c92a84194f6eba28da96095c
Parents: d608fc1
Author: Zhong <nj...@apache.org>
Authored: Wed Aug 30 18:04:09 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../kylin/cube/cuboid/TreeCuboidScheduler.java  |   3 +-
 .../engine/mr/common/CuboidStatsReaderUtil.java |  10 +-
 .../kylin/rest/controller/CubeController.java   | 181 +++++++++++++++++++
 .../kylin/rest/response/CuboidTreeResponse.java | 123 +++++++++++++
 .../apache/kylin/rest/service/CubeService.java  |  87 +++++++++
 5 files changed, 399 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/c6e80936/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidScheduler.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidScheduler.java b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidScheduler.java
index 903e358..45b741e 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidScheduler.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidScheduler.java
@@ -19,7 +19,6 @@
 package org.apache.kylin.cube.cuboid;
 
 import java.io.PrintWriter;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -244,7 +243,7 @@ public class TreeCuboidScheduler extends CuboidScheduler {
         @JsonIgnore
         int level;
         @JsonProperty("children")
-        List<TreeNode> children = new ArrayList<>();
+        List<TreeNode> children = Lists.newArrayList();
 
         public long getCuboidId() {
             return cuboidId;

http://git-wip-us.apache.org/repos/asf/kylin/blob/c6e80936/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsReaderUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsReaderUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsReaderUtil.java
index 56ab504..1542aa2 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsReaderUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsReaderUtil.java
@@ -38,9 +38,13 @@ public class CuboidStatsReaderUtil {
 
     private static final Logger logger = LoggerFactory.getLogger(CuboidStatsReaderUtil.class);
 
-    public static Map<Long, Long> readCuboidStatsFromCube(Set<Long> cuboidIds, CubeInstance cubeInstance)
-            throws IOException {
-        Map<Long, Long> statisticsMerged = readCuboidStatsAndSizeFromCube(cuboidIds, cubeInstance).getFirst();
+    public static Map<Long, Long> readCuboidStatsFromCube(Set<Long> cuboidIds, CubeInstance cubeInstance) {
+        Map<Long, Long> statisticsMerged = null;
+        try {
+            statisticsMerged = readCuboidStatsAndSizeFromCube(cuboidIds, cubeInstance).getFirst();
+        } catch (IOException e) {
+            logger.warn("Fail to read statistics for cube " + cubeInstance.getName() + " due to " + e);
+        }
         return statisticsMerged.isEmpty() ? null : statisticsMerged;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/c6e80936/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
index 77bd498..aa59e30 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
@@ -21,21 +21,30 @@ package org.apache.kylin.rest.controller;
 import static org.apache.kylin.rest.service.CubeService.VALID_CUBENAME;
 
 import java.io.IOException;
+import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
 
+import javax.servlet.http.HttpServletResponse;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.cuboid.TreeCuboidScheduler;
 import org.apache.kylin.cube.model.CubeBuildTypeEnum;
 import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.RowKeyColDesc;
 import org.apache.kylin.dimension.DimensionEncodingFactory;
 import org.apache.kylin.engine.EngineFactory;
+import org.apache.kylin.engine.mr.common.CuboidStatsReaderUtil;
 import org.apache.kylin.job.JobInstance;
 import org.apache.kylin.job.JoinedFlatTable;
 import org.apache.kylin.job.exception.JobException;
@@ -45,6 +54,8 @@ import org.apache.kylin.metadata.model.SegmentRange;
 import org.apache.kylin.metadata.model.SegmentRange.TSRange;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
+import org.apache.kylin.metrics.MetricsManager;
+import org.apache.kylin.metrics.property.QueryCubePropertyEnum;
 import org.apache.kylin.rest.exception.BadRequestException;
 import org.apache.kylin.rest.exception.ForbiddenException;
 import org.apache.kylin.rest.exception.InternalErrorException;
@@ -53,6 +64,8 @@ import org.apache.kylin.rest.request.CubeRequest;
 import org.apache.kylin.rest.request.JobBuildRequest;
 import org.apache.kylin.rest.request.JobBuildRequest2;
 import org.apache.kylin.rest.request.JobOptimizeRequest;
+import org.apache.kylin.rest.request.SQLRequest;
+import org.apache.kylin.rest.response.CuboidTreeResponse;
 import org.apache.kylin.rest.response.EnvelopeResponse;
 import org.apache.kylin.rest.response.GeneralResponse;
 import org.apache.kylin.rest.response.HBaseResponse;
@@ -60,6 +73,7 @@ import org.apache.kylin.rest.response.ResponseCode;
 import org.apache.kylin.rest.service.CubeService;
 import org.apache.kylin.rest.service.JobService;
 import org.apache.kylin.rest.service.ProjectService;
+import org.apache.kylin.rest.service.QueryService;
 import org.apache.kylin.source.kafka.util.KafkaClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -81,6 +95,7 @@ import com.fasterxml.jackson.databind.JsonMappingException;
 import com.google.common.base.Joiner;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 /**
  * CubeController is defined as Restful API entrance for UI.
@@ -102,6 +117,10 @@ public class CubeController extends BasicController {
     @Qualifier("projectService")
     private ProjectService projectService;
 
+    @Autowired
+    @Qualifier("queryService")
+    private QueryService queryService;
+
     @RequestMapping(value = "/validate/{cubeName}", method = RequestMethod.GET, produces = { "application/json" })
     @ResponseBody
     public EnvelopeResponse<Boolean> validateModelName(@PathVariable String cubeName) {
@@ -768,6 +787,168 @@ public class CubeController extends BasicController {
 
     }
 
+    @RequestMapping(value = "/{cubeName}/cuboids/export", method = RequestMethod.GET)
+    @ResponseBody
+    public void cuboidsExport(@PathVariable String cubeName, @RequestParam(value = "top") Integer top,
+            HttpServletResponse response) throws IOException {
+        CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
+        if (cube == null) {
+            logger.error("Get cube: [" + cubeName + "] failed when get recommend cuboids");
+            throw new BadRequestException("Get cube: [" + cubeName + "] failed when get recommend cuboids");
+        }
+        Map<Long, Long> cuboidList = getRecommendCuboidList(cube);
+        if (cuboidList == null || cuboidList.isEmpty()) {
+            logger.warn("Cannot get recommend cuboid list for cube " + cubeName);
+        }
+        if (cuboidList.size() < top) {
+            logger.info("Only recommend " + cuboidList.size() + " cuboids less than topn " + top);
+        }
+        Iterator<Long> cuboidIterator = cuboidList.keySet().iterator();
+        RowKeyColDesc[] rowKeyColDescList = cube.getDescriptor().getRowkey().getRowKeyColumns();
+
+        List<Set<String>> dimensionSetList = Lists.newLinkedList();
+        while (top-- > 0 && cuboidIterator.hasNext()) {
+            Set<String> dimensionSet = Sets.newHashSet();
+            dimensionSetList.add(dimensionSet);
+            long cuboid = cuboidIterator.next();
+            for (int i = 0; i < rowKeyColDescList.length; i++) {
+                if ((cuboid & (1L << rowKeyColDescList[i].getBitIndex())) > 0) {
+                    dimensionSet.add(rowKeyColDescList[i].getColumn());
+                }
+            }
+        }
+
+        response.setContentType("text/json;charset=utf-8");
+        response.setHeader("Content-Disposition", "attachment; filename=\"" + cubeName + ".json\"");
+        try (PrintWriter writer = response.getWriter()) {
+            writer.write(JsonUtil.writeValueAsString(dimensionSetList));
+        } catch (IOException e) {
+            logger.error("", e);
+            throw new InternalErrorException("Failed to write: " + e.getLocalizedMessage());
+        }
+    }
+
+    @RequestMapping(value = "/{cubeName}/cuboids/current", method = RequestMethod.GET)
+    @ResponseBody
+    public CuboidTreeResponse getCurrentCuboids(@PathVariable String cubeName) {
+        CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
+        if (cube == null) {
+            logger.error("Get cube: [" + cubeName + "] failed when get current cuboids");
+            throw new BadRequestException("Get cube: [" + cubeName + "] failed when get current cuboids");
+        }
+        // The cuboid tree displayed should be consistent with the current one
+        CuboidScheduler cuboidScheduler = cube.getCuboidScheduler();
+        Map<Long, Long> cuboidStatsMap = cube.getCuboids();
+        if (cuboidStatsMap == null) {
+            cuboidStatsMap = CuboidStatsReaderUtil.readCuboidStatsFromCube(cuboidScheduler.getAllCuboidIds(), cube);
+        }
+
+        Map<Long, Long> hitFrequencyMap = null;
+        Map<Long, Long> queryMatchMap = null;
+        try {
+            hitFrequencyMap = getTargetCuboidHitFrequency(cubeName);
+            queryMatchMap = getCuboidQueryMatchCount(cubeName);
+        } catch (Exception e) {
+            logger.warn("Fail to query on system cube due to " + e);
+        }
+
+        Set<Long> currentCuboidSet = cube.getCuboidScheduler().getAllCuboidIds();
+        return cubeService.getCuboidTreeResponse(cuboidScheduler, cuboidStatsMap, hitFrequencyMap, queryMatchMap,
+                currentCuboidSet);
+    }
+
+    @RequestMapping(value = "/{cubeName}/cuboids/recommend", method = RequestMethod.GET)
+    @ResponseBody
+    public CuboidTreeResponse getRecommendCuboids(@PathVariable String cubeName) throws IOException {
+        CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
+        if (cube == null) {
+            logger.error("Get cube: [" + cubeName + "] failed when get recommend cuboids");
+            throw new BadRequestException("Get cube: [" + cubeName + "] failed when get recommend cuboids");
+        }
+        Map<Long, Long> recommendCuboidStatsMap = getRecommendCuboidList(cube);
+        if (recommendCuboidStatsMap == null || recommendCuboidStatsMap.isEmpty()) {
+            return new CuboidTreeResponse();
+        }
+        CuboidScheduler cuboidScheduler = new TreeCuboidScheduler(cube.getDescriptor(),
+                Lists.newArrayList(recommendCuboidStatsMap.keySet()),
+                new TreeCuboidScheduler.CuboidCostComparator(recommendCuboidStatsMap));
+
+        // Get cuboid target info for displaying heat map of cuboid hit
+        Map<Long, Long> displayHitFrequencyMap = getTargetCuboidHitFrequency(cubeName);
+        // Get exactly matched cuboid query count
+        Map<Long, Long> queryMatchMap = getCuboidQueryMatchCount(cubeName);
+
+        Set<Long> currentCuboidSet = cube.getCuboidScheduler().getAllCuboidIds();
+        return cubeService.getCuboidTreeResponse(cuboidScheduler, recommendCuboidStatsMap, displayHitFrequencyMap,
+                queryMatchMap, currentCuboidSet);
+    }
+
+    private Map<Long, Long> getRecommendCuboidList(CubeInstance cube) throws IOException {
+        // Get cuboid source info
+        Map<Long, Long> optimizeHitFrequencyMap = getSourceCuboidHitFrequency(cube.getName());
+        Map<Long, Map<Long, Long>> rollingUpCountSourceMap = getCuboidRollingUpCount(cube.getName());
+        return cubeService.getRecommendCuboidStatistics(cube, optimizeHitFrequencyMap, rollingUpCountSourceMap);
+    }
+
+    private Map<Long, Long> getSourceCuboidHitFrequency(String cubeName) {
+        return getCuboidHitFrequency(cubeName, true);
+    }
+
+    private Map<Long, Long> getTargetCuboidHitFrequency(String cubeName) {
+        return getCuboidHitFrequency(cubeName, false);
+    }
+
+    private Map<Long, Long> getCuboidHitFrequency(String cubeName, boolean ifSource) {
+        SQLRequest sqlRequest = new SQLRequest();
+        sqlRequest.setProject(MetricsManager.SYSTEM_PROJECT);
+        String cuboidColumn = ifSource ? QueryCubePropertyEnum.CUBOID_SOURCE.toString()
+                : QueryCubePropertyEnum.CUBOID_TARGET.toString();
+        String hitMeasure = QueryCubePropertyEnum.WEIGHT_PER_HIT.toString();
+        String table = cubeService.getMetricsManager()
+                .getSystemTableFromSubject(cubeService.getConfig().getKylinMetricsSubjectQueryCube());
+        String sql = "select " + cuboidColumn + ", sum(" + hitMeasure + ") " //
+                + "from " + table//
+                + " where " + QueryCubePropertyEnum.CUBE.toString() + " = '" + cubeName + "' " //
+                + "group by " + cuboidColumn;
+        sqlRequest.setSql(sql);
+        List<List<String>> orgHitFrequency = queryService.queryWithoutSecure(sqlRequest).getResults();
+        return cubeService.formatQueryCount(orgHitFrequency);
+    }
+
+    private Map<Long, Map<Long, Long>> getCuboidRollingUpCount(String cubeName) {
+        SQLRequest sqlRequest = new SQLRequest();
+        sqlRequest.setProject(MetricsManager.SYSTEM_PROJECT);
+        String cuboidSource = QueryCubePropertyEnum.CUBOID_SOURCE.toString();
+        String cuboidTarget = QueryCubePropertyEnum.CUBOID_TARGET.toString();
+        String aggCount = QueryCubePropertyEnum.AGGR_COUNT.toString();
+        String table = cubeService.getMetricsManager()
+                .getSystemTableFromSubject(cubeService.getConfig().getKylinMetricsSubjectQueryCube());
+        String sql = "select " + cuboidSource + ", " + cuboidTarget + ", sum(" + aggCount + ")/count(*) " //
+                + "from " + table //
+                + " where " + QueryCubePropertyEnum.CUBE.toString() + " = '" + cubeName + "' " //
+                + "group by " + cuboidSource + ", " + cuboidTarget;
+        sqlRequest.setSql(sql);
+        List<List<String>> orgRollingUpCount = queryService.queryWithoutSecure(sqlRequest).getResults();
+        return cubeService.formatRollingUpCount(orgRollingUpCount);
+    }
+
+    private Map<Long, Long> getCuboidQueryMatchCount(String cubeName) {
+        SQLRequest sqlRequest = new SQLRequest();
+        sqlRequest.setProject(MetricsManager.SYSTEM_PROJECT);
+        String cuboidSource = QueryCubePropertyEnum.CUBOID_SOURCE.toString();
+        String hitMeasure = QueryCubePropertyEnum.WEIGHT_PER_HIT.toString();
+        String table = cubeService.getMetricsManager()
+                .getSystemTableFromSubject(cubeService.getConfig().getKylinMetricsSubjectQueryCube());
+        String sql = "select " + cuboidSource + ", sum(" + hitMeasure + ") " //
+                + "from " + table //
+                + " where " + QueryCubePropertyEnum.CUBE.toString() + " = '" + cubeName + "' and "
+                + QueryCubePropertyEnum.IF_MATCH.toString() + " = true " //
+                + "group by " + cuboidSource;
+        sqlRequest.setSql(sql);
+        List<List<String>> orgMatchHitFrequency = queryService.queryWithoutSecure(sqlRequest).getResults();
+        return cubeService.formatQueryCount(orgMatchHitFrequency);
+    }
+
     /**
      * Initiate the very beginning of a streaming cube. Will seek the latest offests of each partition from streaming
      * source (kafka) and record in the cube descriptor; In the first build job, it will use these offests as the start point.

http://git-wip-us.apache.org/repos/asf/kylin/blob/c6e80936/server-base/src/main/java/org/apache/kylin/rest/response/CuboidTreeResponse.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/response/CuboidTreeResponse.java b/server-base/src/main/java/org/apache/kylin/rest/response/CuboidTreeResponse.java
new file mode 100644
index 0000000..b416084
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/response/CuboidTreeResponse.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.rest.response;
+
+import java.io.Serializable;
+import java.util.List;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Lists;
+
+public class CuboidTreeResponse implements Serializable {
+
+    private static final long serialVersionUID = 2835980715891990832L;
+
+    private NodeInfo root;
+
+    public NodeInfo getRoot() {
+        return root;
+    }
+
+    public void setRoot(NodeInfo root) {
+        this.root = root;
+    }
+
+    public static class NodeInfo {
+        @JsonProperty("cuboid_id")
+        private Long id;
+        @JsonProperty("name")
+        private String name;
+        @JsonProperty("query_count")
+        private Long queryCount;
+        @JsonProperty("query_rate")
+        private Float queryRate;
+        @JsonProperty("exactly_match_count")
+        private Long exactlyMatchCount;
+        @JsonProperty("row_count")
+        private Long rowCount;
+        @JsonProperty("existed")
+        private Boolean existed;
+        @JsonProperty("children")
+        List<NodeInfo> children = Lists.newArrayList();
+
+        public Long getId() {
+            return id;
+        }
+
+        public void setId(Long id) {
+            this.id = id;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        public Long getQueryCount() {
+            return queryCount;
+        }
+
+        public void setQueryCount(Long queryCount) {
+            this.queryCount = queryCount;
+        }
+
+        public Float getQueryRate() {
+            return queryRate;
+        }
+
+        public void setQueryRate(Float queryRate) {
+            this.queryRate = queryRate;
+        }
+
+        public Long getExactlyMatchCount() {
+            return exactlyMatchCount;
+        }
+
+        public void setExactlyMatchCount(Long exactlyMatchCount) {
+            this.exactlyMatchCount = exactlyMatchCount;
+        }
+
+        public Long getRowCount() {
+            return rowCount;
+        }
+
+        public void setRowCount(Long rowCount) {
+            this.rowCount = rowCount;
+        }
+
+        public Boolean getExisted() {
+            return existed;
+        }
+
+        public void setExisted(Boolean existed) {
+            this.existed = existed;
+        }
+
+        public void addChild(NodeInfo child) {
+            this.children.add(child);
+        }
+
+        public List<NodeInfo> getChildren() {
+            return children;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/c6e80936/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
index d5805a1..b86ff1d 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -24,6 +24,8 @@ import java.util.Collections;
 import java.util.Date;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.KylinConfig;
@@ -32,10 +34,13 @@ import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.CubeUpdate;
+import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.cuboid.CuboidCLI;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.engine.EngineFactory;
 import org.apache.kylin.engine.mr.CubingJob;
+import org.apache.kylin.engine.mr.common.CuboidRecommenderUtil;
 import org.apache.kylin.job.exception.JobException;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
@@ -55,6 +60,8 @@ import org.apache.kylin.rest.exception.ForbiddenException;
 import org.apache.kylin.rest.msg.Message;
 import org.apache.kylin.rest.msg.MsgPicker;
 import org.apache.kylin.rest.request.MetricsRequest;
+import org.apache.kylin.rest.response.CuboidTreeResponse;
+import org.apache.kylin.rest.response.CuboidTreeResponse.NodeInfo;
 import org.apache.kylin.rest.response.HBaseResponse;
 import org.apache.kylin.rest.response.MetricsResponse;
 import org.apache.kylin.rest.security.AclPermission;
@@ -761,4 +768,84 @@ public class CubeService extends BasicService implements InitializingBean {
             }
         }
     }
+
+    public CuboidTreeResponse getCuboidTreeResponse(CuboidScheduler cuboidScheduler, Map<Long, Long> rowCountMap,
+            Map<Long, Long> hitFrequencyMap, Map<Long, Long> queryMatchMap, Set<Long> currentCuboidSet) {
+        long baseCuboidId = cuboidScheduler.getBaseCuboidId();
+        int dimensionCount = Long.bitCount(baseCuboidId);
+
+        // get cube query count total
+        long cubeQueryCount = 0L;
+        if (hitFrequencyMap != null) {
+            for (long queryCount : hitFrequencyMap.values()) {
+                cubeQueryCount += queryCount;
+            }
+        }
+
+        NodeInfo root = generateNodeInfo(baseCuboidId, dimensionCount, cubeQueryCount, rowCountMap, hitFrequencyMap,
+                queryMatchMap, currentCuboidSet);
+
+        List<NodeInfo> nodeQueue = Lists.newLinkedList();
+        nodeQueue.add(root);
+        while (!nodeQueue.isEmpty()) {
+            NodeInfo parentNode = nodeQueue.remove(0);
+            for (long childId : cuboidScheduler.getSpanningCuboid(parentNode.getId())) {
+                NodeInfo childNode = generateNodeInfo(childId, dimensionCount, cubeQueryCount, rowCountMap,
+                        hitFrequencyMap, queryMatchMap, currentCuboidSet);
+                parentNode.addChild(childNode);
+                nodeQueue.add(childNode);
+            }
+        }
+
+        CuboidTreeResponse result = new CuboidTreeResponse();
+        result.setRoot(root);
+        return result;
+    }
+
+    private NodeInfo generateNodeInfo(long cuboidId, int dimensionCount, long cubeQueryCount,
+            Map<Long, Long> rowCountMap, Map<Long, Long> hitFrequencyMap, Map<Long, Long> queryMatchMap,
+            Set<Long> currentCuboidSet) {
+        Long queryCount = hitFrequencyMap == null || hitFrequencyMap.get(cuboidId) == null ? 0L
+                : hitFrequencyMap.get(cuboidId);
+        float queryRate = cubeQueryCount <= 0 ? 0 : queryCount.floatValue() / cubeQueryCount;
+        long queryExactlyMatchCount = queryMatchMap == null || queryMatchMap.get(cuboidId) == null ? 0L
+                : queryMatchMap.get(cuboidId);
+        boolean ifExist = currentCuboidSet.contains(cuboidId);
+        long rowCount = rowCountMap == null ? 0L : rowCountMap.get(cuboidId);
+
+        NodeInfo node = new NodeInfo();
+        node.setId(cuboidId);
+        node.setName(Cuboid.getDisplayName(cuboidId, dimensionCount));
+        node.setQueryCount(queryCount);
+        node.setQueryRate(queryRate);
+        node.setExactlyMatchCount(queryExactlyMatchCount);
+        node.setExisted(ifExist);
+        node.setRowCount(rowCount);
+        return node;
+    }
+
+    /** cube planner services */
+    public Map<Long, Long> formatQueryCount(List<List<String>> orgQueryCount) {
+        Map<Long, Long> formattedQueryCount = Maps.newLinkedHashMap();
+        for (List<String> hit : orgQueryCount) {
+            formattedQueryCount.put(Long.parseLong(hit.get(0)), (long) Double.parseDouble(hit.get(1)));
+        }
+        return formattedQueryCount;
+    }
+
+    public Map<Long, Map<Long, Long>> formatRollingUpCount(List<List<String>> orgRollingUpCount) {
+        Map<Long, Map<Long, Long>> formattedRollingUpCount = Maps.newLinkedHashMap();
+        for (List<String> rollingUp : orgRollingUpCount) {
+            Map<Long, Long> childMap = Maps.newLinkedHashMap();
+            childMap.put(Long.parseLong(rollingUp.get(1)), (long) Double.parseDouble(rollingUp.get(2)));
+            formattedRollingUpCount.put(Long.parseLong(rollingUp.get(0)), childMap);
+        }
+        return formattedRollingUpCount;
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION')")
+    public Map<Long, Long> getRecommendCuboidStatistics(CubeInstance cube, Map<Long, Long> hitFrequencyMap,
+            Map<Long, Map<Long, Long>> rollingUpCountSourceMap) throws IOException {
+        return CuboidRecommenderUtil.getRecommendCuboidList(cube, hitFrequencyMap, rollingUpCountSourceMap);
+    }
 }


[12/18] kylin git commit: APACHE-KYLIN-2732: add option for InMemCuboidFromBaseCuboidJob to indicate whether collect base cuboid or not

Posted by li...@apache.org.
APACHE-KYLIN-2732: add option for InMemCuboidFromBaseCuboidJob to indicate whether collect base cuboid or not

Signed-off-by: Zhong <nj...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/e849cc2a
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/e849cc2a
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/e849cc2a

Branch: refs/heads/ci-dong
Commit: e849cc2aae8d4aba04cadfd1830427ea1d1ee5bf
Parents: 1b6d8fe
Author: Ma Gang <mg...@163.com>
Authored: Thu Nov 9 15:20:51 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../cube/inmemcubing/InMemCubeBuilder.java      |  6 +----
 .../cube/inmemcubing/InputConverterUnit.java    | 13 ++++++++--
 .../InputConverterUnitForBaseCuboid.java        | 25 ++++++++++++++++----
 .../InputConverterUnitForRawData.java           | 20 ++++++++++++----
 .../engine/mr/common/AbstractHadoopJob.java     |  3 +++
 .../kylin/engine/mr/common/BatchConstants.java  |  2 ++
 .../mr/steps/InMemCuboidFromBaseCuboidJob.java  |  6 +++++
 .../steps/InMemCuboidFromBaseCuboidMapper.java  | 12 +++++++---
 .../engine/mr/steps/InMemCuboidMapper.java      |  5 ++--
 .../engine/mr/steps/InMemCuboidMapperBase.java  | 14 +++++------
 .../inmemcubing/ITInMemCubeBuilderTest.java     |  2 +-
 11 files changed, 77 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/e849cc2a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
index 97bb1de..3a4cf4c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
@@ -87,7 +87,6 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
     private CuboidResult baseResult;
     private Object[] totalSumForSanityCheck;
     private ICuboidCollector resultCollector;
-    private boolean ifBaseCuboidCollected = true;
 
     public InMemCubeBuilder(CuboidScheduler cuboidScheduler, IJoinedFlatTableDesc flatDesc,
             Map<TblColRef, Dictionary<String>> dictionaryMap) {
@@ -138,9 +137,6 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
 
     public <T> ConcurrentNavigableMap<Long, CuboidResult> build(RecordConsumeBlockingQueueController<T> input)
             throws IOException {
-        if (input.inputConverterUnit instanceof InputConverterUnitForBaseCuboid) {
-            ifBaseCuboidCollected = false;
-        }
         final ConcurrentNavigableMap<Long, CuboidResult> result = new ConcurrentSkipListMap<Long, CuboidResult>();
         build(input, new ICuboidCollector() {
             @Override
@@ -364,7 +360,7 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
         int mbEstimateBaseAggrCache = (int) (aggregationScanner.getEstimateSizeOfAggrCache() / MemoryBudgetController.ONE_MB);
         logger.info("Wild estimate of base aggr cache is " + mbEstimateBaseAggrCache + " MB");
 
-        return updateCuboidResult(baseCuboidId, baseCuboid, count, timeSpent, 0, ifBaseCuboidCollected);
+        return updateCuboidResult(baseCuboidId, baseCuboid, count, timeSpent, 0, input.inputConverterUnit.ifChange());
     }
 
     private CuboidResult updateCuboidResult(long cuboidId, GridTable table, int nRows, long timeSpent, int aggrCacheMB) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/e849cc2a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnit.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnit.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnit.java
index fe32937..aa1c9f4 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnit.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnit.java
@@ -21,13 +21,22 @@ package org.apache.kylin.cube.inmemcubing;
 import org.apache.kylin.gridtable.GTRecord;
 
 public interface InputConverterUnit<T> {
+
+    /** Convert currentObject to a GTRecord*/
     public void convert(T currentObject, GTRecord record);
 
+    /** Check if currentObject is for indicating the end of the data stream*/
     public boolean ifEnd(T currentObject);
 
+    /** Check if currentObject is for cutting the data stream*/
     public boolean ifCut(T currentObject);
 
-    public T getEmptyUnit();
+    /** Get the object indicating the end of the data stream*/
+    public T getEndRow();
+
+    /** Get the object for cutting the data stream*/
+    public T getCutRow();
 
-    public T getCutUnit();
+    /** Get whether the input source is different from the final output cuboid*/
+    public boolean ifChange();
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e849cc2a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForBaseCuboid.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForBaseCuboid.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForBaseCuboid.java
index 9110a87..be1a38e 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForBaseCuboid.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForBaseCuboid.java
@@ -23,22 +23,32 @@ import org.apache.kylin.gridtable.GTRecord;
 
 public class InputConverterUnitForBaseCuboid implements InputConverterUnit<ByteArray> {
 
-    public static final ByteArray EMPTY_ROW = new ByteArray();
+    public static final ByteArray END_ROW = new ByteArray();
     public static final ByteArray CUT_ROW = new ByteArray(0);
 
+    private final boolean ifChange;
+
+    public InputConverterUnitForBaseCuboid(boolean ifChange) {
+        this.ifChange = ifChange;
+    }
+
+    @Override
     public void convert(ByteArray currentObject, GTRecord record) {
         record.loadColumns(currentObject.asBuffer());
     }
 
+    @Override
     public boolean ifEnd(ByteArray currentObject) {
-        return currentObject == EMPTY_ROW;
+        return currentObject == END_ROW;
     }
 
-    public ByteArray getEmptyUnit() {
-        return EMPTY_ROW;
+    @Override
+    public ByteArray getEndRow() {
+        return END_ROW;
     }
 
-    public ByteArray getCutUnit() {
+    @Override
+    public ByteArray getCutRow() {
         return CUT_ROW;
     }
 
@@ -46,4 +56,9 @@ public class InputConverterUnitForBaseCuboid implements InputConverterUnit<ByteA
     public boolean ifCut(ByteArray currentObject) {
         return currentObject == CUT_ROW;
     }
+
+    @Override
+    public boolean ifChange() {
+        return ifChange;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/e849cc2a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForRawData.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForRawData.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForRawData.java
index f6548b2..fc34f37 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForRawData.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForRawData.java
@@ -44,7 +44,7 @@ public class InputConverterUnitForRawData implements InputConverterUnit<String[]
     private static final Logger logger = LoggerFactory.getLogger(InputConverterUnitForRawData.class);
     
     public static final byte[] HIVE_NULL = Bytes.toBytes("\\N");
-    public static final String[] EMPTY_ROW = new String[0];
+    public static final String[] END_ROW = new String[0];
     public static final String[] CUT_ROW = { "" };
 
     private final CubeJoinedFlatTableEnrich flatDesc;
@@ -64,6 +64,7 @@ public class InputConverterUnitForRawData implements InputConverterUnit<String[]
         initNullBytes(cubeDesc);
     }
 
+    @Override
     public final void convert(String[] row, GTRecord record) {
         Object[] dimensions = buildKey(row);
         Object[] metricsValues = buildValue(row);
@@ -73,19 +74,23 @@ public class InputConverterUnitForRawData implements InputConverterUnit<String[]
         record.setValues(recordValues);
     }
 
+    @Override
     public boolean ifEnd(String[] currentObject) {
-        return currentObject == EMPTY_ROW;
+        return currentObject == END_ROW;
     }
 
+    @Override
     public boolean ifCut(String[] currentObject) {
         return currentObject == CUT_ROW;
     }
 
-    public String[] getEmptyUnit() {
-        return EMPTY_ROW;
+    @Override
+    public String[] getEndRow() {
+        return END_ROW;
     }
 
-    public String[] getCutUnit() {
+    @Override
+    public String[] getCutRow() {
         return CUT_ROW;
     }
 
@@ -103,6 +108,11 @@ public class InputConverterUnitForRawData implements InputConverterUnit<String[]
         return key;
     }
 
+    @Override
+    public boolean ifChange() {
+        return true;
+    }
+
     private Object[] buildValue(String[] row) {
         Object[] values = new Object[measureCount];
         for (int i = 0; i < measureCount; i++) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/e849cc2a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
index f5cee9f..b4fb0a0 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
@@ -118,6 +118,9 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
             .withDescription("Statistics sampling percentage").create(BatchConstants.ARG_STATS_SAMPLING_PERCENT);
     protected static final Option OPTION_CUBOID_MODE = OptionBuilder.withArgName(BatchConstants.ARG_CUBOID_MODE)
             .hasArg().isRequired(false).withDescription("Cuboid Mode").create(BatchConstants.ARG_CUBOID_MODE);
+    protected static final Option OPTION_NEED_UPDATE_BASE_CUBOID_SHARD = OptionBuilder
+            .withArgName(BatchConstants.ARG_UPDATE_SHARD).hasArg().isRequired(false)
+            .withDescription("If need to update base cuboid shard").create(BatchConstants.ARG_UPDATE_SHARD);
 
     private static final String MAP_REDUCE_CLASSPATH = "mapreduce.application.classpath";
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/e849cc2a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
index 1d6a582..129c6dd 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
@@ -29,6 +29,7 @@ public interface BatchConstants {
      * ConFiGuration entry names for MR jobs
      */
 
+    String CFG_UPDATE_SHARD = "update.shard";
     String CFG_CUBOID_MODE = "cuboid.mode";
     String CFG_CUBE_NAME = "cube.name";
     String CFG_CUBE_SEGMENT_NAME = "cube.segment.name";
@@ -76,6 +77,7 @@ public interface BatchConstants {
     String ARG_OUTPUT = "output";
     String ARG_PROJECT = "project";
     String ARG_CUBOID_MODE = "cuboidMode";
+    String ARG_UPDATE_SHARD = "updateShard"; // indicate if need update base cuboid shard
     String ARG_JOB_NAME = "jobname";
     String ARG_CUBING_JOB_ID = "cubingJobId";
     String ARG_CUBE_NAME = "cubename";

http://git-wip-us.apache.org/repos/asf/kylin/blob/e849cc2a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidJob.java
index 62109f4..7bfa33a 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidJob.java
@@ -76,6 +76,7 @@ public class InMemCuboidFromBaseCuboidJob extends AbstractHadoopJob {
             options.addOption(OPTION_CUBING_JOB_ID);
             options.addOption(OPTION_INPUT_PATH);
             options.addOption(OPTION_CUBOID_MODE);
+            options.addOption(OPTION_NEED_UPDATE_BASE_CUBOID_SHARD);
             parseOptions(options, args);
 
             String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
@@ -91,6 +92,10 @@ public class InMemCuboidFromBaseCuboidJob extends AbstractHadoopJob {
             if (cuboidModeName == null) {
                 cuboidModeName = CuboidModeEnum.CURRENT.toString();
             }
+            String ifNeedUpdateBaseCuboidShard = getOptionValue(OPTION_NEED_UPDATE_BASE_CUBOID_SHARD);
+            if (ifNeedUpdateBaseCuboidShard == null) {
+                ifNeedUpdateBaseCuboidShard = "false";
+            }
 
             CuboidScheduler cuboidScheduler = CuboidSchedulerUtil.getCuboidSchedulerByMode(cubeSeg, cuboidModeName);
 
@@ -111,6 +116,7 @@ public class InMemCuboidFromBaseCuboidJob extends AbstractHadoopJob {
             job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
             job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_ID, segmentID);
             job.getConfiguration().set(BatchConstants.CFG_CUBOID_MODE, cuboidModeName);
+            job.getConfiguration().set(BatchConstants.CFG_UPDATE_SHARD, ifNeedUpdateBaseCuboidShard);
 
             String input = getOptionValue(OPTION_INPUT_PATH);
             FileInputFormat.setInputPaths(job, new Path(input));

http://git-wip-us.apache.org/repos/asf/kylin/blob/e849cc2a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidMapper.java
index 73a39d6..1beebc7 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidFromBaseCuboidMapper.java
@@ -40,6 +40,7 @@ import org.apache.kylin.cube.inmemcubing.InputConverterUnit;
 import org.apache.kylin.cube.inmemcubing.InputConverterUnitForBaseCuboid;
 import org.apache.kylin.cube.kv.CubeDimEncMap;
 import org.apache.kylin.engine.mr.ByteArrayWritable;
+import org.apache.kylin.engine.mr.common.BatchConstants;
 import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.metadata.model.TblColRef;
 
@@ -64,13 +65,18 @@ public class InMemCuboidFromBaseCuboidMapper
     }
 
     @Override
-    protected InputConverterUnit<ByteArray> getInputConverterUnit() {
-        return new InputConverterUnitForBaseCuboid();
+    protected InputConverterUnit<ByteArray> getInputConverterUnit(Context context) {
+        String updateShard = context.getConfiguration().get(BatchConstants.CFG_UPDATE_SHARD);
+        if (updateShard == null || updateShard.equalsIgnoreCase("false")) {
+            return new InputConverterUnitForBaseCuboid(false);
+        } else {
+            return new InputConverterUnitForBaseCuboid(true);
+        }
     }
 
     @Override
     protected Future getCubingThreadFuture(Context context, Map<TblColRef, Dictionary<String>> dictionaryMap,
-            int reserveMemoryMB, CuboidScheduler cuboidScheduler, InputConverterUnit<ByteArray> inputConverterUnit) {
+            int reserveMemoryMB, CuboidScheduler cuboidScheduler) {
         AbstractInMemCubeBuilder cubeBuilder = new DoggedCubeBuilder(cuboidScheduler, flatDesc, dictionaryMap);
         cubeBuilder.setReserveMemoryMB(reserveMemoryMB);
         cubeBuilder.setConcurrentThreads(taskThreadCount);

http://git-wip-us.apache.org/repos/asf/kylin/blob/e849cc2a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
index 859e126..551a17b 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
@@ -41,7 +41,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 public class InMemCuboidMapper<KEYIN>
         extends InMemCuboidMapperBase<KEYIN, Object, ByteArrayWritable, ByteArrayWritable, String[]> {
 
-
     private IMRInput.IMRTableInputFormat flatTableInputFormat;
 
     @Override
@@ -52,7 +51,7 @@ public class InMemCuboidMapper<KEYIN>
     }
 
     @Override
-    protected InputConverterUnit<String[]> getInputConverterUnit() {
+    protected InputConverterUnit<String[]> getInputConverterUnit(Context context) {
         Preconditions.checkNotNull(cubeDesc);
         Preconditions.checkNotNull(dictionaryMap);
         return new InputConverterUnitForRawData(cubeDesc, flatDesc, dictionaryMap);
@@ -65,7 +64,7 @@ public class InMemCuboidMapper<KEYIN>
 
     @Override
     protected Future getCubingThreadFuture(Context context, Map<TblColRef, Dictionary<String>> dictionaryMap,
-            int reserveMemoryMB, CuboidScheduler cuboidScheduler, InputConverterUnit<String[]> inputConverterUnit) {
+            int reserveMemoryMB, CuboidScheduler cuboidScheduler) {
         AbstractInMemCubeBuilder cubeBuilder = new DoggedCubeBuilder(cuboidScheduler, flatDesc, dictionaryMap);
         cubeBuilder.setReserveMemoryMB(reserveMemoryMB);
         cubeBuilder.setConcurrentThreads(taskThreadCount);

http://git-wip-us.apache.org/repos/asf/kylin/blob/e849cc2a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java
index 7b4738b..73af138 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java
@@ -73,10 +73,10 @@ public abstract class InMemCuboidMapperBase<KEYIN, VALUEIN, KEYOUT, VALUEOUT, T>
     protected InputConverterUnit<T> inputConverterUnit;
     private Future<?> future;
 
-    protected abstract InputConverterUnit<T> getInputConverterUnit();
+    protected abstract InputConverterUnit<T> getInputConverterUnit(Context context);
 
-    protected abstract Future getCubingThreadFuture(Context context, Map<TblColRef, Dictionary<String>> dictionaryMap, int reserveMemoryMB, //
-                                                    CuboidScheduler cuboidScheduler, InputConverterUnit<T> inputConverterUnit);
+    protected abstract Future getCubingThreadFuture(Context context, Map<TblColRef, Dictionary<String>> dictionaryMap,
+            int reserveMemoryMB, CuboidScheduler cuboidScheduler);
 
     protected abstract T getRecordFromKeyValue(KEYIN key, VALUEIN value);
 
@@ -116,8 +116,8 @@ public abstract class InMemCuboidMapperBase<KEYIN, VALUEIN, KEYOUT, VALUEOUT, T>
 
         taskThreadCount = config.getCubeAlgorithmInMemConcurrentThreads();
         reserveMemoryMB = calculateReserveMB(conf);
-        inputConverterUnit = getInputConverterUnit();
-        future = getCubingThreadFuture(context, dictionaryMap, reserveMemoryMB, cuboidScheduler, inputConverterUnit);
+        inputConverterUnit = getInputConverterUnit(context);
+        future = getCubingThreadFuture(context, dictionaryMap, reserveMemoryMB, cuboidScheduler);
     }
 
     private int calculateReserveMB(Configuration configuration) {
@@ -145,7 +145,7 @@ public abstract class InMemCuboidMapperBase<KEYIN, VALUEIN, KEYOUT, VALUEOUT, T>
         }
 
         if (counter % unitRows == 0 && shouldCutSplit(nSplit, countOfLastSplit)) {
-            if (offer(context, inputConverterUnit.getCutUnit(), 1, TimeUnit.MINUTES, 60)) {
+            if (offer(context, inputConverterUnit.getCutRow(), 1, TimeUnit.MINUTES, 60)) {
                 countOfLastSplit = 0;
             } else {
                 throw new IOException("Failed to offer row to internal queue due to queue full!");
@@ -159,7 +159,7 @@ public abstract class InMemCuboidMapperBase<KEYIN, VALUEIN, KEYOUT, VALUEOUT, T>
         logger.info("Totally handled " + mapCounter + " records!");
 
         while (!future.isDone()) {
-            if (queue.offer(inputConverterUnit.getEmptyUnit(), 1, TimeUnit.SECONDS)) {
+            if (queue.offer(inputConverterUnit.getEndRow(), 1, TimeUnit.SECONDS)) {
                 break;
             }
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e849cc2a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java
index ad754cd..3f97f80 100644
--- a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java
@@ -186,7 +186,7 @@ public class ITInMemCubeBuilderTest extends LocalFileMetadataTestCase {
             }
             queue.put(row);
         }
-        queue.put(InputConverterUnitForRawData.EMPTY_ROW);
+        queue.put(InputConverterUnitForRawData.END_ROW);
     }
 
     static Map<TblColRef, Dictionary<String>> getDictionaryMap(CubeInstance cube, String flatTable) throws IOException {


[16/18] kylin git commit: APACHE-KYLIN-2734: backend support for hot cuboids export & import

Posted by li...@apache.org.
APACHE-KYLIN-2734: backend support for hot cuboids export & import

Signed-off-by: Zhong <nj...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/0aec7557
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/0aec7557
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/0aec7557

Branch: refs/heads/ci-dong
Commit: 0aec7557f71e6ee9f8f6bba5f780ef62f4677867
Parents: c6e8093
Author: Ma Gang <mg...@163.com>
Authored: Wed Aug 30 14:51:19 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/cube/model/CubeDesc.java   | 41 +++++++++++++++++++-
 .../kylin/rest/controller/CubeController.java   |  8 ++--
 .../apache/kylin/rest/service/BasicService.java |  1 -
 .../apache/kylin/rest/service/QueryService.java |  7 ++--
 4 files changed, 49 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/0aec7557/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index 01dd302..9c0a7cc 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -58,13 +58,13 @@ import org.apache.kylin.measure.extendedcolumn.ExtendedColumnMeasureType;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.DataModelManager;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.IEngineAware;
 import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.JoinDesc;
 import org.apache.kylin.metadata.model.JoinTableDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.DataModelManager;
 import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.project.ProjectInstance;
@@ -185,6 +185,10 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
     @JsonInclude(JsonInclude.Include.NON_NULL)
     private int parentForward = 3;
 
+    @JsonProperty("mandatory_dimension_set_list")
+    @JsonInclude(JsonInclude.Include.NON_NULL)
+    private List<Set<String>> mandatoryDimensionSetList = Collections.emptyList();
+
     private LinkedHashSet<TblColRef> allColumns = new LinkedHashSet<>();
     private LinkedHashSet<ColumnDesc> allColumnDescs = new LinkedHashSet<>();
     private LinkedHashSet<TblColRef> dimensionColumns = new LinkedHashSet<>();
@@ -446,6 +450,14 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
         this.overrideKylinProps = overrideKylinProps;
     }
 
+    public List<Set<String>> getMandatoryDimensionSetList() {
+        return mandatoryDimensionSetList;
+    }
+
+    public void setMandatoryDimensionSetList(List<Set<String>> mandatoryDimensionSetList) {
+        this.mandatoryDimensionSetList = mandatoryDimensionSetList;
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o)
@@ -537,6 +549,13 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
                     .append(JsonUtil.writeValueAsString(this.engineType)).append("|")//
                     .append(JsonUtil.writeValueAsString(this.storageType)).append("|");
 
+            if (mandatoryDimensionSetList != null && !mandatoryDimensionSetList.isEmpty()) {
+                for (Set<String> mandatoryDimensionSet : mandatoryDimensionSetList) {
+                    TreeSet<String> sortedSet = Sets.newTreeSet(mandatoryDimensionSet);
+                    sigString.append(JsonUtil.writeValueAsString(sortedSet)).append("|");
+                }
+            }
+
             String signatureInput = sigString.toString().replaceAll("\\s+", "").toLowerCase();
 
             byte[] signature = md.digest(signatureInput.getBytes());
@@ -631,6 +650,26 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
 
         initDictionaryDesc();
         amendAllColumns();
+
+        // check if mandatory dimension set list is valid
+        validateMandatoryDimensionSetList();
+    }
+
+    public void validateMandatoryDimensionSetList() {
+        Set<String> rowKeyColumns = Sets.newHashSet();
+        for (RowKeyColDesc entry : getRowkey().getRowKeyColumns()) {
+            rowKeyColumns.add(entry.getColumn());
+        }
+
+        for (Set<String> mandatoryDimensionSet : this.mandatoryDimensionSetList) {
+            for (String columnName : mandatoryDimensionSet) {
+                if (!rowKeyColumns.contains(columnName)) {
+                    logger.info("Column " + columnName + " in " + mandatoryDimensionSet + " does not exist");
+                    throw new IllegalStateException(
+                            "Column " + columnName + " in " + mandatoryDimensionSet + " does not exist");
+                }
+            }
+        }
     }
 
     public CuboidScheduler getInitialCuboidScheduler() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/0aec7557/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
index aa59e30..0e9f4ba 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
@@ -898,11 +898,13 @@ public class CubeController extends BasicController {
         return getCuboidHitFrequency(cubeName, false);
     }
 
-    private Map<Long, Long> getCuboidHitFrequency(String cubeName, boolean ifSource) {
+    private Map<Long, Long> getCuboidHitFrequency(String cubeName, boolean isCuboidSource) {
         SQLRequest sqlRequest = new SQLRequest();
         sqlRequest.setProject(MetricsManager.SYSTEM_PROJECT);
-        String cuboidColumn = ifSource ? QueryCubePropertyEnum.CUBOID_SOURCE.toString()
-                : QueryCubePropertyEnum.CUBOID_TARGET.toString();
+        String cuboidColumn = QueryCubePropertyEnum.CUBOID_SOURCE.toString();
+        if (!isCuboidSource) {
+            cuboidColumn = QueryCubePropertyEnum.CUBOID_TARGET.toString();
+        }
         String hitMeasure = QueryCubePropertyEnum.WEIGHT_PER_HIT.toString();
         String table = cubeService.getMetricsManager()
                 .getSystemTableFromSubject(cubeService.getConfig().getKylinMetricsSubjectQueryCube());

http://git-wip-us.apache.org/repos/asf/kylin/blob/0aec7557/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java b/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java
index f8f50f3..9ac2602 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java
@@ -98,5 +98,4 @@ public abstract class BasicService {
     public MetricsManager getMetricsManager() {
         return MetricsManager.getInstance();
     }
-
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/0aec7557/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 3f16646..9c3d34f 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -464,7 +464,7 @@ public class QueryService extends BasicService {
                     sqlResponse.setTotalScanBytes(0);
                 }
 
-                checkQueryAuth(sqlResponse, project);
+                checkQueryAuth(sqlResponse, project, secureEnabled);
 
             } catch (Throwable e) { // calcite may throw AssertError
                 logger.error("Exception while executing query", e);
@@ -526,8 +526,9 @@ public class QueryService extends BasicService {
         return response;
     }
 
-    protected void checkQueryAuth(SQLResponse sqlResponse, String project) throws AccessDeniedException {
-        if (!sqlResponse.getIsException() && KylinConfig.getInstanceFromEnv().isQuerySecureEnabled()) {
+    protected void checkQueryAuth(SQLResponse sqlResponse, String project, boolean secureEnabled)
+            throws AccessDeniedException {
+        if (!sqlResponse.getIsException() && KylinConfig.getInstanceFromEnv().isQuerySecureEnabled() && secureEnabled) {
             checkAuthorization(sqlResponse, project);
         }
     }


[10/18] kylin git commit: APACHE-KYLIN-2733: Don't allow building if still optimizing & don't allow optimizing if has building segments

Posted by li...@apache.org.
APACHE-KYLIN-2733: Don't allow building if still optimizing & don't allow optimizing if has building segments


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/5ecc48aa
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/5ecc48aa
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/5ecc48aa

Branch: refs/heads/ci-dong
Commit: 5ecc48aa501dbbd7f4d0f4ce904fc409ca82fc36
Parents: 37f83ed
Author: Zhong <nj...@apache.org>
Authored: Wed Nov 8 15:13:58 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../apache/kylin/rest/service/JobService.java   | 30 ++++++++++++++++++++
 1 file changed, 30 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/5ecc48aa/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
index 9f2d0d9..89e996d 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
@@ -61,6 +61,7 @@ import org.apache.kylin.job.lock.JobLock;
 import org.apache.kylin.metadata.model.SegmentRange;
 import org.apache.kylin.metadata.model.SegmentRange.TSRange;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.metadata.model.Segments;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.rest.exception.BadRequestException;
 import org.apache.kylin.rest.msg.Message;
@@ -232,6 +233,12 @@ public class JobService extends BasicService implements InitializingBean {
         }
 
         checkCubeDescSignature(cube);
+        checkAllowBuilding(cube);
+
+        if (buildType == CubeBuildTypeEnum.BUILD || buildType == CubeBuildTypeEnum.REFRESH) {
+            checkAllowParallelBuilding(cube);
+        }
+
         DefaultChainedExecutable job;
 
         CubeSegment newSeg = null;
@@ -416,7 +423,30 @@ public class JobService extends BasicService implements InitializingBean {
                     String.format(msg.getINCONSISTENT_CUBE_DESC_SIGNATURE(), cube.getDescriptor()));
     }
 
+    private void checkAllowBuilding(CubeInstance cube) {
+        Segments<CubeSegment> readyPendingSegments = cube.getSegments(SegmentStatusEnum.READY_PENDING);
+        if (readyPendingSegments.size() > 0) {
+            throw new BadRequestException("The cube " + cube.getName() + " has READY_PENDING segments "
+                    + readyPendingSegments + ". It's not allowed for building");
+        }
+    }
+
+    private void checkAllowParallelBuilding(CubeInstance cube) {
+        if (cube.getCuboids() == null) {
+            Segments<CubeSegment> cubeSegments = cube.getSegments();
+            if (cubeSegments.size() > 0 && cubeSegments.getSegments(SegmentStatusEnum.READY).size() <= 0) {
+                throw new BadRequestException("The cube " + cube.getName() + " has segments " + cubeSegments
+                        + ", but none of them is READY. It's not allowed for parallel building");
+            }
+        }
+    }
+
     private void checkAllowOptimization(CubeInstance cube, Set<Long> cuboidsRecommend) {
+        Segments<CubeSegment> buildingSegments = cube.getBuildingSegments();
+        if (buildingSegments.size() > 0) {
+            throw new BadRequestException("The cube " + cube.getName() + " has building segments " + buildingSegments
+                    + ". It's not allowed for optimization");
+        }
         long baseCuboid = cube.getCuboidScheduler().getBaseCuboidId();
         if (!cuboidsRecommend.contains(baseCuboid)) {
             throw new BadRequestException("The recommend cuboids should contain the base cuboid " + baseCuboid);


[13/18] kylin git commit: APACHE-KYLIN-2707: fix NPE

Posted by li...@apache.org.
APACHE-KYLIN-2707: fix NPE

Signed-off-by: Zhong <nj...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/70262f33
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/70262f33
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/70262f33

Branch: refs/heads/ci-dong
Commit: 70262f334f8ebb20d00f34c4eb4ded01752601c2
Parents: e849cc2
Author: Wang Ken <mi...@ebay.com>
Authored: Mon Nov 13 20:55:33 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../apache/kylin/engine/mr/common/JobInfoConverter.java   | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/70262f33/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/JobInfoConverter.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/JobInfoConverter.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/JobInfoConverter.java
index 3098c15..a230517 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/JobInfoConverter.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/JobInfoConverter.java
@@ -62,6 +62,11 @@ public class JobInfoConverter {
         }
 
         Output output = outputs.get(job.getId());
+        if (output == null) {
+            logger.warn("job output is null.");
+            return null;
+        }
+
         final JobInstance result = new JobInstance();
         result.setName(job.getName());
         result.setRelatedCube(CubingExecutableUtil.getCubeName(job.getParams()));
@@ -91,6 +96,11 @@ public class JobInfoConverter {
         }
 
         Output output = outputs.get(job.getId());
+        if (output == null) {
+            logger.warn("job output is null.");
+            return null;
+        }
+
         final JobInstance result = new JobInstance();
         result.setName(job.getName());
         result.setRelatedCube(CubingExecutableUtil.getCubeName(job.getParams()));


[17/18] kylin git commit: APACHE-KYLIN-2822 Introduce sunburst chart to show cuboid tree

Posted by li...@apache.org.
APACHE-KYLIN-2822 Introduce sunburst chart to show cuboid tree


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/1fce1930
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/1fce1930
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/1fce1930

Branch: refs/heads/ci-dong
Commit: 1fce1930a74a896e184096320a1b6b0fbadb0f40
Parents: 12fefdc
Author: liapan <li...@ebay.com>
Authored: Mon Nov 20 10:26:44 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/cube/CubeInstance.java     |   8 +
 .../job/execution/CheckpointExecutable.java     |  37 ++++
 webapp/app/js/controllers/cube.js               | 179 ++++++++++++++++++-
 webapp/app/js/model/cubeConfig.js               |  78 +++++++-
 webapp/app/js/services/cubes.js                 |  45 ++++-
 webapp/app/less/app.less                        |  25 +++
 webapp/app/partials/cubes/cube_detail.html      |  46 ++++-
 7 files changed, 413 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/1fce1930/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
index 462223a..70477eb 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
@@ -435,6 +435,14 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
         }
     }
 
+    public long getCuboidLastOptimized() {
+        return cuboidLastOptimized;
+    }
+
+    public void setCuboidLastOptimized(long lastOptimized) {
+        this.cuboidLastOptimized = lastOptimized;
+    }
+
     /**
      * Get cuboid level count except base cuboid
      * @return

http://git-wip-us.apache.org/repos/asf/kylin/blob/1fce1930/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java b/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java
index db477cb..c5f1c0a 100644
--- a/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java
@@ -18,8 +18,12 @@
 
 package org.apache.kylin.job.execution;
 
+import java.io.IOException;
 import java.util.List;
 
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeUpdate;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,6 +37,7 @@ public class CheckpointExecutable extends DefaultChainedExecutable {
 
     private static final String DEPLOY_ENV_NAME = "envName";
     private static final String PROJECT_INSTANCE_NAME = "projectName";
+    private static final String CUBE_NAME = "cubeName";
 
     private final List<AbstractExecutable> subTasksForCheck = Lists.newArrayList();
 
@@ -62,6 +67,33 @@ public class CheckpointExecutable extends DefaultChainedExecutable {
         return true;
     }
 
+    @Override
+    protected void onExecuteFinished(ExecuteResult result, ExecutableContext executableContext) {
+        super.onExecuteFinished(result, executableContext);
+        if (!isDiscarded() && result.succeed()) {
+            List<? extends Executable> jobs = getTasks();
+            boolean allSucceed = true;
+            for (Executable task : jobs) {
+                final ExecutableState status = task.getStatus();
+                if (status != ExecutableState.SUCCEED) {
+                    allSucceed = false;
+                }
+            }
+            if (allSucceed) {
+                // Add last optimization time
+                CubeManager cubeManager = CubeManager.getInstance(executableContext.getConfig());
+                CubeInstance cube = cubeManager.getCube(getCubeName());
+                try{
+                    cube.setCuboidLastOptimized(getEndTime());
+                    CubeUpdate cubeUpdate = new CubeUpdate(cube);
+                    cubeManager.updateCube(cubeUpdate);
+                } catch (IOException e) {
+                    logger.error("Failed to update last optimized for " + getCubeName(), e);
+                }
+            }
+        }
+    }
+
     public String getDeployEnvName() {
         return getParam(DEPLOY_ENV_NAME);
     }
@@ -78,8 +110,13 @@ public class CheckpointExecutable extends DefaultChainedExecutable {
         setParam(PROJECT_INSTANCE_NAME, name);
     }
 
+    public String getCubeName() {
+        return getParam(CUBE_NAME);
+    }
+
     @Override
     public int getDefaultPriority() {
         return DEFAULT_PRIORITY;
     }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1fce1930/webapp/app/js/controllers/cube.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cube.js b/webapp/app/js/controllers/cube.js
index d3a5079..b2f6ad7 100755
--- a/webapp/app/js/controllers/cube.js
+++ b/webapp/app/js/controllers/cube.js
@@ -18,7 +18,7 @@
 
 'use strict';
 
-KylinApp.controller('CubeCtrl', function ($scope, AccessService, MessageService, CubeService, TableService, ModelGraphService, UserService,SweetAlert,loadingRequest,modelsManager,$modal,cubesManager) {
+KylinApp.controller('CubeCtrl', function ($scope, AccessService, MessageService, CubeService, cubeConfig, TableService, ModelGraphService, UserService,SweetAlert,loadingRequest,modelsManager,$modal,cubesManager, $location) {
     $scope.newAccess = null;
     $scope.state = {jsonEdit: false};
 
@@ -111,5 +111,182 @@ KylinApp.controller('CubeCtrl', function ($scope, AccessService, MessageService,
         }
     };
 
+    // cube api to refresh current chart after get recommend data.
+    $scope.currentChart = {};
+
+    // click planner tab to get current cuboid chart
+    $scope.getCubePlanner = function(cube) {
+        $scope.enableRecommend = cube.segments.length > 0 && _.some(cube.segments, function(segment){ return segment.status === 'READY'; });
+        if (!cube.currentCuboids) {
+            CubeService.getCurrentCuboids({cubeId: cube.name}, function(data) {
+                if (data && data.nodeInfos) {
+                    $scope.createChart(data, 'current');
+                    cube.currentCuboids = data;
+                } else {
+                    $scope.currentOptions = angular.copy(cubeConfig.chartOptions);
+                    $scope.currentData = [];
+                }
+            }, function(e) {
+                SweetAlert.swal('Oops...', 'Failed to get current cuboid.', 'error');
+                console.error('current cuboid error', e.data);
+            });
+        } else {
+            $scope.createChart(cube.currentCuboids, 'current');
+        }
+    };
+
+    // get recommend cuboid chart
+    $scope.getRecommendCuboids = function(cube) {
+        if (!cube.recommendCuboids) {
+            loadingRequest.show();
+            CubeService.getRecommendCuboids({cubeId: cube.name}, function(data) {
+                loadingRequest.hide();
+                if (data && data.nodeInfos) {
+                    // recommending
+                    if (data.nodeInfos.length === 1 && !data.nodeInfos[0].cuboid_id) {
+                         SweetAlert.swal('Loading', 'Please wait a minute, servers are recommending for you', 'success');
+                    } else {
+                        $scope.createChart(data, 'recommend');
+                        cube.recommendCuboids = data;
+                        // update current chart mark delete node gray.
+                        angular.forEach(cube.currentCuboids.nodeInfos, function(nodeInfo) {
+                            var tempNode = _.find(data.nodeInfos, function(o) { return o.cuboid_id == nodeInfo.cuboid_id; });
+                            if (!tempNode) {
+                                nodeInfo.deleted = true;
+                            }
+                        });
+                        $scope.createChart(cube.currentCuboids, 'current');
+                        $scope.currentChart.api.refresh();
+                    }
+                } else {
+                    $scope.currentOptions = angular.copy(cubeConfig.chartOptions);
+                    $scope.recommendData = [];
+                }
+            }, function(e) {
+                loadingRequest.hide();
+                SweetAlert.swal('Oops...', 'Failed to get recommend cuboid.', 'error');
+                console.error('recommend cuboid error', e.data);
+            });
+        } else {
+            $scope.createChart(cube.recommendCuboids, 'recommend');
+        }
+    };
+
+    // optimize cuboid
+    $scope.optimizeCuboids = function(cube){
+        SweetAlert.swal({
+            title: '',
+            text: 'Are you sure to optimize the cube?',
+            type: '',
+            showCancelButton: true,
+            confirmButtonColor: '#DD6B55',
+            confirmButtonText: "Yes",
+            closeOnConfirm: true
+        }, function(isConfirm) {
+              if(isConfirm) {
+                var cuboidsRecommendArr = [];
+                angular.forEach(cube.recommendCuboids.nodeInfos, function(node) {
+                    cuboidsRecommendArr.push(node.cuboid_id);
+                });
+                loadingRequest.show();
+                CubeService.optimize({cubeId: cube.name}, {cuboidsRecommend: cuboidsRecommendArr},
+                    function(job){
+                        loadingRequest.hide();
+                        SweetAlert.swal({
+                            title: 'Success!',
+                            text: 'Optimize cube job has been started!',
+                            type: 'success'},
+                            function() {
+                                $location.path("/jobs");
+                            }
+                        );
+                    }, function(e) {
+                        loadingRequest.hide();
+                        if (e.status === 400) {
+                            SweetAlert.swal('Oops...', e.data.exception, 'error');
+                        } else {
+                            SweetAlert.swal('Oops...', "Failed to create optimize cube job.", 'error');
+                            console.error('optimize cube error', e.data);
+                        }
+                });
+            }
+        });
+    };
+
+    // transform chart data and customized options.
+    $scope.createChart = function(data, type) {
+        var chartData = data.treeNode;
+        if ('current' === type) {
+            $scope.currentData = [chartData];
+            $scope.currentOptions = angular.copy(cubeConfig.baseChartOptions);
+            $scope.currentOptions.caption = angular.copy(cubeConfig.currentCaption);
+            if ($scope.cube.recommendCuboids){
+                $scope.currentOptions.caption.css['text-align'] = 'right';
+                $scope.currentOptions.caption.css['right'] = '-12px';
+            }
+            $scope.currentOptions.chart.color = function(d) {
+                var cuboid = _.find(data.nodeInfos, function(o) { return o.name == d; });
+                if (cuboid.deleted) {
+                    return d3.scale.category20c().range()[17];
+                } else {
+                    return getColorByQuery(0, 1/data.nodeInfos.length, cuboid.query_rate);
+                }
+            };
+            $scope.currentOptions.chart.sunburst = getSunburstDispatch();
+            $scope.currentOptions.title.text = 'Current Cuboid Distribution';
+            $scope.currentOptions.subtitle.text = '[Cuboid Count: ' + data.nodeInfos.length + '] [Row Count: ' + data.totalRowCount + ']';
+        } else if ('recommend' === type) {
+            $scope.recommendData = [chartData];
+            $scope.recommendOptions = angular.copy(cubeConfig.baseChartOptions);
+            $scope.recommendOptions.caption = angular.copy(cubeConfig.recommendCaption);
+            $scope.recommendOptions.chart.color = function(d) {
+                var cuboid = _.find(data.nodeInfos, function(o) { return o.name == d; });
+                if (cuboid.row_count < 0) {
+                    return d3.scale.category20c().range()[5];
+                } else {
+                    var colorIndex = 0;
+                    if (!cuboid.existed) {
+                        colorIndex = 8;
+                    }
+                    return getColorByQuery(colorIndex, 1/data.nodeInfos.length, cuboid.query_rate);
+                }
+            };
+            $scope.recommendOptions.chart.sunburst = getSunburstDispatch();
+            $scope.recommendOptions.title.text = 'Recommend Cuboid Distribution';
+            $scope.recommendOptions.subtitle.text = '[Cuboid Count: ' + data.nodeInfos.length + '] [Row Count: ' + data.totalRowCount + ']';
+        }
+    };
+
+    // Hover behavior for highlight dimensions
+    function getSunburstDispatch() {
+        return {
+            dispatch: {
+                elementMouseover: function(t, u) {
+                    $scope.selectCuboid = t.data.name;
+                    $scope.$apply();
+                },
+                renderEnd: function(t, u) {
+                    var chartElements = document.getElementsByClassName('nv-sunburst');
+                    angular.element(chartElements).on('mouseleave', function() {
+                        $scope.selectCuboid = '0';
+                        $scope.$apply();
+                    });
+                }
+            }
+        };
+    };
+
+    // Different color for chart element by query count
+    function getColorByQuery(colorIndex, baseRate, queryRate) {
+        if (queryRate > (3 * baseRate)) {
+            return d3.scale.category20c().range()[colorIndex];
+        } else if (queryRate > (2 * baseRate)) {
+            return d3.scale.category20c().range()[colorIndex+1];
+        } else if (queryRate > baseRate) {
+            return d3.scale.category20c().range()[colorIndex+2];
+        } else {
+            return d3.scale.category20c().range()[colorIndex+3];
+        }
+    }
 });
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/1fce1930/webapp/app/js/model/cubeConfig.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/cubeConfig.js b/webapp/app/js/model/cubeConfig.js
index d04af76..e163d75 100644
--- a/webapp/app/js/model/cubeConfig.js
+++ b/webapp/app/js/model/cubeConfig.js
@@ -113,5 +113,79 @@ KylinApp.constant('cubeConfig', {
     {name:"Global Dictionary", value:"org.apache.kylin.dict.GlobalDictionaryBuilder"},
     {name:"Segment Dictionary", value:"org.apache.kylin.dict.global.SegmentAppendTrieDictBuilder"}
   ],
-  needSetLengthEncodingList:['fixed_length','fixed_length_hex','int','integer']
-});
+  needSetLengthEncodingList:['fixed_length','fixed_length_hex','int','integer'],
+  baseChartOptions: {
+    chart: {
+      type: 'sunburstChart',
+      height: 500,
+      duration: 250,
+      groupColorByParent: false,
+      tooltip: {
+        contentGenerator: function(obj) {
+          var preCalculatedStr = '';
+          if (typeof obj.data.existed !== 'undefined' && obj.data.existed !== null) {
+            preCalculatedStr = '<tr><td align="right"><b>Existed:</b></td><td>' + obj.data.existed + '</td></tr>';
+          }
+          var rowCountRateStr = '';
+          if (obj.data.row_count) {
+            rowCountRateStr = '<tr><td align="right"><b>Row Count:</b></td><td>' + obj.data.row_count + '</td></tr><tr><td align="right"><b>Rollup Rate:</b></td><td>' + (obj.data.row_count * 100 / obj.data.parent_row_count).toFixed(2) + '%</td></tr>';
+          }
+          return '<table><tbody>'
+          + '<tr><td align="right"><i class="fa fa-square" style="color: ' + obj.color + '; margin-right: 15px;" aria-hidden="true"></i><b>Name:</b></td><td class="key"><b>' + obj.data.name +'</b></td></tr>'
+          + '<tr><td align="right"><b>ID:</b></td><td>' + obj.data.cuboid_id + '</td></tr>'
+          + '<tr><td align="right"><b>Query Count:</b></td><td>' + obj.data.query_count + '  [' + (obj.data.query_rate * 100).toFixed(2) + '%]</td></tr>'
+          + '<tr><td align="right"><b>Exactly Match Count:</b></td><td>' + obj.data.exactly_match_count + '</td></tr>'
+          + rowCountRateStr
+          + preCalculatedStr
+          + '</tbody></table>';
+        }
+      }
+    },
+    title: {
+      enable: true,
+      text: '',
+      className: 'h4',
+      css: {
+        position: 'relative',
+        top: '30px'
+      }
+    },
+    subtitle: {
+      enable: true,
+      text: '',
+      className: 'h5',
+      css: {
+        position: 'relative',
+        top: '40px'
+      }
+    }
+  },
+  currentCaption: {
+    enable: true,
+    html: '<div>Existed: <i class="fa fa-square" style="color:#38c;"></i> Hottest '
+          + '<i class="fa fa-square" style="color:#7bd;"></i> Hot '
+          + '<i class="fa fa-square" style="color:#ade;"></i> Warm '
+          + '<i class="fa fa-square" style="color:#cef;"></i> Cold '
+          + '<i class="fa fa-square" style="color:#999;"></i> Retire</div>',
+    css: {
+      position: 'relative',
+      top: '-35px',
+      height: 0
+    }
+  },
+  recommendCaption: {
+    enable: true,
+    html: '<div>New: <i class="fa fa-square" style="color:#3a5;"></i> Hottest '
+      + '<i class="fa fa-square" style="color:#7c7;"></i> Hot '
+      + '<i class="fa fa-square" style="color:#aea;"></i> Warm '
+      + '<i class="fa fa-square" style="color:#cfc;"></i> Cold '
+      + '<i class="fa fa-square" style="color:#f94;"></i> Mandatory</div>',
+    css: {
+      position: 'relative',
+      top: '-35px',
+      height: 0,
+      'text-align': 'left',
+      'left': '-12px'
+    }
+  }
+});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/1fce1930/webapp/app/js/services/cubes.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/services/cubes.js b/webapp/app/js/services/cubes.js
index 26ecd3f..bc0dfcd 100644
--- a/webapp/app/js/services/cubes.js
+++ b/webapp/app/js/services/cubes.js
@@ -17,6 +17,25 @@
  */
 
 KylinApp.factory('CubeService', ['$resource', function ($resource, config) {
+  function transformCuboidsResponse(data) {
+    var cuboids = {
+      nodeInfos: [],
+      treeNode: data.root,
+      totalRowCount: 0
+    };
+    function iterator(node, parentRowCount) {
+      node.parent_row_count = parentRowCount;
+      cuboids.nodeInfos.push(node);
+      cuboids.totalRowCount += node.row_count;
+      if (node.children.length) {
+        angular.forEach(node.children, function(child) {
+          iterator(child, node.row_count);
+        });
+      }
+    };
+    iterator(data.root, data.root.row_count);
+    return cuboids;
+  };
   return $resource(Config.service.url + 'cubes/:cubeId/:propName/:propValue/:action', {}, {
     list: {method: 'GET', params: {}, isArray: true},
     getValidEncodings: {method: 'GET', params: {action:"validEncodings"}, isArray: false},
@@ -34,6 +53,30 @@ KylinApp.factory('CubeService', ['$resource', function ($resource, config) {
     drop: {method: 'DELETE', params: {}, isArray: false},
     save: {method: 'POST', params: {}, isArray: false},
     update: {method: 'PUT', params: {}, isArray: false},
-    getHbaseInfo: {method: 'GET', params: {propName: 'hbase'}, isArray: true}
+    getHbaseInfo: {method: 'GET', params: {propName: 'hbase'}, isArray: true},
+    getCurrentCuboids: {
+      method: 'GET',
+      params: {
+          propName: 'cuboids',
+          propValue: 'current'
+      },
+      isArray: false,
+      interceptor: {
+        response: function(response) {
+          return transformCuboidsResponse(response.data);
+        }
+      }
+    },
+    getRecommendCuboids: {
+      method: 'GET',
+      params: {propName: 'cuboids', propValue: 'recommend'},
+      isArray: false,
+      interceptor: {
+        response: function(response) {
+          return transformCuboidsResponse(response.data);
+        }
+      }
+    },
+    optimize: {method: 'PUT', params: {action: 'optimize'}, isArray: false}
   });
 }]);

http://git-wip-us.apache.org/repos/asf/kylin/blob/1fce1930/webapp/app/less/app.less
----------------------------------------------------------------------
diff --git a/webapp/app/less/app.less b/webapp/app/less/app.less
index fcba436..7a23acc 100644
--- a/webapp/app/less/app.less
+++ b/webapp/app/less/app.less
@@ -899,4 +899,29 @@ pre {
     font-size: 18px;
     color: #6a6a6a;
   }
+}
+/* cube planner*/
+.cube-planner-column {
+  margin: 0 60px;
+  table {
+    border: 0;
+    tr {
+      font-weight: bolder;
+      color: #EEEEEE;
+      th {
+        text-align: center;
+        vertical-align: middle;
+        width: 20%;
+        padding: 2px;
+      }
+      .column-in-cuobid {
+        color: #9E9E9E;
+        font-weight: bolder;
+      }
+      .column-not-in-cuboid {
+        color: #EEEEEE;
+        font-weight: bolder;
+      }
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/1fce1930/webapp/app/partials/cubes/cube_detail.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubes/cube_detail.html b/webapp/app/partials/cubes/cube_detail.html
index 674e3f0..e80bb09 100755
--- a/webapp/app/partials/cubes/cube_detail.html
+++ b/webapp/app/partials/cubes/cube_detail.html
@@ -41,6 +41,9 @@
             ng-if="userService.hasRole('ROLE_ADMIN')">
             <a href="" ng-click="cube.visiblePage='hbase';getHbaseInfo(cube)">Storage</a>
         </li>
+        <li class="{{cube.visiblePage=='planner'? 'active':''}}" ng-if="userService.hasRole('ROLE_ADMIN') || hasPermission(cube, permissions.ADMINISTRATION.mask)">
+            <a href="" ng-click="cube.visiblePage='planner';getCubePlanner(cube);">Planner</a>
+        </li>
     </ul>
 
     <div class="cube-detail" ng-if="!cube.visiblePage || cube.visiblePage=='metadata'">
@@ -117,5 +120,46 @@
             </div>
         </div>
     </div>
-  </div>
+    <div class="cube-detail" ng-if="cube.visiblePage=='planner'">
+        <div style="padding: 15px;">
+            <div class="row">
+                <div class="col-sm-12">
+                    <h4 style="display: inline;">Cuboid Distribution</h4>
+                    <button ng-if="enableRecommend" class="btn btn-success btn-sm pull-right" ng-click="getRecommendCuboids(cube)" ng-if="currentData">
+                        Recommend
+                    </button>
+                    <div ng-if="cube.cuboid_last_optimized" class="pull-right" style="padding: 5px;">Last Optimized Time: {{cube.cuboid_last_optimized | utcToConfigTimeZone}}</div>
+                </div>
+            </div>
+            <div class="row">
+                <div class="col-md-6 col-sm-12">
+                     <nvd3 options="currentOptions" data="currentData" api="currentChart.api"></nvd3>
+                </div>
+                <div class="col-md-6 col-sm-12" ng-if="recommendData">
+                    <nvd3 options="recommendOptions" data="recommendData"></nvd3>
+                </div>
+            </div>
+            <div class="row cube-planner-column" ng-if="currentData || recommendData">
+                <table class="table table-bordered">
+                    <tbody>
+                        <tr ng-repeat="row in cube.detail.rowkey.rowkey_columns track by $index" ng-if="$index % 5 == 0" class="row">
+                            <th ng-class="{'column-in-cuobid': selectCuboid.charAt($index) == 1, 'column-not-in-cuboid': selectCuboid.charAt($index) == 0}">{{cube.detail.rowkey.rowkey_columns[$index].column}}</th>
+                            <th ng-class="{'column-in-cuobid': selectCuboid.charAt($index + 1) == 1, 'column-not-in-cuboid': selectCuboid.charAt($index + 1) == 0}">{{cube.detail.rowkey.rowkey_columns[$index + 1].column}}</th>
+                            <th ng-class="{'column-in-cuobid': selectCuboid.charAt($index + 2) == 1, 'column-not-in-cuboid': selectCuboid.charAt($index + 2) == 0}">{{cube.detail.rowkey.rowkey_columns[$index + 2].column}}</th>
+                            <th ng-class="{'column-in-cuobid': selectCuboid.charAt($index + 3) == 1, 'column-not-in-cuboid': selectCuboid.charAt($index + 3) == 0}">{{cube.detail.rowkey.rowkey_columns[$index + 3].column}}</th>
+                            <th ng-class="{'column-in-cuobid': selectCuboid.charAt($index + 4) == 1, 'column-not-in-cuboid': selectCuboid.charAt($index + 4) == 0}">{{cube.detail.rowkey.rowkey_columns[$index + 4].column}}</th>
+                        </tr>
+                    </tbody>
+                </table>
+            </div>
+            <div class="row">
+                <div class="col-sm-12">
+                    <button class="btn btn-success btn-next pull-right" ng-click="optimizeCuboids(cube)" ng-if="recommendData">
+                        Optimize
+                    </button>
+                </div>
+            </div>
+        </div>
+    </div>
+</div>
 


[04/18] kylin git commit: APACHE-KYLIN-2731: Introduce checkpoint executable

Posted by li...@apache.org.
APACHE-KYLIN-2731: Introduce checkpoint executable


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/fbfbee41
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/fbfbee41
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/fbfbee41

Branch: refs/heads/ci-dong
Commit: fbfbee4190481e1dd8522ec92eb6124c9f9a3c3f
Parents: e83a2e5
Author: Zhong <nj...@apache.org>
Authored: Fri Aug 25 11:35:26 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../kylin/cube/model/CubeBuildTypeEnum.java     |   7 +-
 .../kylin/job/execution/AbstractExecutable.java |   5 +
 .../job/execution/CheckpointExecutable.java     |  78 +++++++++
 .../engine/mr/common/JobInfoConverter.java      |  57 +++++--
 .../apache/kylin/rest/service/JobService.java   | 167 ++++++++++++++++++-
 5 files changed, 298 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/fbfbee41/core-cube/src/main/java/org/apache/kylin/cube/model/CubeBuildTypeEnum.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeBuildTypeEnum.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeBuildTypeEnum.java
index e3ae214..6a14025 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeBuildTypeEnum.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeBuildTypeEnum.java
@@ -35,5 +35,10 @@ public enum CubeBuildTypeEnum {
     /**
      * refresh segments
      */
-    REFRESH
+    REFRESH,
+
+    /**
+     * checkpoint for set of other jobs
+     */
+    CHECKPOINT
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/fbfbee41/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java b/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
index d36f598..30b6421 100644
--- a/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
@@ -384,6 +384,11 @@ public abstract class AbstractExecutable implements Executable, Idempotent {
         return getDuration(getStartTime(), getEndTime(), getInterruptTime());
     }
 
+    public boolean isReady() {
+        final Output output = getManager().getOutput(id);
+        return output.getState() == ExecutableState.READY;
+    }
+
     /*
     * discarded is triggered by JobService, the Scheduler is not awake of that
     *

http://git-wip-us.apache.org/repos/asf/kylin/blob/fbfbee41/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java b/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java
new file mode 100644
index 0000000..9864400
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/CheckpointExecutable.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.job.execution;
+
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+public class CheckpointExecutable extends DefaultChainedExecutable {
+
+    private static final Logger logger = LoggerFactory.getLogger(CheckpointExecutable.class);
+
+    private static final String DEPLOY_ENV_NAME = "envName";
+    private static final String PROJECT_INSTANCE_NAME = "projectName";
+
+    private final List<AbstractExecutable> subTasksForCheck = Lists.newArrayList();
+
+    public void addTaskForCheck(AbstractExecutable executable) {
+        this.subTasksForCheck.add(executable);
+    }
+
+    public void addTaskListForCheck(List<AbstractExecutable> executableList) {
+        this.subTasksForCheck.addAll(executableList);
+    }
+
+    public List<AbstractExecutable> getSubTasksForCheck() {
+        return subTasksForCheck;
+    }
+
+    @Override
+    public boolean isReady() {
+        if (!super.isReady()) {
+            return false;
+        }
+        for (Executable task : subTasksForCheck) {
+            final Output output = getManager().getOutput(task.getId());
+            if (output.getState() != ExecutableState.SUCCEED) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public String getDeployEnvName() {
+        return getParam(DEPLOY_ENV_NAME);
+    }
+
+    public void setDeployEnvName(String name) {
+        setParam(DEPLOY_ENV_NAME, name);
+    }
+
+    public String getProjectName() {
+        return getParam(PROJECT_INSTANCE_NAME);
+    }
+
+    public void setProjectName(String name) {
+        setParam(PROJECT_INSTANCE_NAME, name);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/fbfbee41/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/JobInfoConverter.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/JobInfoConverter.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/JobInfoConverter.java
index 9b8400c..3098c15 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/JobInfoConverter.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/JobInfoConverter.java
@@ -28,6 +28,7 @@ import org.apache.kylin.job.common.ShellExecutable;
 import org.apache.kylin.job.constant.JobStatusEnum;
 import org.apache.kylin.job.constant.JobStepStatusEnum;
 import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.CheckpointExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.execution.Output;
 import org.slf4j.Logger;
@@ -36,7 +37,7 @@ import org.slf4j.LoggerFactory;
 public class JobInfoConverter {
     private static final Logger logger = LoggerFactory.getLogger(JobInfoConverter.class);
 
-    public static JobInstance parseToJobInstanceQuietly(AbstractExecutable job, Map<String, Output> outputs) {
+    public static JobInstance parseToJobInstanceQuietly(CubingJob job, Map<String, Output> outputs) {
         try {
             return parseToJobInstance(job, outputs);
         } catch (Exception e) {
@@ -45,26 +46,29 @@ public class JobInfoConverter {
         }
     }
 
-    public static JobInstance parseToJobInstance(AbstractExecutable job, Map<String, Output> outputs) {
-        if (job == null) {
-            logger.warn("job is null.");
+    public static JobInstance parseToJobInstanceQuietly(CheckpointExecutable job, Map<String, Output> outputs) {
+        try {
+            return parseToJobInstance(job, outputs);
+        } catch (Exception e) {
+            logger.error("Failed to parse job instance: uuid={}", job, e);
             return null;
         }
+    }
 
-        if (!(job instanceof CubingJob)) {
-            logger.warn("illegal job type, id:" + job.getId());
+    public static JobInstance parseToJobInstance(CubingJob job, Map<String, Output> outputs) {
+        if (job == null) {
+            logger.warn("job is null.");
             return null;
         }
 
-        CubingJob cubeJob = (CubingJob) job;
         Output output = outputs.get(job.getId());
         final JobInstance result = new JobInstance();
         result.setName(job.getName());
-        result.setRelatedCube(CubingExecutableUtil.getCubeName(cubeJob.getParams()));
-        result.setRelatedSegment(CubingExecutableUtil.getSegmentId(cubeJob.getParams()));
+        result.setRelatedCube(CubingExecutableUtil.getCubeName(job.getParams()));
+        result.setRelatedSegment(CubingExecutableUtil.getSegmentId(job.getParams()));
         result.setLastModified(output.getLastModified());
-        result.setSubmitter(cubeJob.getSubmitter());
-        result.setUuid(cubeJob.getId());
+        result.setSubmitter(job.getSubmitter());
+        result.setUuid(job.getId());
         result.setType(CubeBuildTypeEnum.BUILD);
         result.setStatus(parseToJobStatus(output.getState()));
         result.setMrWaiting(AbstractExecutable.getExtraInfoAsLong(output, CubingJob.MAP_REDUCE_WAIT_TIME, 0L) / 1000);
@@ -73,8 +77,35 @@ public class JobInfoConverter {
         result.setExecInterruptTime(AbstractExecutable.getInterruptTime(output));
         result.setDuration(AbstractExecutable.getDuration(result.getExecStartTime(), result.getExecEndTime(),
                 result.getExecInterruptTime()) / 1000);
-        for (int i = 0; i < cubeJob.getTasks().size(); ++i) {
-            AbstractExecutable task = cubeJob.getTasks().get(i);
+        for (int i = 0; i < job.getTasks().size(); ++i) {
+            AbstractExecutable task = job.getTasks().get(i);
+            result.addStep(parseToJobStep(task, i, outputs.get(task.getId())));
+        }
+        return result;
+    }
+
+    public static JobInstance parseToJobInstance(CheckpointExecutable job, Map<String, Output> outputs) {
+        if (job == null) {
+            logger.warn("job is null.");
+            return null;
+        }
+
+        Output output = outputs.get(job.getId());
+        final JobInstance result = new JobInstance();
+        result.setName(job.getName());
+        result.setRelatedCube(CubingExecutableUtil.getCubeName(job.getParams()));
+        result.setLastModified(output.getLastModified());
+        result.setSubmitter(job.getSubmitter());
+        result.setUuid(job.getId());
+        result.setType(CubeBuildTypeEnum.CHECKPOINT);
+        result.setStatus(parseToJobStatus(output.getState()));
+        result.setExecStartTime(AbstractExecutable.getStartTime(output));
+        result.setExecEndTime(AbstractExecutable.getEndTime(output));
+        result.setExecInterruptTime(AbstractExecutable.getInterruptTime(output));
+        result.setDuration(AbstractExecutable.getDuration(result.getExecStartTime(), result.getExecEndTime(),
+                result.getExecInterruptTime()) / 1000);
+        for (int i = 0; i < job.getTasks().size(); ++i) {
+            AbstractExecutable task = job.getTasks().get(i);
             result.addStep(parseToJobStep(task, i, outputs.get(task.getId())));
         }
         return result;

http://git-wip-us.apache.org/repos/asf/kylin/blob/fbfbee41/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
index d27b39a..529f3b8 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
@@ -50,6 +50,7 @@ import org.apache.kylin.job.constant.JobTimeFilterEnum;
 import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.job.exception.SchedulerException;
 import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.CheckpointExecutable;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.execution.Output;
@@ -317,6 +318,33 @@ public class JobService extends BasicService implements InitializingBean {
         return result;
     }
 
+    protected JobInstance getCheckpointJobInstance(AbstractExecutable job) {
+        Message msg = MsgPicker.getMsg();
+
+        if (job == null) {
+            return null;
+        }
+        if (!(job instanceof CheckpointExecutable)) {
+            throw new BadRequestException(String.format(msg.getILLEGAL_JOB_TYPE(), job.getId()));
+        }
+
+        CheckpointExecutable checkpointExecutable = (CheckpointExecutable) job;
+        final JobInstance result = new JobInstance();
+        result.setName(job.getName());
+        result.setRelatedCube(CubingExecutableUtil.getCubeName(job.getParams()));
+        result.setLastModified(job.getLastModified());
+        result.setSubmitter(job.getSubmitter());
+        result.setUuid(job.getId());
+        result.setType(CubeBuildTypeEnum.CHECKPOINT);
+        result.setStatus(JobInfoConverter.parseToJobStatus(job.getStatus()));
+        result.setDuration(job.getDuration() / 1000);
+        for (int i = 0; i < checkpointExecutable.getTasks().size(); ++i) {
+            AbstractExecutable task = checkpointExecutable.getTasks().get(i);
+            result.addStep(JobInfoConverter.parseToJobStep(task, i, getExecutableManager().getOutput(task.getId())));
+        }
+        return result;
+    }
+
     public void resumeJob(JobInstance job) {
         aclEvaluate.checkProjectOperationPermission(job);
         getExecutableManager().resumeJob(job.getId());
@@ -373,6 +401,7 @@ public class JobService extends BasicService implements InitializingBean {
         Integer limit = (null == limitValue) ? 30 : limitValue;
         Integer offset = (null == offsetValue) ? 0 : offsetValue;
         List<JobInstance> jobs = searchJobsByCubeName(cubeNameSubstring, projectName, statusList, timeFilter);
+
         Collections.sort(jobs);
 
         if (jobs.size() <= offset) {
@@ -388,12 +417,40 @@ public class JobService extends BasicService implements InitializingBean {
 
     public List<JobInstance> searchJobsByCubeName(final String cubeNameSubstring, final String projectName,
             final List<JobStatusEnum> statusList, final JobTimeFilterEnum timeFilter) {
-        return innerSearchCubingJobs(cubeNameSubstring, null, projectName, statusList, timeFilter);
+        return searchJobsByCubeName(cubeNameSubstring, projectName, statusList, timeFilter, JobSearchMode.ALL);
+    }
+
+    public List<JobInstance> searchJobsByCubeName(final String cubeNameSubstring, final String projectName,
+            final List<JobStatusEnum> statusList, final JobTimeFilterEnum timeFilter, JobSearchMode jobSearchMode) {
+        return innerSearchJobs(cubeNameSubstring, null, projectName, statusList, timeFilter, jobSearchMode);
     }
 
     public List<JobInstance> searchJobsByJobName(final String jobName, final String projectName,
             final List<JobStatusEnum> statusList, final JobTimeFilterEnum timeFilter) {
-        return innerSearchCubingJobs(null, jobName, projectName, statusList, timeFilter);
+        return searchJobsByJobName(jobName, projectName, statusList, timeFilter, JobSearchMode.ALL);
+    }
+
+    public List<JobInstance> searchJobsByJobName(final String jobName, final String projectName,
+            final List<JobStatusEnum> statusList, final JobTimeFilterEnum timeFilter, JobSearchMode jobSearchMode) {
+        return innerSearchJobs(null, jobName, projectName, statusList, timeFilter, jobSearchMode);
+    }
+
+    public List<JobInstance> innerSearchJobs(final String cubeName, final String jobName, final String projectName,
+            final List<JobStatusEnum> statusList, final JobTimeFilterEnum timeFilter, JobSearchMode jobSearchMode) {
+        List<JobInstance> result = Lists.newArrayList();
+        switch (jobSearchMode) {
+        case CUBING_ONLY:
+            result.addAll(innerSearchCubingJobs(cubeName, jobName, projectName, statusList, timeFilter));
+            break;
+        case CHECKPOINT_ONLY:
+            result.addAll(innerSearchCheckpointJobs(cubeName, jobName, projectName, statusList, timeFilter));
+            break;
+        case ALL:
+        default:
+            result.addAll(innerSearchCubingJobs(cubeName, jobName, projectName, statusList, timeFilter));
+            result.addAll(innerSearchCheckpointJobs(cubeName, jobName, projectName, statusList, timeFilter));
+        }
+        return result;
     }
 
     public List<JobInstance> innerSearchCubingJobs(final String cubeName, final String jobName,
@@ -503,6 +560,109 @@ public class JobService extends BasicService implements InitializingBean {
         return results;
     }
 
+    public List<JobInstance> innerSearchCheckpointJobs(final String cubeName, final String jobName,
+            final String projectName, final List<JobStatusEnum> statusList, final JobTimeFilterEnum timeFilter) {
+        // prepare time range
+        Calendar calendar = Calendar.getInstance();
+        calendar.setTime(new Date());
+        long timeStartInMillis = getTimeStartInMillis(calendar, timeFilter);
+        long timeEndInMillis = Long.MAX_VALUE;
+        Set<ExecutableState> states = convertStatusEnumToStates(statusList);
+        final Map<String, Output> allOutputs = getExecutableManager().getAllOutputs(timeStartInMillis, timeEndInMillis);
+
+        return Lists
+                .newArrayList(FluentIterable
+                        .from(innerSearchCheckpointJobs(cubeName, jobName, states, timeStartInMillis, timeEndInMillis,
+                                allOutputs, false, projectName))
+                        .transform(new Function<CheckpointExecutable, JobInstance>() {
+                            @Override
+                            public JobInstance apply(CheckpointExecutable checkpointExecutable) {
+                                return JobInfoConverter.parseToJobInstanceQuietly(checkpointExecutable, allOutputs);
+                            }
+                        }));
+    }
+
+    public List<CheckpointExecutable> innerSearchCheckpointJobs(final String cubeName, final String jobName,
+            final Set<ExecutableState> statusList, long timeStartInMillis, long timeEndInMillis,
+            final Map<String, Output> allOutputs, final boolean nameExactMatch, final String projectName) {
+        List<CheckpointExecutable> results = Lists
+                .newArrayList(
+                        FluentIterable
+                                .from(getExecutableManager().getAllAbstractExecutables(timeStartInMillis,
+                                        timeEndInMillis, CheckpointExecutable.class))
+                                .filter(new Predicate<AbstractExecutable>() {
+                                    @Override
+                                    public boolean apply(AbstractExecutable executable) {
+                                        if (executable instanceof CheckpointExecutable) {
+                                            if (StringUtils.isEmpty(cubeName)) {
+                                                return true;
+                                            }
+                                            String executableCubeName = CubingExecutableUtil
+                                                    .getCubeName(executable.getParams());
+                                            if (executableCubeName == null)
+                                                return true;
+                                            if (nameExactMatch)
+                                                return executableCubeName.equalsIgnoreCase(cubeName);
+                                            else
+                                                return executableCubeName.toLowerCase()
+                                                        .contains(cubeName.toLowerCase());
+                                        } else {
+                                            return false;
+                                        }
+                                    }
+                                }).transform(new Function<AbstractExecutable, CheckpointExecutable>() {
+                                    @Override
+                                    public CheckpointExecutable apply(AbstractExecutable executable) {
+                                        return (CheckpointExecutable) executable;
+                                    }
+                                }).filter(Predicates.and(new Predicate<CheckpointExecutable>() {
+                                    @Override
+                                    public boolean apply(CheckpointExecutable executable) {
+                                        if (null == projectName
+                                                || null == getProjectManager().getProject(projectName)) {
+                                            return true;
+                                        } else {
+                                            return projectName.equalsIgnoreCase(executable.getProjectName());
+                                        }
+                                    }
+                                }, new Predicate<CheckpointExecutable>() {
+                                    @Override
+                                    public boolean apply(CheckpointExecutable executable) {
+                                        try {
+                                            Output output = allOutputs.get(executable.getId());
+                                            if (output == null) {
+                                                return false;
+                                            }
+
+                                            ExecutableState state = output.getState();
+                                            boolean ret = statusList.contains(state);
+                                            return ret;
+                                        } catch (Exception e) {
+                                            throw e;
+                                        }
+                                    }
+                                }, new Predicate<CheckpointExecutable>() {
+                                    @Override
+                                    public boolean apply(@Nullable CheckpointExecutable checkpointExecutable) {
+                                        if (checkpointExecutable == null) {
+                                            return false;
+                                        }
+
+                                        if (Strings.isEmpty(jobName)) {
+                                            return true;
+                                        }
+
+                                        if (nameExactMatch) {
+                                            return checkpointExecutable.getName().equalsIgnoreCase(jobName);
+                                        } else {
+                                            return checkpointExecutable.getName().toLowerCase()
+                                                    .contains(jobName.toLowerCase());
+                                        }
+                                    }
+                                })));
+        return results;
+    }
+
     public List<CubingJob> listJobsByRealizationName(final String realizationName, final String projectName,
             final Set<ExecutableState> statusList) {
         return innerSearchCubingJobs(realizationName, null, statusList, 0L, Long.MAX_VALUE,
@@ -513,4 +673,7 @@ public class JobService extends BasicService implements InitializingBean {
         return listJobsByRealizationName(realizationName, projectName, EnumSet.allOf(ExecutableState.class));
     }
 
+    public enum JobSearchMode {
+        CUBING_ONLY, CHECKPOINT_ONLY, ALL
+    }
 }


[15/18] kylin git commit: APACHE-KYLIN-2843 Upgrade nvd3 version

Posted by li...@apache.org.
APACHE-KYLIN-2843 Upgrade nvd3 version


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/f7d265b6
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/f7d265b6
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/f7d265b6

Branch: refs/heads/ci-dong
Commit: f7d265b613df03569b7f3479637ea9bcc226a882
Parents: 70262f3
Author: liapan <li...@ebay.com>
Authored: Fri Nov 17 14:43:46 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 webapp/app/index.html                       |   9 +-
 webapp/app/js/app.js                        |   2 +-
 webapp/app/js/controllers/query.js          | 110 ++++++++++++------
 webapp/app/js/factories/graph.js            |  77 -------------
 webapp/app/js/model/queryConfig.js          | 139 +++++++++++++++++++++++
 webapp/app/js/services/graph.js             |  54 ---------
 webapp/app/partials/query/query_detail.html |  90 ++++-----------
 webapp/bower.json                           |   8 +-
 webapp/grunt.json                           |   6 +-
 9 files changed, 246 insertions(+), 249 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/f7d265b6/webapp/app/index.html
----------------------------------------------------------------------
diff --git a/webapp/app/index.html b/webapp/app/index.html
index 407f179..d235b06 100644
--- a/webapp/app/index.html
+++ b/webapp/app/index.html
@@ -45,7 +45,7 @@
   <link rel="stylesheet" type="text/css" href="components/chosen/chosen.css">
   <link rel="stylesheet" type="text/css" href="components/angular-chosen-localytics/chosen-spinner.css">
   <link rel="stylesheet" type="text/css" href="components/animate.css/animate.css">
-  <link rel="stylesheet" type="text/css" href="components/nvd3/nv.d3.min.css">
+  <link rel="stylesheet" type="text/css" href="components/nvd3/build/nv.d3.min.css">
 
   <link rel="stylesheet" type="text/css" href="css/AdminLTE.css">
   <link rel="stylesheet" type="text/css" href="components/bootstrap-sweetalert/lib/sweet-alert.css">
@@ -102,8 +102,7 @@
 
 <script src="components/moment/moment.js"></script>
 <script src="components/d3/d3.min.js"></script>
-<script src="components/nvd3/nv.d3.min.js"></script>
-<script src="components/angularjs-nvd3-directives/dist/angularjs-nvd3-directives.js"></script>
+<script src="components/nvd3/build/nv.d3.min.js"></script>
 <script src="components/bootstrap-sweetalert/lib/sweet-alert.js"></script>
 <script src="components/angular-sweetalert/SweetAlert.js"></script>
 <script src="components/underscore/underscore.js"></script>
@@ -112,6 +111,7 @@
 <script src="components/angular-ui-sortable/sortable.js"></script>
 <script src="components/angular-toggle-switch/angular-toggle-switch.js"></script>
 <script src="components/angular-sanitize/angular-sanitize.js"></script>
+<script src="components/angular-nvd3/dist/angular-nvd3.min.js"></script>
 
 <script src="js/app.js"></script>
 <script src="js/config.js"></script>
@@ -124,7 +124,6 @@
 <script src="js/directives/select.js"></script>
 <script src="js/directives/ui-grid.js"></script>
 
-<script src="js/factories/graph.js"></script>
 <script src="js/services/cache.js"></script>
 <script src="js/services/message.js"></script>
 <script src="js/services/access.js"></script>
@@ -136,7 +135,6 @@
 <script src="js/services/encodings.js"></script>
 <script src="js/services/cubes.js"></script>
 <script src="js/services/streaming.js"></script>
-<script src="js/services/graph.js"></script>
 <script src="js/services/jobs.js"></script>
 <script src="js/services/message.js"></script>
 <script src="js/services/projects.js"></script>
@@ -164,6 +162,7 @@
 <script src="js/model/cubeListModel.js"></script>
 <script src="js/model/jobListModel.js"></script>
 <script src="js/model/cubesManager.js"></script>
+<script src="js/model/queryConfig.js"></script>
 
 <!--New GUI-->
 <script src="js/model/modelsManager.js"></script>

http://git-wip-us.apache.org/repos/asf/kylin/blob/f7d265b6/webapp/app/js/app.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/app.js b/webapp/app/js/app.js
index 629617e..44a91ed 100644
--- a/webapp/app/js/app.js
+++ b/webapp/app/js/app.js
@@ -17,4 +17,4 @@
  */
 
 //Kylin Application Module
-KylinApp = angular.module('kylin', ['ngRoute', 'ngResource', 'ngGrid', 'ui.grid', 'ui.grid.resizeColumns', 'ui.grid.grouping', 'ui.bootstrap', 'ui.ace', 'base64', 'angularLocalStorage', 'localytics.directives', 'treeControl', 'nvd3ChartDirectives', 'ngLoadingRequest', 'oitozero.ngSweetAlert', 'ngCookies', 'angular-underscore', 'ngAnimate', 'ui.sortable', 'angularBootstrapNavTree', 'toggle-switch', 'ngSanitize', 'ui.select', 'ui.bootstrap.datetimepicker']);
+KylinApp = angular.module('kylin', ['ngRoute', 'ngResource', 'ngGrid', 'ui.grid', 'ui.grid.resizeColumns', 'ui.grid.grouping', 'ui.bootstrap', 'ui.ace', 'base64', 'angularLocalStorage', 'localytics.directives', 'treeControl', 'ngLoadingRequest', 'oitozero.ngSweetAlert', 'ngCookies', 'angular-underscore', 'ngAnimate', 'ui.sortable', 'angularBootstrapNavTree', 'toggle-switch', 'ngSanitize', 'ui.select', 'ui.bootstrap.datetimepicker', 'nvd3']);

http://git-wip-us.apache.org/repos/asf/kylin/blob/f7d265b6/webapp/app/js/controllers/query.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/query.js b/webapp/app/js/controllers/query.js
index 945ddef..4014093 100644
--- a/webapp/app/js/controllers/query.js
+++ b/webapp/app/js/controllers/query.js
@@ -282,15 +282,6 @@ KylinApp
             $scope.query($scope.curQuery);
         }
 
-        $scope.resetGraph = function (query) {
-            var dimension = (query.graph.meta.dimensions && query.graph.meta.dimensions.length > 0) ? query.graph.meta.dimensions[0] : null;
-            var metrics = (query.graph.meta.metrics && query.graph.meta.metrics.length > 0) ? query.graph.meta.metrics[0] : null;
-            query.graph.state = {
-                dimensions: dimension,
-                metrics: ((query.graph.type.metrics.multiple) ? [metrics] : metrics)
-            };
-        }
-
         $scope.loadMore = function (query) {
             query.result.loading = true;
             var query = query;
@@ -424,7 +415,7 @@ KylinApp
 
 
     })
-    .controller('QueryResultCtrl', function ($scope, storage, $base64, $q, $location, $anchorScroll, $routeParams, QueryService, GraphService) {
+    .controller('QueryResultCtrl', function ($scope, storage, $base64, $q, $location, $anchorScroll, $routeParams, QueryService, queryConfig) {
         $scope.buildGraphMetadata = function (query) {
             if (!query.graph.show) {
                 return;
@@ -467,34 +458,85 @@ KylinApp
             return $scope.curQuery.graph.type.dimension.types.indexOf(dimension.type) > -1;
         }
 
-        $scope.refreshGraphData = function (query) {
-            if (query.graph.show) {
-                query.graph.data = GraphService.buildGraph(query);
-            }
-            else {
-                query.graph.data = [];
-            }
+        $scope.resetGraph = function (query) {
+            var dimension = (query.graph.meta.dimensions && query.graph.meta.dimensions.length > 0) ? query.graph.meta.dimensions[0] : null;
+            var metrics = (query.graph.meta.metrics && query.graph.meta.metrics.length > 0) ? query.graph.meta.metrics[0] : null;
+            query.graph.state = {
+                dimensions: dimension,
+                metrics: ((query.graph.type.metrics.multiple) ? [metrics] : metrics)
+            };
+            $scope.refreshGraphData(query);
         }
 
-        $scope.xAxisTickFormatFunction = function () {
-            return function (d) {
-                return d3.time.format("%Y-%m-%d")(moment.unix(d).toDate());
-            }
-        };
+        $scope.refreshGraphData = function (query) {
+            if (query.graph.show) {
+                $scope.chart = undefined;
+
+                var selectedDimension = query.graph.state.dimensions;
+                if (selectedDimension && query.graph.type.dimension.types.indexOf(selectedDimension.type) > -1) {
+                    $scope.chart = {};
+
+                    var chartType = query.graph.type.value;
+                    var selectedMetric = query.graph.state.metrics;
+
+                    var dataValues = [];
+                    angular.forEach(query.result.results, function(result, ind) {
+                        var data = {
+                            label: result[selectedDimension.index],
+                            value: parseFloat(result[selectedMetric.index])
+                        };
+                        if (selectedDimension.type === 'date' && chartType === 'line') {
+                            data.label = parseInt(moment(data.label).format('X'));
+                        }
+                        dataValues.push(data);
+                    });
 
-        $scope.xFunction = function () {
-            return function (d) {
-                return d.key;
-            }
-        };
+                    dataValues = _.sortBy(dataValues, 'label');
+                    var oldLabel = dataValues[0].label;
+                    var groupValues = [{label: dataValues[0].label, value: 0}];
+                    angular.forEach(dataValues, function(data) {
+                        if (data.label === oldLabel) {
+                            groupValues[groupValues.length-1].value += data.value;
+                        } else {
+                            groupValues.push(data);
+                            oldLabel = data.label;
+                        }
+                    });
 
-        $scope.yFunction = function () {
-            return function (d) {
-                return d.y;
+                    $scope.chart.data = [{
+                        key: selectedMetric.column.label,
+                        values: groupValues
+                    }];
+
+                    if (chartType === 'line') {
+                        $scope.chart.options = angular.copy(queryConfig.lineChartOptions);
+                        if (selectedDimension.type === 'date') {
+                            $scope.chart.options.chart.xAxis.tickFormat = function (d) {
+                                return d3.time.format('%Y-%m-%d')(moment.unix(d).toDate());
+                            };
+                        }
+                    } else if (chartType === 'bar') {
+                        $scope.chart.options = angular.copy(queryConfig.barChartOptions);
+                        if (groupValues.length > 15) {
+                            $scope.chart.options.chart.showLegend = false;
+                            $scope.chart.options.chart.xAxis.height = 100;
+                            $scope.chart.options.chart.margin.bottom =  150;
+                            $scope.chart.options.chart.xAxis.rotateLabels = -90;
+                            if (groupValues.length > 50) {
+                                $scope.chart.options.chart.showXAxis = false;
+                            }
+                        }
+                    } else if (chartType === 'pie') {
+                        $scope.chart.options = angular.copy(queryConfig.pieChartOptions);
+                        $scope.chart.data = groupValues;
+                        if (groupValues.length > 15) {
+                            $scope.chart.options.chart.showLegend = false;
+                            $scope.chart.options.chart.showLabels = false;
+                        }
+                    }
+                }
+            } else {
+                $scope.chart.data = [];
             }
         }
-
-        $scope.$on('elementClick.directive', function (angularEvent, event) {
-            console.log('clicked.');
-        });
     });

http://git-wip-us.apache.org/repos/asf/kylin/blob/f7d265b6/webapp/app/js/factories/graph.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/factories/graph.js b/webapp/app/js/factories/graph.js
deleted file mode 100644
index e02e13a..0000000
--- a/webapp/app/js/factories/graph.js
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-KylinApp.factory('GraphBuilder', function () {
-  var graphBuilder = {};
-
-  graphBuilder.buildLineGraph = function (dimension, metrics, aggregatedData) {
-    var values = [];
-    angular.forEach(getSortedKeys(aggregatedData), function (sortedKey, index) {
-      values.push([(dimension.type == 'date') ? moment(sortedKey).unix() : sortedKey, aggregatedData[sortedKey]]);
-    });
-
-    var newGraph = [
-      {
-        "key": metrics.column.label,
-        "values": values
-      }
-    ];
-
-    return newGraph;
-  }
-
-  graphBuilder.buildBarGraph = function (dimension, metrics, aggregatedData) {
-    var newGraph = [];
-    angular.forEach(getSortedKeys(aggregatedData), function (sortedKey, index) {
-      newGraph.push({
-        key: sortedKey,
-        values: [
-          [sortedKey, aggregatedData[sortedKey]]
-        ]
-      });
-    });
-
-    return newGraph;
-  }
-
-  graphBuilder.buildPieGraph = function (dimension, metrics, aggregatedData) {
-    var newGraph = [];
-    angular.forEach(getSortedKeys(aggregatedData), function (sortedKey, index) {
-      newGraph.push({
-        key: sortedKey,
-        y: aggregatedData[sortedKey]
-      });
-    });
-
-    return newGraph;
-  }
-
-  function getSortedKeys(results) {
-    var sortedKeys = [];
-    for (var k in results) {
-      if (results.hasOwnProperty(k)) {
-        sortedKeys.push(k);
-      }
-    }
-    sortedKeys.sort();
-
-    return sortedKeys;
-  }
-
-  return graphBuilder;
-});

http://git-wip-us.apache.org/repos/asf/kylin/blob/f7d265b6/webapp/app/js/model/queryConfig.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/queryConfig.js b/webapp/app/js/model/queryConfig.js
new file mode 100644
index 0000000..2f92f57
--- /dev/null
+++ b/webapp/app/js/model/queryConfig.js
@@ -0,0 +1,139 @@
+/*
+ * 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.
+*/
+
+KylinApp.constant('queryConfig', {
+  lineChartOptions: {
+    chart: {
+      type: 'lineChart',
+      height: 500,
+      margin : {
+        top: 20,
+        right: 55,
+        bottom: 60,
+        left: 55
+      },
+      useInteractiveGuideline: true,
+      interpolate: 'cardinal',
+      x: function(d){return d.label;},
+      y: function(d){return d.value;},
+      xAxis: {
+        axisLabelDistance: 50,
+        staggerLabels: false,
+        tickFormat: function(d) {
+          if (d.length > 10) {
+            return d.substring(0,10) + '...';
+          } else {
+            return d;
+          }
+        }
+      },
+      yAxis: {
+        tickFormat: function(d) {
+          if (d < 1000) {
+            if (parseFloat(d) === d) {
+              return d3.format('.1')(d);
+            } else {
+              return d3.format('.2f')(d);
+            }
+          } else {
+            var prefix = d3.formatPrefix(d);
+            return prefix.scale(d) + prefix.symbol;
+          }
+        },
+        showMaxMin: false
+      },
+      valueFormat: function(d){
+        return d3.format('.1')(d);
+      },
+      transitionDuration: 500,
+      tooltipContent: function (key, x, y, e, graph) {
+        return '<h3>' + e.point.label + '</h3>' + '<p>' +  y + '</p>';
+      }
+    }
+  },
+  barChartOptions: {
+    chart: {
+      type: 'discreteBarChart',
+      height: 500,
+      margin : {
+        top: 20,
+        right: 20,
+        bottom: 60,
+        left: 55
+      },
+      x: function(d){return d.label;},
+      y: function(d){return d.value;},
+      xAxis: {
+        axisLabelDistance: 50,
+        staggerLabels: false,
+        tickFormat: function(d) {
+          if (d.length > 10) {
+            return d.substring(0,10) + '...';
+          } else {
+            return d;
+          }
+        }
+      },
+      yAxis: {
+        tickFormat: function(d) {
+          if (d < 1000) {
+            if (parseFloat(d) === d) {
+              return d3.format('.1')(d);
+            } else {
+              return d3.format('.2f')(d);
+            }
+          } else {
+            var prefix = d3.formatPrefix(d);
+            return prefix.scale(d) + prefix.symbol;
+          }
+        },
+        showMaxMin: false
+      },
+      valueFormat: function(d){
+        return d3.format('.1')(d);
+      },
+      transitionDuration: 500,
+      tooltipContent: function (key, x, y, e, graph) {
+        return '<h3>' + e.point.label + '</h3>' + '<p>' +  y + '</p>';
+      }
+    }
+  },
+  pieChartOptions: {
+    chart: {
+      type: 'pieChart',
+      height: 500,
+      showLabels: true,
+      duration: 500,
+      labelThreshold: 0.01,
+      labelSunbeamLayout: true,
+      legend: {
+        margin : {
+          top: 20,
+          right: 20,
+          bottom: 60,
+          left: 55
+        }
+      },
+      x: function(d){return d.label;},
+      y: function(d){return d.value;},
+      valueFormat: function(d){
+        return d3.format('.1')(d);
+      }
+    }
+  }
+});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/f7d265b6/webapp/app/js/services/graph.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/services/graph.js b/webapp/app/js/services/graph.js
deleted file mode 100644
index dc69519..0000000
--- a/webapp/app/js/services/graph.js
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-KylinApp.service('GraphService', function (GraphBuilder, VdmUtil) {
-
-  this.buildGraph = function (query) {
-    var graphData = null;
-    var dimension = query.graph.state.dimensions;
-
-    if (dimension && query.graph.type.dimension.types.indexOf(dimension.type) > -1) {
-      var metricsList = [];
-      metricsList = metricsList.concat(query.graph.state.metrics);
-      angular.forEach(metricsList, function (metrics, index) {
-        var aggregatedData = {};
-        angular.forEach(query.result.results,function(row,index){
-          angular.forEach(row,function(column,value){
-            var float = VdmUtil.SCToFloat(column);
-              if (float!=""){
-                query.result.results[index][value]=float;
-              }
-          });
-        });
-        angular.forEach(query.result.results, function (data, index) {
-          aggregatedData[data[dimension.index]] = (!!aggregatedData[data[dimension.index]] ? aggregatedData[data[dimension.index]] : 0)
-          + parseFloat(data[metrics.index].replace(/[^\d\.\-]/g, ""));
-        });
-
-        var newData = GraphBuilder["build" + capitaliseFirstLetter(query.graph.type.value) + "Graph"](dimension, metrics, aggregatedData);
-        graphData = (!!graphData) ? graphData.concat(newData) : newData;
-      });
-    }
-
-    return graphData;
-  }
-
-  function capitaliseFirstLetter(string) {
-    return string.charAt(0).toUpperCase() + string.slice(1);
-  }
-});

http://git-wip-us.apache.org/repos/asf/kylin/blob/f7d265b6/webapp/app/partials/query/query_detail.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/query/query_detail.html b/webapp/app/partials/query/query_detail.html
index 8e1286c..c94a709 100644
--- a/webapp/app/partials/query/query_detail.html
+++ b/webapp/app/partials/query/query_detail.html
@@ -82,7 +82,7 @@
       </button>
       </span>
     <div class="pull-right" ng-if="curQuery.status=='success'">
-        <button class="btn btn-default btn-xs"  ng-click="curQuery.graph.show=!curQuery.graph.show;buildGraphMetadata(curQuery);resetGraph(curQuery);refreshGraphData(curQuery)">
+        <button class="btn btn-default btn-xs"  ng-click="curQuery.graph.show=!curQuery.graph.show;buildGraphMetadata(curQuery);resetGraph(curQuery)">
             <span ng-if="!curQuery.graph.show"><i class="fa fa-bar-chart-o"></i> Visualization</span>
             <span ng-if="curQuery.graph.show"><i class="fa fa-list-ul"></i> Grid</span>
         </button>
@@ -126,42 +126,34 @@
     </div>
 
     <div ng-if="curQuery.graph.show" class="row">
-        <div class="col-xs-4 col-lg-3">
+        <div class="col-xs-4">
             <div class="graph_content">
                 <label>Graph Type</label>
-
-                <select ng-model="curQuery.graph.type"
-                        ng-change="resetGraph(curQuery);refreshGraphData(curQuery)"
+                <div>
+                    <select chosen style="width: 100%;"
+                        ng-model="curQuery.graph.type"
+                        ng-change="resetGraph(curQuery)"
                         ng-options="type as type.name for type in chartTypes"></select>
+                </div>
             </div>
-
+        </div>
+        <div class="col-xs-4">
             <div class="graph_content">
                 <label>Dimensions</label>
                 <div>
-                    <select
-                            chosen style="width: 100%;"
+                    <select chosen style="width: 100%;"
                             ng-model="curQuery.graph.state.dimensions" ng-change="refreshGraphData(curQuery)"
                             ng-options="dimension as dimension.column.label for dimension in curQuery.graph.meta.dimensions | filter: mappingToChartype">
                         <option value="">-- choose dimension --</option>
                     </select>
                 </div>
             </div>
-
+        </div>
+        <div class="col-xs-4">
             <div class="graph_content">
                 <label>Metrics</label>
-
                 <div>
-                    <select ng-if="curQuery.graph.type.metrics.multiple"
-                            chosen style="width: 100%;"
-                            multiple
-                            data-placeholder="Select Metrics.."
-                            ng-model="curQuery.graph.state.metrics"
-                            ng-options="metrics as metrics.column.label for metrics in curQuery.graph.meta.metrics"
-                            ng-change="refreshGraphData(curQuery)">
-                        <option value="">-- choose metrics --</option>
-                    </select>
-                    <select ng-if="!curQuery.graph.type.metrics.multiple"
-                            chosen style="width: 100%"
+                    <select chosen style="width: 100%;"
                             data-placeholder="Select Metrics.."
                             ng-model="curQuery.graph.state.metrics"
                             ng-options="metrics as metrics.column.label for metrics in curQuery.graph.meta.metrics"
@@ -171,57 +163,13 @@
                 </div>
             </div>
         </div>
-
-        <div class="col-xs-8 col-lg-9">
-            <div ng-if="curQuery.graph.data.length > 0">
-                <div ng-if="curQuery.graph.type.value == 'bar'">
-                    <nvd3-multi-bar-chart
-                            data="curQuery.graph.data"
-                            height="{{curQuery.graph.data.length * 5 + 250}}"
-                            showXAxis="true"
-                            showYAxis="true"
-                            noData="No Graph"
-                            showLegend="true"
-                            interactive="true"
-                            tooltips="true"
-                            >
-                        <svg></svg>
-                    </nvd3-multi-bar-chart>
-                </div>
-
-                <div ng-if="curQuery.graph.type.value == 'line'">
-                    <nvd3-line-chart
-                            height="350"
-                            data="curQuery.graph.data"
-                            xAxisTickFormat="xAxisTickFormatFunction()"
-                            showXAxis="true"
-                            showYAxis="true"
-                            noData="No Graph"
-                            clipVoronoi="true"
-                            interactive="true"
-                            tooltips="true"
-                            >
-                        <svg></svg>
-                    </nvd3-line-chart>
-                </div>
-
-                <div ng-if="curQuery.graph.type.value == 'pie'">
-                    <nvd3-pie-chart
-                            data="curQuery.graph.data"
-                            height="{{ curQuery.graph.data.length | resizePieHeight }}"
-                            x="xFunction()"
-                            y="yFunction()"
-                            noData="No Graph"
-                            tooltips="true"
-                            showLabels="true"
-                            showLegend="true"
-                            >
-                        <svg></svg>
-                    </nvd3-pie-chart>
-                </div>
+    </div>
+    <div ng-if="curQuery.graph.show" class="row">
+         <div class="col-xs-12">
+            <div ng-if="chart">
+                <nvd3 options="chart.options" data="chart.data"></nvd3>
             </div>
-
-            <div ng-if="!curQuery.graph.data || curQuery.graph.data.length == 0" style="padding-top: 10%">
+             <div ng-if="!chart" style="padding-top: 10%">
                 <div no-result text="No Graph Generated."></div>
             </div>
         </div>

http://git-wip-us.apache.org/repos/asf/kylin/blob/f7d265b6/webapp/bower.json
----------------------------------------------------------------------
diff --git a/webapp/bower.json b/webapp/bower.json
index 4d5a311..eac2cb3 100755
--- a/webapp/bower.json
+++ b/webapp/bower.json
@@ -19,8 +19,7 @@
     "messenger": "1.4.1",
     "moment": "2.5.1",
     "d3": "3.4.4",
-    "nvd3": "1.1.15-beta",
-    "angularjs-nvd3-directives": "0.0.5-beta",
+    "nvd3": "1.8.4",
     "angular-sweetalert": "~1.0.3",
     "angular-underscore": "~0.5.0",
     "angular-ui-sortable": "0.13.1",
@@ -35,7 +34,8 @@
     "angular-ui-select": "0.13.2",
     "angular-sanitize": "1.2.18",
     "angular-tree-control": "0.2.8",
-    "angular-bootstrap-datetimepicker": "0.3.15"
+    "angular-bootstrap-datetimepicker": "0.3.15",
+    "angular-nvd3": "1.0.9"
   },
   "devDependencies": {
     "less.js": "~1.4.0",
@@ -43,7 +43,7 @@
   },
   "resolutions": {
     "angular": "1.2.29",
-    "nvd3": "1.1.15-beta",
+    "nvd3": "1.8.4",
     "d3": "3.4.4",
     "moment": "2.4.0",
     "angular-resource": "1.2.15",

http://git-wip-us.apache.org/repos/asf/kylin/blob/f7d265b6/webapp/grunt.json
----------------------------------------------------------------------
diff --git a/webapp/grunt.json b/webapp/grunt.json
index d0ee545..06720b2 100755
--- a/webapp/grunt.json
+++ b/webapp/grunt.json
@@ -37,8 +37,7 @@
         "app/components/angular-chosen-localytics/chosen.js",
         "app/components/moment/min/moment.min.js",
         "app/components/d3/d3.min.js",
-        "app/components/nvd3/nv.d3.min.js",
-        "app/components/angularjs-nvd3-directives/dist/angularjs-nvd3-directives.js",
+        "app/components/nvd3/build/nv.d3.min.js",
         "app/components/bootstrap-sweetalert/lib/sweet-alert.js",
         "app/components/angular-sweetalert/SweetAlert.js",
         "app/components/underscore/underscore.js",
@@ -48,6 +47,7 @@
         "app/components/angular-toggle-switch/angular-toggle-switch.js",
         "app/components/angular-ui-select/dist/select.js",
         "app/components/angular-sanitize/angular-sanitize.js",
+        "app/components/angular-nvd3/dist/angular-nvd3.js",
         "tmp/js/scripts.js"
       ],
       "dest": "tmp/js/scripts.min.js"
@@ -65,7 +65,7 @@
         "app/css/messenger-theme-ice.css",
         "app/components/chosen/chosen.css",
         "app/components/angular-chosen-localytics/chosen-spinner.css",
-        "app/components/nvd3/nv.d3.min.css",
+        "app/components/nvd3/build/nv.d3.min.css",
         "app/css/AdminLTE.css",
         "app/css/skins/_all-skins.min.css",
         "app/components/bootstrap-sweetalert/lib/sweet-alert.css",


[11/18] kylin git commit: APACHE-KYLIN-3018: Add getLongestDepth() in CuboidUtil for getting a reasonable maxLevel for layered cubing

Posted by li...@apache.org.
APACHE-KYLIN-3018: Add getLongestDepth() in CuboidUtil for getting a reasonable maxLevel for layered cubing

Signed-off-by: Zhong <nj...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/1b6d8fe9
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/1b6d8fe9
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/1b6d8fe9

Branch: refs/heads/ci-dong
Commit: 1b6d8fe91f8cfc25b8684ab66007358ef8c535f9
Parents: 5ecc48a
Author: Wang Ken <mi...@ebay.com>
Authored: Tue Nov 7 19:43:00 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../apache/kylin/cube/cuboid/CuboidUtil.java    |  37 +++++++
 .../cube/cuboid/TreeCuboidSchedulerManager.java | 102 -------------------
 .../kylin/cube/cuboid/CuboidUtilTest.java       |  57 +++++++++++
 .../kylin/engine/mr/BatchCubingJobBuilder.java  |   3 +-
 .../kylin/engine/mr/BatchCubingJobBuilder2.java |   4 +-
 .../engine/mr/BatchOptimizeJobBuilder2.java     |   7 +-
 .../engine/mr/common/CuboidSchedulerUtil.java   |  26 ++---
 .../kylin/engine/mr/steps/NDCuboidMapper.java   |   1 +
 .../kylin/engine/spark/SparkCubingByLayer.java  |   3 +-
 9 files changed, 119 insertions(+), 121 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6d8fe9/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidUtil.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidUtil.java b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidUtil.java
index a84f153..e5404c8 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidUtil.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidUtil.java
@@ -18,7 +18,17 @@
 
 package org.apache.kylin.cube.cuboid;
 
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kylin.cube.cuboid.algorithm.CuboidStatsUtil;
+
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 public class CuboidUtil {
 
@@ -45,4 +55,31 @@ public class CuboidUtil {
         }
         return allCuboidsBitSet;
     }
+
+    public static int getLongestDepth(Set<Long> cuboidSet) {
+        Map<Long, List<Long>> directChildrenCache = CuboidStatsUtil.createDirectChildrenCache(cuboidSet);
+        List<Long> cuboids = Lists.newArrayList(cuboidSet);
+        Collections.sort(cuboids, new Comparator<Long>() {
+            @Override
+            public int compare(Long o1, Long o2) {
+                return -Long.compare(o1, o2);
+            }
+        });
+
+        int longestDepth = 0;
+        Map<Long, Integer> cuboidDepthMap = Maps.newHashMap();
+        for (Long cuboid : cuboids) {
+            int parentDepth = cuboidDepthMap.get(cuboid) == null ? 0 : cuboidDepthMap.get(cuboid);
+            for (Long childCuboid : directChildrenCache.get(cuboid)) {
+                if (cuboidDepthMap.get(childCuboid) == null || cuboidDepthMap.get(childCuboid) < parentDepth + 1) {
+                    cuboidDepthMap.put(childCuboid, parentDepth + 1);
+                    if (longestDepth < parentDepth + 1) {
+                        longestDepth = parentDepth + 1;
+                    }
+                }
+            }
+        }
+
+        return longestDepth;
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6d8fe9/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java
deleted file mode 100644
index 22e636b..0000000
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.cube.cuboid;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.metadata.cachesync.Broadcaster;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-public class TreeCuboidSchedulerManager {
-    private static ConcurrentMap<String, TreeCuboidScheduler> cache = Maps.newConcurrentMap();
-
-    private class TreeCuboidSchedulerSyncListener extends Broadcaster.Listener {
-        @Override
-        public void onClearAll(Broadcaster broadcaster) throws IOException {
-            cache.clear();
-        }
-
-        @Override
-        public void onEntityChange(Broadcaster broadcaster, String entity, Broadcaster.Event event, String cacheKey)
-                throws IOException {
-            cache.remove(cacheKey);
-        }
-    }
-
-    public TreeCuboidSchedulerManager() {
-        Broadcaster.getInstance(KylinConfig.getInstanceFromEnv())
-                .registerListener(new TreeCuboidSchedulerSyncListener(), "cube");
-    }
-
-    private static TreeCuboidSchedulerManager instance = new TreeCuboidSchedulerManager();
-
-    public static TreeCuboidSchedulerManager getInstance() {
-        return instance;
-    }
-
-    /**
-     *
-     * @param cubeName
-     * @return null if the cube has no pre-built cuboids
-     */
-    public TreeCuboidScheduler getTreeCuboidScheduler(String cubeName) {
-        TreeCuboidScheduler result = cache.get(cubeName);
-        if (result == null) {
-            CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
-            CubeInstance cubeInstance = cubeManager.getCube(cubeName);
-            if (cubeInstance == null) {
-                return null;
-            }
-            TreeCuboidScheduler treeCuboidScheduler = getTreeCuboidScheduler(cubeInstance.getDescriptor(),
-                    cubeManager.getCube(cubeName).getCuboids());
-            if (treeCuboidScheduler == null) {
-                return null;
-            }
-            cache.put(cubeName, treeCuboidScheduler);
-            result = treeCuboidScheduler;
-        }
-        return result;
-    }
-
-    public TreeCuboidScheduler getTreeCuboidScheduler(CubeDesc cubeDesc, Map<Long, Long> cuboidsWithRowCnt) {
-        if (cuboidsWithRowCnt == null || cuboidsWithRowCnt.isEmpty()) {
-            return null;
-        }
-        return getTreeCuboidScheduler(cubeDesc, Lists.newArrayList(cuboidsWithRowCnt.keySet()), cuboidsWithRowCnt);
-    }
-
-    public TreeCuboidScheduler getTreeCuboidScheduler(CubeDesc cubeDesc, List<Long> cuboidIds,
-            Map<Long, Long> cuboidsWithRowCnt) {
-        if (cuboidIds == null || cuboidsWithRowCnt == null) {
-            return null;
-        }
-        TreeCuboidScheduler treeCuboidScheduler = new TreeCuboidScheduler(cubeDesc, cuboidIds,
-                new TreeCuboidScheduler.CuboidCostComparator(cuboidsWithRowCnt));
-        return treeCuboidScheduler;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6d8fe9/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidUtilTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidUtilTest.java b/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidUtilTest.java
new file mode 100644
index 0000000..18a9312
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidUtilTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.cube.cuboid;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Test;
+
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Sets;
+
+public class CuboidUtilTest {
+    @Test
+    public void testGetLongestDepth() {
+        Stopwatch sw = new Stopwatch();
+
+        Set<Long> cuboidSet1 = Sets.newHashSet(7L, 6L, 5L, 4L, 3L, 2L, 1L);
+        sw.start();
+        assertEquals(2, CuboidUtil.getLongestDepth(cuboidSet1));
+        System.out.println("Time cost for GetLongestDepth: " + sw.elapsed(TimeUnit.MILLISECONDS) + "ms");
+
+        Set<Long> cuboidSet2 = Sets.newHashSet(1024L, 1666L, 1667L, 1728L, 1730L, 1731L, 1760L, 1762L, 1763L, 1776L,
+                1778L, 1779L, 1784L, 1788L, 1790L, 1791L, 1920L, 1922L, 1923L, 1984L, 1986L, 1987L, 2016L, 2018L, 2019L,
+                2032L, 2034L, 2035L, 2040L, 2044L, 2046L, 2047L);
+        sw.reset();
+        sw.start();
+        assertEquals(8, CuboidUtil.getLongestDepth(cuboidSet2));
+        System.out.println("Time cost for GetLongestDepth: " + sw.elapsed(TimeUnit.MILLISECONDS) + "ms");
+
+        Set<Long> cuboidSet3 = Sets.newHashSet(31L, 11L, 5L, 3L, 1L);
+        sw.reset();
+        sw.start();
+        assertEquals(3, CuboidUtil.getLongestDepth(cuboidSet3));
+        System.out.println("Time cost for GetLongestDepth: " + sw.elapsed(TimeUnit.MILLISECONDS) + "ms");
+
+        sw.stop();
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6d8fe9/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
index f64365a..432e1ab 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
@@ -19,6 +19,7 @@
 package org.apache.kylin.engine.mr;
 
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidUtil;
 import org.apache.kylin.cube.model.RowKeyDesc;
 import org.apache.kylin.engine.mr.IMRInput.IMRBatchCubingInputSide;
 import org.apache.kylin.engine.mr.IMROutput.IMRBatchCubingOutputSide;
@@ -64,7 +65,7 @@ public class BatchCubingJobBuilder extends JobBuilderSupport {
 
         // Phase 3: Build Cube
         RowKeyDesc rowKeyDesc = seg.getCubeDesc().getRowkey();
-        final int groupRowkeyColumnsCount = seg.getCuboidScheduler().getBuildLevel();
+        final int groupRowkeyColumnsCount = CuboidUtil.getLongestDepth(seg.getCuboidScheduler().getAllCuboidIds());
         // base cuboid step
         result.addTask(createBaseCuboidStep(getCuboidOutputPathsByLevel(cuboidRootPath, 0), jobId));
         // n dim cuboid steps

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6d8fe9/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
index 4b808d1..8fbc0c9 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
@@ -20,6 +20,7 @@ package org.apache.kylin.engine.mr;
 
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidUtil;
 import org.apache.kylin.engine.mr.IMRInput.IMRBatchCubingInputSide;
 import org.apache.kylin.engine.mr.IMROutput2.IMRBatchCubingOutputSide2;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
@@ -98,7 +99,8 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
     }
 
     protected void addLayerCubingSteps(final CubingJob result, final String jobId, final String cuboidRootPath) {
-        final int maxLevel = seg.getCuboidScheduler().getBuildLevel();
+        // Don't know statistics so that tree cuboid scheduler is not determined. Determine the maxLevel at runtime
+        final int maxLevel = CuboidUtil.getLongestDepth(seg.getCuboidScheduler().getAllCuboidIds());
         // base cuboid step
         result.addTask(createBaseCuboidStep(getCuboidOutputPathsByLevel(cuboidRootPath, 0), jobId));
         // n dim cuboid steps

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6d8fe9/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobBuilder2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobBuilder2.java
index a8127cc..af92368 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobBuilder2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchOptimizeJobBuilder2.java
@@ -20,6 +20,7 @@ package org.apache.kylin.engine.mr;
 
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.CuboidModeEnum;
+import org.apache.kylin.cube.cuboid.CuboidUtil;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.common.BatchConstants;
 import org.apache.kylin.engine.mr.common.MapReduceExecutable;
@@ -151,14 +152,14 @@ public class BatchOptimizeJobBuilder2 extends JobBuilderSupport {
     private void addLayerCubingSteps(final CubingJob result, final String jobId, final CuboidModeEnum cuboidMode,
             final String cuboidRootPath) {
         // Don't know statistics so that tree cuboid scheduler is not determined. Determine the maxLevel at runtime
-        final int maxLevel = seg.getCubeDesc().getRowkey().getRowKeyColumns().length;
+        final int maxLevel = CuboidUtil.getLongestDepth(seg.getCubeInstance().getCuboidsByMode(cuboidMode));
         // Don't need to build base cuboid
         // n dim cuboid steps
         for (int i = 1; i <= maxLevel; i++) {
             String parentCuboidPath = i == 1 ? getBaseCuboidPath(cuboidRootPath)
                     : getCuboidOutputPathsByLevel(cuboidRootPath, i - 1);
-            result.addTask(createNDimensionCuboidStep(parentCuboidPath,
-                    getCuboidOutputPathsByLevel(cuboidRootPath, i), i, jobId, cuboidMode));
+            result.addTask(createNDimensionCuboidStep(parentCuboidPath, getCuboidOutputPathsByLevel(cuboidRootPath, i),
+                    i, jobId, cuboidMode));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6d8fe9/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidSchedulerUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidSchedulerUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidSchedulerUtil.java
index d684c04..1809ff0 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidSchedulerUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidSchedulerUtil.java
@@ -19,36 +19,36 @@
 package org.apache.kylin.engine.mr.common;
 
 import java.io.IOException;
+import java.util.Comparator;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.cuboid.CuboidModeEnum;
 import org.apache.kylin.cube.cuboid.CuboidScheduler;
-import org.apache.kylin.cube.cuboid.DefaultCuboidScheduler;
-import org.apache.kylin.cube.cuboid.TreeCuboidSchedulerManager;
+import org.apache.kylin.cube.cuboid.TreeCuboidScheduler;
+
+import com.google.common.collect.Lists;
 
 public class CuboidSchedulerUtil {
 
     public static CuboidScheduler getCuboidSchedulerByMode(CubeSegment segment, String cuboidModeName) {
-        return getCuboidSchedulerByMode(segment, segment.getCubeInstance().getCuboidsByMode(cuboidModeName));
+        return getCuboidSchedulerByMode(segment, CuboidModeEnum.getByModeName(cuboidModeName));
     }
 
     public static CuboidScheduler getCuboidSchedulerByMode(CubeSegment segment, CuboidModeEnum cuboidMode) {
-        return getCuboidSchedulerByMode(segment, segment.getCubeInstance().getCuboidsByMode(cuboidMode));
+        return getCuboidScheduler(segment, segment.getCubeInstance().getCuboidsByMode(cuboidMode));
     }
 
-    public static CuboidScheduler getCuboidSchedulerByMode(CubeSegment segment, Set<Long> cuboidSet) {
-        CuboidScheduler cuboidScheduler;
+    public static CuboidScheduler getCuboidScheduler(CubeSegment segment, Set<Long> cuboidSet) {
         try {
-            cuboidScheduler = TreeCuboidSchedulerManager.getInstance().getTreeCuboidScheduler(segment.getCubeDesc(), //
-                    CuboidStatsReaderUtil.readCuboidStatsFromSegment(cuboidSet, segment));
+            Map<Long, Long> cuboidsWithRowCnt = CuboidStatsReaderUtil.readCuboidStatsFromSegment(cuboidSet, segment);
+            Comparator<Long> comparator = cuboidsWithRowCnt == null ? Cuboid.cuboidSelectComparator
+                    : new TreeCuboidScheduler.CuboidCostComparator(cuboidsWithRowCnt);
+            return new TreeCuboidScheduler(segment.getCubeDesc(), Lists.newArrayList(cuboidSet), comparator);
         } catch (IOException e) {
             throw new RuntimeException("Fail to cube stats for segment" + segment + " due to " + e);
         }
-
-        if (cuboidScheduler == null) {
-            cuboidScheduler = new DefaultCuboidScheduler(segment.getCubeDesc());
-        }
-        return cuboidScheduler;
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6d8fe9/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
index f936393..a58415a 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
@@ -103,6 +103,7 @@ public class NDCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
 
         if (handleCounter++ % BatchConstants.NORMAL_RECORD_LOG_THRESHOLD == 0) {
             logger.info("Handling record with ordinal: " + handleCounter);
+            logger.info("Parent cuboid: " + parentCuboid.getId() + "; Children: " + myChildren);
         }
 
         for (Long child : myChildren) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/1b6d8fe9/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
index 8d75070..f7c5fee 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
@@ -45,6 +45,7 @@ import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.common.RowKeySplitter;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.cuboid.CuboidUtil;
 import org.apache.kylin.cube.kv.AbstractRowKeyEncoder;
 import org.apache.kylin.cube.kv.RowKeyEncoderProvider;
 import org.apache.kylin.cube.model.CubeDesc;
@@ -181,7 +182,7 @@ public class SparkCubingByLayer extends AbstractApplication implements Serializa
             reducerFunction2 = new CuboidReducerFunction2(cubeName, metaUrl, needAggr);
         }
 
-        final int totalLevels = cubeSegment.getCuboidScheduler().getBuildLevel();
+        final int totalLevels = CuboidUtil.getLongestDepth(cubeSegment.getCuboidScheduler().getAllCuboidIds());
         JavaPairRDD<ByteArray, Object[]>[] allRDDs = new JavaPairRDD[totalLevels + 1];
         int level = 0;
         int partition = estimateRDDPartitionNum(level, cubeStatsReader, envConfig);


[03/18] kylin git commit: APACHE-KYLIN-2726 Introduce a dashboard for showing kylin service related metrics, like query count, query latency, job count, etc

Posted by li...@apache.org.
APACHE-KYLIN-2726 Introduce a dashboard for showing kylin service related metrics, like query count, query latency, job count, etc


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/d608fc10
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/d608fc10
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/d608fc10

Branch: refs/heads/ci-dong
Commit: d608fc10822c82e56fc5bd464c975716d4edd5f9
Parents: f7d265b
Author: liapan <li...@ebay.com>
Authored: Fri Nov 17 14:56:12 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../rest/controller/DashboardController.java    | 129 +++++++
 .../apache/kylin/rest/service/BasicService.java |   5 +
 .../kylin/rest/service/DashboardService.java    | 333 +++++++++++++++++++
 .../apache/kylin/rest/service/QueryService.java |   8 +
 webapp/app/index.html                           |   8 +-
 webapp/app/js/controllers/dashboard.js          | 296 +++++++++++++++++
 webapp/app/js/controllers/job.js                |   3 +
 webapp/app/js/controllers/query.js              |   3 +
 webapp/app/js/directives/directives.js          |  97 ++++++
 webapp/app/js/filters/filter.js                 |   9 +
 webapp/app/js/model/dashboardConfig.js          |  96 ++++++
 webapp/app/js/services/dashboard.js             |  95 ++++++
 webapp/app/less/app.less                        |  58 ++++
 webapp/app/partials/dashboard/dashboard.html    | 151 +++++++++
 webapp/app/partials/header.html                 |   3 +
 webapp/app/routes.json                          |   8 +
 webapp/bower.json                               |   8 +-
 webapp/grunt.json                               |   3 +
 18 files changed, 1309 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/server-base/src/main/java/org/apache/kylin/rest/controller/DashboardController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/DashboardController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/DashboardController.java
new file mode 100644
index 0000000..35ba615
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/DashboardController.java
@@ -0,0 +1,129 @@
+/*
+ * 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.rest.controller;
+
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.metadata.project.ProjectInstance;
+import org.apache.kylin.metrics.MetricsManager;
+import org.apache.kylin.rest.request.SQLRequest;
+import org.apache.kylin.rest.response.MetricsResponse;
+import org.apache.kylin.rest.response.SQLResponse;
+import org.apache.kylin.rest.service.CubeService;
+import org.apache.kylin.rest.service.DashboardService;
+import org.apache.kylin.rest.service.QueryService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.access.AccessDeniedException;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.RequestParam;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+import java.util.List;
+
+@Controller
+@RequestMapping(value = "/dashboard")
+public class DashboardController extends BasicController {
+    private static final Logger logger = LoggerFactory.getLogger(DashboardController.class);
+
+    @Autowired
+    private DashboardService dashboardService;
+
+    @Autowired
+    private QueryService queryService;
+
+    @Autowired
+    private CubeService cubeService;
+
+    @RequestMapping(value = "/metric/cube", method = { RequestMethod.GET })
+    @ResponseBody
+    public MetricsResponse getCubeMetrics(@RequestParam(value = "projectName", required = false) String projectName, @RequestParam(value = "cubeName", required = false) String cubeName) {
+        checkAuthorization(projectName);
+        return dashboardService.getCubeMetrics(projectName, cubeName);
+    }
+
+    @RequestMapping(value = "/metric/query", method = RequestMethod.GET)
+    @ResponseBody
+    public MetricsResponse getQueryMetrics(@RequestParam(value = "projectName", required = false) String projectName, @RequestParam(value = "cubeName", required = false) String cubeName, @RequestParam(value = "startTime") String startTime, @RequestParam(value = "endTime") String endTime) {
+        checkAuthorization(projectName);
+        MetricsResponse queryMetrics = new MetricsResponse();
+        SQLRequest sqlRequest = new SQLRequest();
+        sqlRequest.setProject(MetricsManager.SYSTEM_PROJECT);
+        String sql = dashboardService.getQueryMetricsSQL(startTime, endTime, projectName, cubeName);
+        sqlRequest.setSql(sql);
+        SQLResponse sqlResponse = queryService.queryWithoutSecure(sqlRequest);
+        if(!sqlResponse.getIsException()){
+            queryMetrics.increase("queryCount", dashboardService.getMetricValue(sqlResponse.getResults().get(0).get(0)));
+            queryMetrics.increase("avgQueryLatency", dashboardService.getMetricValue(sqlResponse.getResults().get(0).get(1)));
+            queryMetrics.increase("maxQueryLatency", dashboardService.getMetricValue(sqlResponse.getResults().get(0).get(2)));
+            queryMetrics.increase("minQueryLatency", dashboardService.getMetricValue(sqlResponse.getResults().get(0).get(3)));
+        }
+        return queryMetrics;
+    }
+
+    @RequestMapping(value = "/metric/job", method = RequestMethod.GET)
+    @ResponseBody
+    public MetricsResponse getJobMetrics(@RequestParam(value = "projectName", required = false) String projectName, @RequestParam(value = "cubeName", required = false) String cubeName, @RequestParam(value = "startTime") String startTime, @RequestParam(value = "endTime") String endTime) {
+        checkAuthorization(projectName);
+        MetricsResponse jobMetrics = new MetricsResponse();
+        SQLRequest sqlRequest = new SQLRequest();
+        sqlRequest.setProject(MetricsManager.SYSTEM_PROJECT);
+        String sql = dashboardService.getJobMetricsSQL(startTime, endTime, projectName, cubeName);
+        sqlRequest.setSql(sql);
+        SQLResponse sqlResponse = queryService.queryWithoutSecure(sqlRequest);
+        if(!sqlResponse.getIsException()){
+            jobMetrics.increase("jobCount", dashboardService.getMetricValue(sqlResponse.getResults().get(0).get(0)));
+            jobMetrics.increase("avgJobBuildTime", dashboardService.getMetricValue(sqlResponse.getResults().get(0).get(1)));
+            jobMetrics.increase("maxJobBuildTime", dashboardService.getMetricValue(sqlResponse.getResults().get(0).get(2)));
+            jobMetrics.increase("minJobBuildTime", dashboardService.getMetricValue(sqlResponse.getResults().get(0).get(3)));
+        }
+        return jobMetrics;
+    }
+
+    @RequestMapping(value = "/chart/{category}/{metric}/{dimension}", method = RequestMethod.GET)
+    @ResponseBody
+    public MetricsResponse getChartData(@PathVariable String dimension, @PathVariable String metric, @PathVariable String category, @RequestParam(value = "projectName", required = false) String projectName, @RequestParam(value = "cubeName", required = false) String cubeName, @RequestParam(value = "startTime") String startTime, @RequestParam(value = "endTime") String endTime) {
+        checkAuthorization(projectName);
+        SQLRequest sqlRequest = new SQLRequest();
+        sqlRequest.setProject(MetricsManager.SYSTEM_PROJECT);
+        String sql = dashboardService.getChartSQL(startTime, endTime, projectName, cubeName, dimension, metric, category);
+        sqlRequest.setSql(sql);
+        return dashboardService.transformChartData(queryService.queryWithoutSecure(sqlRequest));
+    }
+
+    private void checkAuthorization(String projectName){
+        if (projectName!=null && !projectName.isEmpty()) {
+            ProjectInstance project = dashboardService.getProjectManager().getProject(projectName);
+            try {
+                dashboardService.checkAuthorization(project);
+            } catch (AccessDeniedException e) {
+                List<CubeInstance> cubes = cubeService.listAllCubes(null, projectName, null, true);
+                if (cubes.isEmpty()) {
+                    throw new AccessDeniedException("Access is denied");
+                }
+            }
+        } else {
+            dashboardService.checkAuthorization();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java b/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java
index 0e9ee7a..f8f50f3 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/BasicService.java
@@ -31,6 +31,7 @@ import org.apache.kylin.metadata.draft.DraftManager;
 import org.apache.kylin.metadata.model.DataModelManager;
 import org.apache.kylin.metadata.project.ProjectManager;
 import org.apache.kylin.metadata.streaming.StreamingManager;
+import org.apache.kylin.metrics.MetricsManager;
 import org.apache.kylin.source.kafka.KafkaConfigManager;
 import org.apache.kylin.storage.hybrid.HybridManager;
 
@@ -94,4 +95,8 @@ public abstract class BasicService {
         return TableACLManager.getInstance(getConfig());
     }
 
+    public MetricsManager getMetricsManager() {
+        return MetricsManager.getInstance();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/server-base/src/main/java/org/apache/kylin/rest/service/DashboardService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/DashboardService.java b/server-base/src/main/java/org/apache/kylin/rest/service/DashboardService.java
new file mode 100644
index 0000000..f1084f3
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/DashboardService.java
@@ -0,0 +1,333 @@
+/*
+ * 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.rest.service;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.metadata.project.ProjectInstance;
+import org.apache.kylin.metadata.project.RealizationEntry;
+import org.apache.kylin.metadata.realization.RealizationType;
+import org.apache.kylin.metrics.MetricsManager;
+import org.apache.kylin.metrics.lib.impl.TimePropertyEnum;
+import org.apache.kylin.metrics.property.JobPropertyEnum;
+import org.apache.kylin.metrics.property.QueryPropertyEnum;
+import org.apache.kylin.rest.constant.Constant;
+import org.apache.kylin.rest.exception.BadRequestException;
+import org.apache.kylin.rest.response.MetricsResponse;
+import org.apache.kylin.rest.response.SQLResponse;
+import org.apache.kylin.storage.hybrid.HybridInstance;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.access.AccessDeniedException;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.stereotype.Component;
+
+import com.google.common.base.Strings;
+
+@Component("dashboardService")
+public class DashboardService extends BasicService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DashboardService.class);
+
+    @Autowired
+    private CubeService cubeService;
+
+    private enum CategoryEnum {QUERY, JOB}
+
+    private enum QueryDimensionEnum {
+        PROJECT(QueryPropertyEnum.PROJECT.toString()),
+        CUBE(QueryPropertyEnum.REALIZATION.toString()),
+        DAY(TimePropertyEnum.DAY_DATE.toString()),
+        WEEK(TimePropertyEnum.WEEK_BEGIN_DATE.toString()),
+        MONTH(TimePropertyEnum.MONTH.toString());
+        private final String sql;
+
+        QueryDimensionEnum(String sql) {
+            this.sql = sql;
+        }
+
+        public String toSQL() {
+            return this.sql;
+        }
+    };
+
+    private enum JobDimensionEnum {
+        PROJECT(JobPropertyEnum.PROJECT.toString()),
+        CUBE(JobPropertyEnum.CUBE.toString()),
+        DAY(TimePropertyEnum.DAY_DATE.toString()),
+        WEEK(TimePropertyEnum.WEEK_BEGIN_DATE.toString()),
+        MONTH(TimePropertyEnum.MONTH.toString());
+        private final String sql;
+
+        JobDimensionEnum(String sql) {
+            this.sql = sql;
+        }
+
+        public String toSQL() {
+            return this.sql;
+        }
+    };
+
+    private enum QueryMetricEnum {
+        QUERY_COUNT("count(*)"),
+        AVG_QUERY_LATENCY("sum(" + QueryPropertyEnum.TIME_COST.toString() + ")/(count(" + QueryPropertyEnum.TIME_COST.toString() + "))"),
+        MAX_QUERY_LATENCY("max(" + QueryPropertyEnum.TIME_COST.toString() + ")"),
+        MIN_QUERY_LATENCY("min(" + QueryPropertyEnum.TIME_COST.toString() + ")");
+
+        private final String sql;
+
+        QueryMetricEnum(String sql) {
+            this.sql = sql;
+        }
+
+        public String toSQL() {
+            return this.sql;
+        }
+    }
+
+    private enum JobMetricEnum {
+        JOB_COUNT("count(*)"),
+        AVG_JOB_BUILD_TIME("sum(" + JobPropertyEnum.PER_BYTES_TIME_COST.toString() + ")/count(" + JobPropertyEnum.PER_BYTES_TIME_COST + ")"),
+        MAX_JOB_BUILD_TIME("max(" + JobPropertyEnum.PER_BYTES_TIME_COST.toString() + ")"),
+        MIN_JOB_BUILD_TIME("min(" + JobPropertyEnum.PER_BYTES_TIME_COST.toString() + ")");
+
+        private final String sql;
+
+        JobMetricEnum(String sql) {
+            this.sql = sql;
+        }
+
+        public String toSQL() {
+            return this.sql;
+        }
+    }
+
+    public MetricsResponse getCubeMetrics(String projectName, String cubeName) {
+        MetricsResponse cubeMetrics = new MetricsResponse();
+        Float totalCubeSize = 0f;
+        long totalRecoadSize = 0;
+        List<CubeInstance> cubeInstances = cubeService.listAllCubes(cubeName, projectName, null, true);
+        Integer totalCube = cubeInstances.size();
+        if (projectName == null) {
+            totalCube += getHybridManager().listHybridInstances().size();
+        } else {
+            ProjectInstance project = getProjectManager().getProject(projectName);
+            totalCube +=  project.getRealizationCount(RealizationType.HYBRID);
+        }
+        Float minCubeExpansion = Float.POSITIVE_INFINITY;
+        Float maxCubeExpansion = Float.NEGATIVE_INFINITY;
+        cubeMetrics.increase("totalCube", totalCube.floatValue());
+        for (CubeInstance cubeInstance : cubeInstances) {
+            if (cubeInstance.getInputRecordSize() > 0) {
+                totalCubeSize += cubeInstance.getSizeKB();
+                totalRecoadSize += cubeInstance.getInputRecordSize();
+                Float cubeExpansion = new Float(cubeInstance.getSizeKB()) * 1024 / cubeInstance.getInputRecordSize();
+                if (cubeExpansion > maxCubeExpansion) {
+                    maxCubeExpansion = cubeExpansion;
+                }
+                if (cubeExpansion < minCubeExpansion) {
+                    minCubeExpansion = cubeExpansion;
+                }
+            }
+        }
+        Float avgCubeExpansion = 0f;
+        if (totalRecoadSize != 0) {
+            avgCubeExpansion = totalCubeSize * 1024 / totalRecoadSize;
+        }
+        cubeMetrics.increase("avgCubeExpansion", avgCubeExpansion);
+        cubeMetrics.increase("maxCubeExpansion", maxCubeExpansion == Float.NEGATIVE_INFINITY ? 0 : maxCubeExpansion);
+        cubeMetrics.increase("minCubeExpansion", minCubeExpansion == Float.POSITIVE_INFINITY ? 0 : minCubeExpansion);
+        return cubeMetrics;
+    }
+
+    private List<CubeInstance> getCubeByHybrid(HybridInstance hybridInstance) {
+        List<CubeInstance> cubeInstances = Lists.newArrayList();
+        List<RealizationEntry> realizationEntries = hybridInstance.getRealizationEntries();
+        for (RealizationEntry realizationEntry : realizationEntries) {
+            String reName = realizationEntry.getRealization();
+            if (RealizationType.CUBE == realizationEntry.getType()) {
+                CubeInstance cubeInstance = getCubeManager().getCube(reName);
+                cubeInstances.add(cubeInstance);
+            } else if (RealizationType.HYBRID == realizationEntry.getType()) {
+                HybridInstance innerHybridInstance = getHybridManager().getHybridInstance(reName);
+                cubeInstances.addAll(getCubeByHybrid(innerHybridInstance));
+            }
+        }
+        return cubeInstances;
+    }
+
+    public String getQueryMetricsSQL(String startTime, String endTime, String projectName, String cubeName) {
+        String[] metrics = new String[] {QueryMetricEnum.QUERY_COUNT.toSQL(), QueryMetricEnum.AVG_QUERY_LATENCY.toSQL(), QueryMetricEnum.MAX_QUERY_LATENCY.toSQL(), QueryMetricEnum.MIN_QUERY_LATENCY.toSQL()};
+        List<String> filters = getBaseFilters(CategoryEnum.QUERY, projectName, startTime, endTime);
+        filters = addCubeFilter(filters, CategoryEnum.QUERY, cubeName);
+        return createSql(null, metrics, getMetricsManager().getSystemTableFromSubject(getConfig().getKylinMetricsSubjectQuery()), filters.toArray(new String[filters.size()]));
+    }
+
+    public String getJobMetricsSQL(String startTime, String endTime, String projectName, String cubeName) {
+        String[] metrics = new String[] {JobMetricEnum.JOB_COUNT.toSQL(), JobMetricEnum.AVG_JOB_BUILD_TIME.toSQL(), JobMetricEnum.MAX_JOB_BUILD_TIME.toSQL(), JobMetricEnum.MIN_JOB_BUILD_TIME.toSQL()};
+        List<String> filters = getBaseFilters(CategoryEnum.JOB, projectName, startTime, endTime);
+        filters = addCubeFilter(filters, CategoryEnum.JOB, cubeName);
+        return createSql(null, metrics, getMetricsManager().getSystemTableFromSubject(getConfig().getKylinMetricsSubjectJob()), filters.toArray(new String[filters.size()]));
+    }
+
+    public String getChartSQL(String startTime, String endTime, String projectName, String cubeName, String dimension, String metric, String category) {
+        try{
+            CategoryEnum categoryEnum = CategoryEnum.valueOf(category);
+            String table = "";
+            String[] dimensionSQL = null;
+            String[] metricSQL = null;
+
+            if(categoryEnum == CategoryEnum.QUERY) {
+                dimensionSQL = new String[] {QueryDimensionEnum.valueOf(dimension).toSQL()};
+                metricSQL = new String[] {QueryMetricEnum.valueOf(metric).toSQL()};
+                table = getMetricsManager().getSystemTableFromSubject(getConfig().getKylinMetricsSubjectQuery());
+            } else if (categoryEnum == CategoryEnum.JOB) {
+                dimensionSQL = new String[] {JobDimensionEnum.valueOf(dimension).toSQL()};
+                metricSQL = new String[] {JobMetricEnum.valueOf(metric).toSQL()};
+                table = getMetricsManager().getSystemTableFromSubject(getConfig().getKylinMetricsSubjectJob());
+            }
+
+            List<String> filters = getBaseFilters(categoryEnum, projectName, startTime, endTime);
+            filters = addCubeFilter(filters, categoryEnum, cubeName);
+
+            return createSql(dimensionSQL, metricSQL, table, filters.toArray(new String[filters.size()]));
+        } catch (IllegalArgumentException e) {
+            String message = "Generate dashboard chart sql failed. Please double check the input parameter: dimension, metric or category.";
+            logger.error(message, e);
+            throw new BadRequestException(message + " Caused by: " + e.getMessage(), null, e.getCause());
+        }
+    }
+
+    public MetricsResponse transformChartData(SQLResponse sqlResponse) {
+        if(!sqlResponse.getIsException()){
+            MetricsResponse metrics = new MetricsResponse();
+            List<List<String>> results = sqlResponse.getResults();
+            for (List<String> result : results) {
+                String dimension = result.get(0);
+                if (dimension !=null && !dimension.isEmpty()) {
+                    String metric = result.get(1);
+                    metrics.increase(dimension, getMetricValue(metric));
+                }
+            }
+            return  metrics;
+        }
+        return null;
+    }
+
+    public Float getMetricValue(String value) {
+        if (value == null || value.isEmpty()) {
+            return 0f;
+        } else {
+            return Float.valueOf(value);
+        }
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#project, 'ADMINISTRATION')")
+    public void checkAuthorization(ProjectInstance project) throws AccessDeniedException {
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
+    public void checkAuthorization() throws AccessDeniedException{
+    }
+
+    private List<String> getBaseFilters(CategoryEnum category, String projectName, String startTime, String endTime) {
+        List<String> filters = new ArrayList<String>();
+        String project = "";
+        if (category == CategoryEnum.QUERY) {
+            project = QueryDimensionEnum.PROJECT.toSQL();
+        } else {
+            project = JobDimensionEnum.PROJECT.toSQL();
+        }
+        filters.add(TimePropertyEnum.DAY_DATE.toString() + " >= '" + startTime + "'");
+        filters.add(TimePropertyEnum.DAY_DATE.toString() + " <= '" + endTime + "'");
+        if (!Strings.isNullOrEmpty(projectName)) {
+            filters.add(project + " ='" + ProjectInstance.getNormalizedProjectName(projectName) + "'");
+        } else {
+            filters.add(project + " <> '" + MetricsManager.SYSTEM_PROJECT + "'");
+        }
+        return filters;
+    }
+
+    private List<String> addCubeFilter(List<String> baseFilter, CategoryEnum category, String cubeName) {
+        if (category == CategoryEnum.QUERY) {
+            baseFilter.add(QueryPropertyEnum.EXCEPTION.toString() + " = 'NULL'");
+            if (!Strings.isNullOrEmpty(cubeName)) {
+                baseFilter.add(QueryPropertyEnum.REALIZATION + " = '" + cubeName + "'");
+            }
+        } else if (category == CategoryEnum.JOB && !Strings.isNullOrEmpty(cubeName)) {
+            HybridInstance hybridInstance = getHybridManager().getHybridInstance(cubeName);
+            if (null != hybridInstance) {
+                StringBuffer cubeNames = new StringBuffer();
+                for (CubeInstance cube:getCubeByHybrid(hybridInstance)) {
+                    cubeNames.append(",'" + cube.getName() + "'");
+                }
+                baseFilter.add(JobPropertyEnum.CUBE.toString() + " IN (" + cubeNames.substring(1) + ")");
+            } else {
+                baseFilter.add(JobPropertyEnum.CUBE.toString() + " ='" + cubeName + "'");
+            }
+        }
+        return baseFilter;
+    }
+
+    private String createSql(String[] dimensions, String[] metrics, String category, String[] filters) {
+        StringBuffer baseSQL = new StringBuffer("select ");
+        StringBuffer groupBy = new StringBuffer("");
+        if (dimensions != null && dimensions.length > 0) {
+            groupBy.append(" group by ");
+            StringBuffer dimensionSQL = new StringBuffer("");
+            for (String dimension : dimensions) {
+                dimensionSQL.append(",");
+                dimensionSQL.append(dimension);
+            }
+            baseSQL.append(dimensionSQL.substring(1));
+            groupBy.append(dimensionSQL.substring(1));
+        }
+        if (metrics != null && metrics.length > 0) {
+            StringBuffer metricSQL = new StringBuffer("");
+            for (String metric : metrics) {
+                metricSQL.append(",");
+                metricSQL.append(metric);
+            }
+            if (groupBy.length() > 0) {
+                baseSQL.append(metricSQL);
+            } else {
+                baseSQL.append(metricSQL.substring(1));
+            }
+        }
+        baseSQL.append(" from ");
+        baseSQL.append(category);
+        if (filters != null && filters.length > 0) {
+            StringBuffer filterSQL = new StringBuffer(" where ");
+            filterSQL.append(filters[0]);
+            for(int i = 1; i < filters.length; i++) {
+                filterSQL.append(" and ");
+                filterSQL.append(filters[i]);
+            }
+            baseSQL.append(filterSQL.toString());
+        }
+        baseSQL.append(groupBy);
+
+        return baseSQL.toString();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 26b25d2..3f16646 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -371,7 +371,15 @@ public class QueryService extends BasicService {
         }
     }
 
+    public SQLResponse queryWithoutSecure(SQLRequest sqlRequest) {
+        return doQueryWithCache(sqlRequest, false);
+    }
+
     public SQLResponse doQueryWithCache(SQLRequest sqlRequest) {
+        return doQueryWithCache(sqlRequest, true);
+    }
+
+    public SQLResponse doQueryWithCache(SQLRequest sqlRequest, boolean secureEnabled) {
         Message msg = MsgPicker.getMsg();
         sqlRequest.setUsername(getUserName());
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/app/index.html
----------------------------------------------------------------------
diff --git a/webapp/app/index.html b/webapp/app/index.html
index d235b06..791aa4a 100644
--- a/webapp/app/index.html
+++ b/webapp/app/index.html
@@ -53,6 +53,7 @@
   <link rel="stylesheet" type="text/css" href="components/angular-toggle-switch/angular-toggle-switch.css">
   <link rel="stylesheet" type="text/css" href="components/angular-ui-select/dist/select.css">
   <link rel="stylesheet" type="text/css" href="components/angular-bootstrap-datetimepicker/src/css/datetimepicker.css">
+  <link rel="stylesheet" type="text/css" href="components/bootstrap-daterangepicker/daterangepicker-bs3.css" />
 
   <link rel="stylesheet/less" href="less/build.less">
   <!-- endref -->
@@ -111,7 +112,9 @@
 <script src="components/angular-ui-sortable/sortable.js"></script>
 <script src="components/angular-toggle-switch/angular-toggle-switch.js"></script>
 <script src="components/angular-sanitize/angular-sanitize.js"></script>
-<script src="components/angular-nvd3/dist/angular-nvd3.min.js"></script>
+<script src="components/angular-nvd3/dist/angular-nvd3.js"></script>
+<script src="components/moment-timezone/builds/moment-timezone-with-data.js"></script>
+<script src="components/bootstrap-daterangepicker/daterangepicker.js"></script>
 
 <script src="js/app.js"></script>
 <script src="js/config.js"></script>
@@ -145,6 +148,7 @@
 <script src="js/services/ngLoading.js"></script>
 <!--New GUI-->
 <script src="js/services/models.js"></script>
+<script src="js/services/dashboard.js"></script>
 
 <script src="js/model/cubeConfig.js"></script>
 <script src="js/model/jobConfig.js"></script>
@@ -163,6 +167,7 @@
 <script src="js/model/jobListModel.js"></script>
 <script src="js/model/cubesManager.js"></script>
 <script src="js/model/queryConfig.js"></script>
+<script src="js/model/dashboardConfig.js"></script>
 
 <!--New GUI-->
 <script src="js/model/modelsManager.js"></script>
@@ -202,6 +207,7 @@
 
 <!--New GUI-->
 <script src="js/controllers/models.js"></script>
+<script src="js/controllers/dashboard.js"></script>
 
 <!-- endref -->
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/app/js/controllers/dashboard.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/dashboard.js b/webapp/app/js/controllers/dashboard.js
new file mode 100644
index 0000000..3114919
--- /dev/null
+++ b/webapp/app/js/controllers/dashboard.js
@@ -0,0 +1,296 @@
+/*
+ * 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.
+ */
+
+'use strict';
+
+KylinApp.controller('DashboardCtrl', function ($scope, $location, storage, kylinConfig, dashboardConfig, DashboardService, MessageService, SweetAlert, loadingRequest, UserService, ProjectModel, $filter) {
+
+  $scope.init = function(){
+    $scope.timezone = 'GMT';
+
+    // Init date range
+    storage.bind($scope, 'dateRange', {defaultValue: {
+      startDate: moment().subtract(7, 'days').clone().tz($scope.timezone).startOf('day').format('x'),
+      endDate: moment().subtract(1, 'days').clone().tz($scope.timezone).endOf('day').format('x')
+    }, storeName: 'dashboard.dateRange'});
+
+    storage.bind($scope, 'barchartDimension', {defaultValue: dashboardConfig.dimensions[0], storeName: 'dashboard.barchart.dimension'});
+    storage.bind($scope, 'barchartCategory', {defaultValue: dashboardConfig.categories[0], storeName: 'dashboard.barchart.category'});
+    storage.bind($scope, 'barchartMetric', {defaultValue: dashboardConfig.metrics[0], storeName: 'dashboard.barchart.metric'});
+
+    storage.bind($scope, 'linechartDimension', {defaultValue: dashboardConfig.dimensions[2], storeName: 'dashboard.linechart.dimension'});
+    storage.bind($scope, 'linechartCategory', {defaultValue: dashboardConfig.categories[0], storeName: 'dashboard.linechart.category'});
+    storage.bind($scope, 'linechartMetric', {defaultValue: dashboardConfig.metrics[0], storeName: 'dashboard.linechart.metric'});
+
+    storage.bind($scope, 'currentSquare', 'queryCount');
+    $scope.initReady = true;
+  };
+
+  $scope.formatDatetime = function(dateTime) {
+    return moment(dateTime, 'x').tz($scope.timezone).format('YYYY-MM-DD');
+  };
+
+  $scope.refreshCubeMetric = function() {
+    $scope.cubeMetricsRefreshTime = moment().tz($scope.timezone).format('YYYY-MM-DD');
+    DashboardService.getCubeMetrics({projectName: ProjectModel.getSelectedProject(), cubeName: $scope.selectedCube}, {}, function (data) {
+      $scope.cubeMetrics = data;
+    }, function(e) {
+        SweetAlert.swal('Oops...', 'Failed to load cube metrics', 'error');
+        console.error('cube metrics error', e.data);
+    });
+  };
+
+  // Need to change name for this function
+  $scope.refreshOtherMetrics = function(){
+    DashboardService.getQueryMetrics({projectName: ProjectModel.getSelectedProject(), cubeName: $scope.selectedCube, startTime: $scope.formatDatetime($scope.dateRange.startDate), endTime: $scope.formatDatetime($scope.dateRange.endDate)}, {}, function (data) {
+      $scope.queryMetrics = data;
+    }, function(e) {
+        SweetAlert.swal('Oops...', 'Failed to load query metrics.', 'error');
+        console.error('query metrics error:', e.data);
+    });
+
+    DashboardService.getJobMetrics({projectName: ProjectModel.getSelectedProject(), cubeName: $scope.selectedCube, startTime: $scope.formatDatetime($scope.dateRange.startDate), endTime: $scope.formatDatetime($scope.dateRange.endDate)}, {}, function (data) {
+      $scope.jobMetrics = data;
+    }, function(e) {
+        SweetAlert.swal('Oops...', 'Failed to load job metrics.', 'error');
+        console.error('job metrics error:', e.data);
+    });
+
+    $scope.createCharts();
+  };
+
+  // Daterangepicker call back
+  $scope.changeDateRange = function(start, end) {
+    console.log("start time:", start);
+    console.log("end time:", end);
+    $scope.dateRange.startDate = start;
+    $scope.dateRange.endDate = end;
+    $scope.refreshOtherMetrics();
+  };
+
+  // Create chart option and data
+  $scope.createChart = function(dataQuery, chartType) {
+    var chartObj = {
+      queryObj: dataQuery
+    };
+
+    // get base options
+    var baseOptions = dashboardConfig.baseChartOptions;
+
+    var title = $filter('startCase')(dataQuery.metric.name) + ' by ' + $filter('startCase')(dataQuery.dimension.name);
+
+    // set title to options
+    chartObj.options = angular.copy(baseOptions);
+    chartObj.options.chart.xAxis.axisLabel = dataQuery.dimension.name;
+    chartObj.options.title.text = title;
+
+    var groupByOptions = [];
+    angular.forEach(dashboardConfig.granularityFilter, function(option) {
+      groupByOptions.push(option.value);
+    });
+    if (groupByOptions.indexOf(dataQuery.dimension.name) > -1) {
+      var formatPattern = '%Y-%m-%d';
+      if (dataQuery.dimension.name === dashboardConfig.granularityFilter[2].value) {
+        formatPattern = '%Y-%m';
+      }
+      chartObj.options.chart.xAxis.tickFormat = function (d) {
+        return d3.time.format(formatPattern)(moment.unix(d/1000).toDate());
+      };
+      chartObj.options.chart.tooltip.contentGenerator = function (d) {
+        return '<table><tr><td class="legend-color-guide"><div style="background-color: '+d.point.color+';"></div></td><td class="key">' + d3.time.format(formatPattern)(moment.unix(d.point.label/1000).toDate()) + '</td><td class="value">'+d.point.value.toFixed(2)+'</td></tr></table>';
+      };
+
+      // chartObj.options.chart.interpolate = 'cardinal';
+
+      chartObj.options.chart.legend = {
+        margin: {
+          left: 15
+        }
+      };
+
+      // Add filter for change month
+      chartObj.dimension = {};
+      chartObj.dimension.options = dashboardConfig.granularityFilter;
+      chartObj.dimension.selected = dataQuery.dimension;
+      angular.forEach(chartObj.dimension.options, function(option, ind) {
+        if (dataQuery.dimension.name.indexOf(option.value) > -1) {
+          chartObj.dimension.selected = chartObj.dimension.options[ind];
+        }
+      });
+    }
+
+    chartObj.data = [];
+
+    if (chartType === 'line') {
+      chartObj.options.chart.type = 'lineChart';
+      $scope.lineChart = chartObj;
+      DashboardService.getChartData({category: dataQuery.category, metric: dataQuery.metric.value, dimension: dataQuery.dimension.value, projectName: ProjectModel.getSelectedProject(), cubeName: $scope.selectedCube, startTime: $scope.formatDatetime($scope.dateRange.startDate), endTime: $scope.formatDatetime($scope.dateRange.endDate)}, {}, function (data) {
+        if (data.length > 6) {
+          $scope.lineChart.options.chart.xAxis.rotateLabels = -50;
+          $scope.lineChart.options.chart.xAxis.axisLabel = '';
+        }
+
+        $scope.lineChart.data = [{key: dataQuery.category, values: _.sortBy(data, 'label')}];
+      }, function(e) {
+          SweetAlert.swal('Oops...', 'Failed to load line chart.', 'error');
+          console.error('line chart error:', e.data);
+      });
+    } else  if (chartType === 'bar'){
+      chartObj.options.chart.type = 'discreteBarChart';
+      chartObj.options.chart.discretebar = {
+        dispatch: {
+          elementClick: function(el) {
+            if (ProjectModel.getSelectedProject()) {
+              $scope.selectedCube = el.data.label;
+            } else {
+              var project = el.data.label;
+              ProjectModel.projects.forEach(function(pro) {
+                if (pro.name.toLowerCase() === project.toLowerCase()) {
+                  project = pro.name;
+                }
+              });
+              ProjectModel.setSelectedProject(project);
+            }
+            $scope.$apply();
+          }
+        }
+      };
+      $scope.barChart = chartObj;
+      DashboardService.getChartData({category: dataQuery.category, metric: dataQuery.metric.value, dimension: dataQuery.dimension.value, projectName: ProjectModel.getSelectedProject(), cubeName: $scope.selectedCube, startTime: $scope.formatDatetime($scope.dateRange.startDate), endTime: $scope.formatDatetime($scope.dateRange.endDate)}, {}, function (data) {
+        if (data.length > 6) {
+          $scope.barChart.options.chart.xAxis.rotateLabels = -50;
+          $scope.barChart.options.chart.xAxis.axisLabel = '';
+        }
+        $scope.barChart.data = [{key: dataQuery.category, values: data}];
+        if ($scope.selectedCube) {
+          angular.forEach($scope.barChart.data[0].values, function (value, index){
+            if (value.label != $scope.selectedCube) {
+              value.color = '#ddd';
+            }
+          });
+        } 
+      }, function(e) {
+          SweetAlert.swal('Oops...', 'Failed to load bar chart.', 'error');
+          console.error('bar chart error:', e.data);
+      });
+    }    
+  };
+
+  // Clean and remove chart
+  $scope.removeChart = function(chartType) {
+    if (chartType === 'all') {
+      $scope.barChart = undefined;
+      $scope.lineChart = undefined;
+    } else if (chartType == 'bar') {
+      $scope.barChart = undefined;
+    } else if (chartType == 'line') {
+      $scope.lineChart = undefined;
+    }
+  };
+
+  $scope.createCharts = function() {
+    $scope.createChart({dimension: $scope.barchartDimension, category: $scope.barchartCategory, metric: $scope.barchartMetric}, 'bar');
+    $scope.createChart({dimension: $scope.linechartDimension, category: $scope.linechartCategory, metric: $scope.linechartMetric}, 'line');
+  };
+
+  // Click query count square
+  $scope.queryCountChart = function() {
+    $scope.currentSquare = 'queryCount'; 
+    $scope.barchartCategory = dashboardConfig.categories[0];
+    $scope.barchartMetric = dashboardConfig.metrics[0];
+    $scope.linechartCategory = dashboardConfig.categories[0];
+    $scope.linechartMetric = dashboardConfig.metrics[0];
+
+    $scope.removeChart('all');
+    $scope.createCharts();
+  };
+
+  // Click avg query latency
+  $scope.queryAvgChart = function() {
+    $scope.currentSquare = 'queryAvg';
+    $scope.barchartCategory = dashboardConfig.categories[0];
+    $scope.barchartMetric = dashboardConfig.metrics[1];
+    $scope.linechartCategory = dashboardConfig.categories[0];
+    $scope.linechartMetric = dashboardConfig.metrics[1];
+
+    $scope.removeChart('all');
+    $scope.createCharts();
+  };
+
+  // Click job count
+  $scope.jobCountChart = function() {
+    $scope.currentSquare = 'jobCount';
+    $scope.barchartCategory = dashboardConfig.categories[1];
+    $scope.barchartMetric = dashboardConfig.metrics[2];
+    $scope.linechartCategory = dashboardConfig.categories[1];
+    $scope.linechartMetric = dashboardConfig.metrics[2];
+
+    $scope.removeChart('all');
+    $scope.createCharts();
+  };
+
+  // Click job count
+  $scope.jobBuildTimeChart = function() {
+    $scope.currentSquare = 'jobBuildTime';
+    $scope.barchartCategory = dashboardConfig.categories[1];
+    $scope.barchartMetric = dashboardConfig.metrics[3];
+    $scope.linechartCategory = dashboardConfig.categories[1];
+    $scope.linechartMetric = dashboardConfig.metrics[3];
+
+    $scope.removeChart('all');
+    $scope.createCharts();
+  };
+
+  // Line chart granularity change.
+  $scope.changeDimensionFilter = function(chartType) {
+    if (chartType === 'line') {
+      var dataQuery = $scope.lineChart.queryObj;
+      angular.forEach(dashboardConfig.dimensions, function(dimension, ind) {
+        if (dimension.name === $scope.lineChart.dimension.selected.value) {
+          dataQuery.dimension = dashboardConfig.dimensions[ind];
+          $scope.linechartDimension = dashboardConfig.dimensions[ind];
+        }
+      });
+      $scope.removeChart(chartType);
+      $scope.createChart(dataQuery, chartType);
+    }
+  };
+
+  // watch the project or cube change
+  $scope.$watch('projectModel.selectedProject +"~"+ selectedCube', function (newValues, oldValues) {
+    if ($scope.initReady) {
+      if (ProjectModel.getSelectedProject() != null) {
+        $scope.barchartDimension = dashboardConfig.dimensions[1];
+      } else {
+        $scope.barchartDimension = dashboardConfig.dimensions[0];
+      }
+      if (newValues.split('~')[0] != oldValues.split('~')[0]) {
+        $scope.selectedCube = undefined;
+      }
+      $scope.refreshCubeMetric();
+      $scope.refreshOtherMetrics();
+    }
+  });
+
+  $scope.init();
+
+  $scope.cleanSelectedCube = function() {
+    $scope.selectedCube = undefined;
+  };
+
+});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/app/js/controllers/job.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/job.js b/webapp/app/js/controllers/job.js
index abb3bed..8e460fa 100644
--- a/webapp/app/js/controllers/job.js
+++ b/webapp/app/js/controllers/job.js
@@ -28,6 +28,9 @@ KylinApp
         //$scope.projects = [];
         $scope.action = {};
         $scope.timeFilter = jobConfig.timeFilter[1];
+        if ($routeParams.jobTimeFilter) {
+            $scope.timeFilter = jobConfig.timeFilter[$routeParams.jobTimeFilter];
+        }
 
         $scope.status = [];
         $scope.toggleSelection = function toggleSelection(current) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/app/js/controllers/query.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/query.js b/webapp/app/js/controllers/query.js
index 4014093..90699df 100644
--- a/webapp/app/js/controllers/query.js
+++ b/webapp/app/js/controllers/query.js
@@ -21,6 +21,9 @@
 KylinApp
     .controller('QueryCtrl', function ($scope, storage, $base64, $q, $location, $anchorScroll, $routeParams, QueryService, $modal, MessageService, $domUtilityService, $timeout, TableService, SweetAlert, VdmUtil) {
         $scope.mainPanel = 'query';
+        if ($routeParams.queryPanel) {
+            $scope.mainPanel = $routeParams.queryPanel;
+        }
         $scope.rowsPerPage = 50000;
         $scope.base64 = $base64;
         $scope.queryString = "";

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/app/js/directives/directives.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/directives/directives.js b/webapp/app/js/directives/directives.js
index bca3b03..d6ed304 100644
--- a/webapp/app/js/directives/directives.js
+++ b/webapp/app/js/directives/directives.js
@@ -430,4 +430,101 @@ KylinApp.directive('kylinPagination', function ($parse, $q) {
       });
     }
   }
+}).directive('kylinDaterangepicker', function() {
+  return {
+    restrict: 'E',
+    scope: {
+      startDate:'=',
+      endDate:'=',
+      minDate:'=',
+      maxDate:'=',
+      timezone: '=',
+      ranges: '=',
+      callbackHandler:'&callback',
+    },
+    template: '<button class="btn btn-default" style="background-color:#ffffff">' +
+                '<i class="fa fa-calendar"></i> ' +
+                ' <span></span> ' +
+                '<b class="caret"></b>' +
+              '</button>',
+    link: function(scope, element, attrs) {
+      // Init
+      var timezone = scope.timezone || 'GMT';
+      var format = attrs.format || 'YYYY-MM-DD';
+      var separator = ' - ';
+      var callback = scope.callbackHandler ? scope.callbackHandler() : function() {};
+
+      function startOfToday() {
+        return moment().tz(timezone).startOf('day');
+      }
+      function endOfToday() {
+        return moment().tz(timezone).endOf('day');
+      }
+
+      function getOption() {
+        var ranges = {
+          'Last 7 Days': [
+            startOfToday().subtract(1, 'weeks'),
+            endOfToday().subtract(1, 'days')
+          ],
+          'This Month': [
+            startOfToday().startOf('month'),
+            endOfToday()
+          ],
+          'Last Month': [
+            startOfToday().subtract(1, 'month').startOf('month'),
+            endOfToday().subtract(1, 'month').endOf('month')
+          ]
+        };
+
+        // Create datepicker, full list of options at https://github.com/dangrossman/bootstrap-daterangepicker
+        var maxDate = moment.tz(moment().tz(timezone).format(format), timezone);
+        var minDate = maxDate.clone().subtract(18, 'month');
+        return {
+          maxDate: scope.maxDate || maxDate,
+          minDate: scope.minDate || minDate,
+          format: format,
+          showDropdowns: true,
+          opens: attrs.opens || 'left',
+          ranges: scope.ranges || ranges
+        };
+      }
+
+      function _refresh() {
+        element.daterangepicker(getOption(), function(start, end, label) {
+          scope.startDate = moment.tz(start.startOf('day').format('YYYY-MM-DD HH:mm:ss'), timezone).format('x');
+          scope.endDate = moment.tz(end.endOf('day').format('YYYY-MM-DD HH:mm:ss'), timezone).format('x');
+          callback(scope.startDate, scope.endDate);
+          scope.$apply();
+        });
+      }
+
+      if (timezone) {
+        _refresh();
+      }
+
+      // Use $watch, update the view if either start or end change. (angular version 1.2 not support $watchGroup)
+      scope.$watch('startDate + "~" + endDate + "~" + timezone', function(newValues) {
+        var valueArr = newValues.split('~');
+
+        if (valueArr[2]) {
+          timezone = scope.timezone;
+          _refresh();
+        }
+
+        if (timezone) {
+          var startDate = valueArr[0] ? moment(valueArr[0], 'x').tz(timezone).format(format) : null;
+          var endDate = valueArr[1]  ? moment(valueArr[1], 'x').tz(timezone).format(format) : null;
+        }
+
+        if (startDate && endDate) {
+          var val = startDate + separator + endDate;
+          element.find('span').html(val);
+          element.data('daterangepicker').setStartDate(startDate);
+          element.data('daterangepicker').setEndDate(endDate);
+        }
+      });
+
+    }
+  };
 });

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/app/js/filters/filter.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/filters/filter.js b/webapp/app/js/filters/filter.js
index 8b6cffa..0d7eb5b 100755
--- a/webapp/app/js/filters/filter.js
+++ b/webapp/app/js/filters/filter.js
@@ -242,5 +242,14 @@ KylinApp
       });
       return out;
     }
+  }).filter('startCase', function($filter) {
+    return function (item) {
+      var words = item.split(' ');
+      var formatWord = '';
+      angular.forEach(words, function(word, ind) {
+        formatWord += ' ' + word.charAt(0).toUpperCase() + word.slice(1);
+      })
+      return formatWord.slice(1);
+    };
   });
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/app/js/model/dashboardConfig.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/dashboardConfig.js b/webapp/app/js/model/dashboardConfig.js
new file mode 100644
index 0000000..1620d1d
--- /dev/null
+++ b/webapp/app/js/model/dashboardConfig.js
@@ -0,0 +1,96 @@
+/*
+ * 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.
+ */
+
+KylinApp.constant('dashboardConfig', {
+  granularityFilter: [
+    {name: 'Daily', value: 'day'},
+    {name: 'Weekly', value: 'week'},
+    {name: 'Monthly', value: 'month'}
+  ],
+  metrics: [
+    {name: 'query count', value: 'QUERY_COUNT'},
+    {name: 'avg query latency', value: 'AVG_QUERY_LATENCY'},
+    {name: 'job count', value: 'JOB_COUNT'},
+    {name: 'avg build time', value: 'AVG_JOB_BUILD_TIME'}
+  ],
+  dimensions: [
+    {name: 'project', value: 'PROJECT'},
+    {name: 'cube', value: 'CUBE'},
+    {name: 'day', value: 'DAY'},
+    {name: 'week', value: 'WEEK'},
+    {name: 'month', value: 'MONTH'}
+  ],
+  categories: [
+    'QUERY', 'JOB'
+  ],
+  baseChartOptions: {
+    chart: {
+      height: 272,
+      margin : {
+        top: 20,
+        right: 40,
+        bottom: 60,
+        left: 45
+      },
+      useInteractiveGuideline: false,
+      x: function(d){return d.label;},
+      y: function(d){return d.value;},
+      xAxis: {
+        axisLabelDistance: 50,
+        staggerLabels: false,
+        tickFormat: function(d) {
+          if (d.length > 10) {
+            return d.substring(0,10) + '...';
+          } else {
+            return d;
+          }
+        }
+      },
+      yAxis: {
+        tickFormat: function(d) {
+          if (d < 1000) {
+            if (parseFloat(d) === d) {
+              return d3.format('.1')(d);
+            } else {
+              return d3.format('.2f')(d);
+            }
+          } else {
+            var prefix = d3.formatPrefix(d);
+            return prefix.scale(d) + prefix.symbol;
+          } 
+        },
+        showMaxMin: false
+      },
+      valueFormat: function(d){
+        return d3.format('.1')(d);
+      },
+      transitionDuration: 500,
+      tooltip: {
+        contentGenerator: function (d) {
+          return '<table><tr><td class="legend-color-guide"><div style="background-color: '+d.color+';"></div></td><td class="key">' + d.data.label + '</td><td class="value">'+d.data.value.toFixed(2)+'</td></tr></table>';
+        }
+      }
+    },
+    title: {
+      enable: true,
+      css: {
+        'margin-top': '20px'
+      }
+    }
+  }
+});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/app/js/services/dashboard.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/services/dashboard.js b/webapp/app/js/services/dashboard.js
new file mode 100644
index 0000000..4340119
--- /dev/null
+++ b/webapp/app/js/services/dashboard.js
@@ -0,0 +1,95 @@
+/*
+ * 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.
+ */
+
+KylinApp.factory('DashboardService', ['$resource', '$location', function ($resource, $location, config) {
+  return $resource(Config.service.url + 'dashboard/:type/:category/:metric/:dimension', {}, {
+      getCubeMetrics: {
+        method: 'GET',
+        params: {type: 'metric', category: 'cube'},
+        isArray: false,
+        interceptor: {
+          response: function(response) {
+            var data = response.data;
+            var cubeMetrics;
+            if (data) {
+              cubeMetrics = {totalCubes: data.totalCube, expansionRate: {avg: data.avgCubeExpansion, max: data.maxCubeExpansion, min: data.minCubeExpansion}};
+            }
+            return cubeMetrics;
+          }
+        }
+      },
+      getQueryMetrics: {
+        method: 'GET',
+        params: {type: 'metric', category: 'query'},
+        isArray: false,
+        interceptor: {
+          response: function(response) {
+            var data = response.data;
+            var queryMetrics;
+            if (data) {
+              queryMetrics =  {queryCount: data.queryCount, queryLatency: {avg: data.avgQueryLatency/1000, max: data.maxQueryLatency/1000, min: data.minQueryLatency/1000}};
+            }
+            return queryMetrics;
+          }
+        }
+      },
+      getJobMetrics: {
+        method: 'GET', 
+        params: {type: 'metric', category: 'job'},
+        isArray: false,
+        interceptor: {
+          response: function(response) {
+            var data = response.data;
+            var jobMetrics;
+            if (data) {
+              jobMetrics = {jobCount: data.jobCount, buildingTime: {avg: data.avgJobBuildTime*1024*1024/1000, max: data.maxJobBuildTime*1024*1024/1000, min: data.minJobBuildTime*1024*1024/1000}};
+            }
+            return jobMetrics;
+          }
+        }
+      },
+      getChartData: {
+        method: 'GET',
+        params: {type: 'chart'},
+        isArray: false,
+        interceptor: {
+          response: function(response) {
+            var data = response.data;
+            var chartValues = [];
+            if (data) {
+              angular.forEach(Object.keys(data), function(key) {
+                var label = key;
+                var regEx = /^\d{4}-\d{1,2}-\d{1,2}$/;
+                if(key.match(regEx)) {
+                  label = moment(key, 'YYYY-MM-DD').format('x');
+                }
+                var value = data[key];
+                if (response.config.url.indexOf('/JOB/AVG_JOB_BUILD_TIME') > -1) { // AVG Job Build time format ms to sec
+                  value = value*1024*1024/1000;
+                } else if (response.config.url.indexOf('/QUERY/AVG_QUERY_LATENCY') > -1) { // AVG Query Latency format ms to sec
+                  value = value/1000;
+                }
+                chartValues.push({'label': label, 'value': value});
+              });
+            }
+            return chartValues;
+          }
+        }
+      }
+    });
+}]);

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/app/less/app.less
----------------------------------------------------------------------
diff --git a/webapp/app/less/app.less b/webapp/app/less/app.less
index 85ad937..fcba436 100644
--- a/webapp/app/less/app.less
+++ b/webapp/app/less/app.less
@@ -842,3 +842,61 @@ pre {
   white-space: -o-pre-wrap;
   word-wrap: break-word;
 }
+/* dashboard page style*/
+.square {
+  border: 2px solid #ddd;
+  text-align: center;
+  width: 215px;
+  height: 170px;
+  cursor: zoom-in;
+  padding-top: 15px;
+  padding-bottom: 15px;
+  .title {
+    font-size: 22px;
+    height: 60px;
+  }
+  .metric {
+    font-size: 35px ;
+    font-weight: bolder;
+    color: #8b1;
+    display: inline-block;
+    white-space: nowrap;
+    .unit {
+      font-size: 16px;
+    }
+  }
+  .description {
+    font-size: 15px;
+    color: #6a6a6a;
+    display: block;
+  }
+}
+.square-active {
+  border: 2px solid #6a6a6a;
+}
+.square-big {
+  border: 5px solid #ddd;
+  text-align: center;
+  width: 165px;
+  height: 240px;
+  padding: 20px 0;
+  margin-bottom: 20px;
+  .title {
+    font-size: 24px;
+    height: 55px;
+  }
+  .metric {
+    padding: 15px 0;
+    font-size: 50px ;
+    font-weight: bolder;
+    height: 100px;
+    color: #06d;
+    .unit {
+      font-size: 35px;
+    }
+  }
+  .description {
+    font-size: 18px;
+    color: #6a6a6a;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/app/partials/dashboard/dashboard.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/dashboard/dashboard.html b/webapp/app/partials/dashboard/dashboard.html
new file mode 100644
index 0000000..891b4b9
--- /dev/null
+++ b/webapp/app/partials/dashboard/dashboard.html
@@ -0,0 +1,151 @@
+<!--
+* 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.
+-->
+<div class="container" style="min-width: 1140px;">
+  <div class="row" style="margin-top:22px">
+    <div class="col-sm-12">
+      <ol class="breadcrumb" style="display: inline-block; padding: 7px 0px; background-color: #ffffff;" ng-if="projectModel.selectedProject">
+        <li ng-if="selectedCube" ><a href="javascript:void(0)" ng-click="cleanSelectedCube()"><i class="fa fa-dashboard"></i> {{projectModel.selectedProject}}</a></li>
+        <li ng-if="!selectedCube"><i class="fa fa-dashboard"></i> {{projectModel.selectedProject}}</li>
+        <li class="active" ng-if="selectedCube">{{selectedCube}}</li>
+      </ol>
+      <div class="input-group pull-right" style="margin-bottom: 20px;">
+        <kylin-daterangepicker start-date="dateRange.startDate" end-date="dateRange.endDate" callback="changeDateRange"></kylin-daterangepicker>
+      </div>
+    </div>
+  </div>
+  <!-- metric boxs-->
+  <div class="row">
+    <div class="col-sm-2">
+      <div class="square-big" tooltip-placement="bottom" tooltip="As of {{cubeMetricsRefreshTime}}">
+        <div class="title">
+          TOTAL CUBE COUNT
+        </div>
+        <div class="metric" ng-if="cubeMetrics.totalCubes || (cubeMetrics.totalCubes === 0)">
+          {{cubeMetrics.totalCubes | number:0}}
+        </div>
+        <div class="metric" ng-if="!cubeMetrics.totalCubes && (cubeMetrics.totalCubes !== 0)">
+          --
+        </div>
+        <a class="description" ng-href="model">
+          More Details
+        </a>
+      </div>
+      <div class="square-big" tooltip-placement="bottom" tooltip-html-unsafe="Max: {{cubeMetrics.expansionRate.max | number:2}} | Min: {{cubeMetrics.expansionRate.min | number:2}}</br> As of {{cubeMetricsRefreshTime}}">
+        <div class="title">
+          AVG CUBE EXPANSION
+        </div>
+        <div class="metric" ng-if="cubeMetrics.expansionRate.avg || (cubeMetrics.expansionRate.avg === 0)">
+          {{cubeMetrics.expansionRate.avg | number:2}}
+        </div>
+        <div class="metric" ng-if="!cubeMetrics.expansionRate.avg && (cubeMetrics.expansionRate.avg !== 0)">
+          --
+        </div>
+        <a class="description" ng-href="model">
+          More Details
+        </a>
+      </div>
+    </div>
+    <div class="col-sm-10">
+      <div class="row">
+        <div class="col-sm-3">
+          <div class="square" ng-class="{'square-active': currentSquare ==='queryCount'}"  ng-click="queryCountChart()">
+            <div class="title">
+              QUERY<br/>COUNT
+            </div>
+            <div class="metric" ng-if="queryMetrics.queryCount || (queryMetrics.queryCount === 0)">
+              {{queryMetrics.queryCount | number:0}}
+            </div>
+            <div class="metric" ng-if="!queryMetrics.queryCount && (queryMetrics.queryCount !== 0)">
+              --
+            </div>
+            <a class="description" ng-href="query?queryPanel=cached" ng-click="$event.stopPropagation();">
+              More Details
+            </a>
+          </div>
+        </div>
+        <div class="col-sm-3">
+          <div class="square" ng-class="{'square-active': currentSquare ==='queryAvg'}"  tooltip-placement="bottom" tooltip="Max: {{queryMetrics.queryLatency.max ? (queryMetrics.queryLatency.max | number:2) : '--'}} sec | Min: {{queryMetrics.queryLatency.min ? (queryMetrics.queryLatency.min| number:2) : '--'}} sec" ng-click="queryAvgChart()">
+            <div class="title">
+              AVG QUERY LATENCY
+            </div>
+            <div class="metric" ng-if="queryMetrics.queryLatency.avg || (queryMetrics.queryLatency.avg === 0)">
+              {{queryMetrics.queryLatency.avg | number:2}}<span class="unit"> sec</span>
+            </div>
+            <div class="metric" ng-if="!queryMetrics.queryLatency.avg && (queryMetrics.queryLatency.avg !== 0)">
+              --
+            </div>
+            <a class="description" ng-href="query?queryPanel=cached" ng-click="$event.stopPropagation();">
+              More Details
+            </a>
+          </div>
+        </div>
+        <div class="col-sm-3">
+          <div class="square" ng-class="{'square-active': currentSquare ==='jobCount'}"  ng-click="jobCountChart()">
+            <div class="title">
+              JOB<br/>COUNT
+            </div>
+            <div class="metric" ng-if="jobMetrics.jobCount || (jobMetrics.jobCount === 0)">
+              {{jobMetrics.jobCount | number:0}}
+            </div>
+            <div class="metric" ng-if="!jobMetrics.jobCount && (jobMetrics.jobCount !== 0)">
+              --
+            </div>
+            <a class="description" ng-href="jobs" ng-click="$event.stopPropagation();">
+              More Details
+            </a>
+          </div>
+        </div>
+        <div class="col-sm-3" >
+          <div class="square" ng-class="{'square-active': currentSquare ==='jobBuildTime'}" tooltip-placement="bottom" tooltip="Max: {{jobMetrics.buildingTime.max ? (jobMetrics.buildingTime.max | number:2) : '--'}} sec | Min: {{jobMetrics.buildingTime.min ? ( jobMetrics.buildingTime.min | number:2) : '--'}} sec" ng-click="jobBuildTimeChart()">
+            <div class="title">
+              AVG BUILD TIME PER MB
+            </div>
+            <div class="metric" ng-if="jobMetrics.buildingTime.avg || jobMetrics.buildingTime.avg === 0">
+              {{jobMetrics.buildingTime.avg | number:2}}<span class="unit"> sec</span>
+            </div>
+            <div class="metric" ng-if="!jobMetrics.buildingTime.avg && (jobMetrics.buildingTime.avg !== 0)">
+               --
+            </div>
+            <a class="description" ng-href="jobs" ng-click="$event.stopPropagation();">
+              More Details
+            </a>
+          </div>
+        </div>
+      </div>
+      <div class="row charts">
+        <div class="col-sm-6" ng-if="barChart">
+          <div style="border: 2px solid #ddd; margin-top:15px;">
+            <div class="form-group" style="width: 90px; position: absolute; right: 15px; bottom: 265px;">
+              showValue: <input type="checkbox" ng-model="barChart.options.chart.showValues">
+            </div>
+            <nvd3 options="barChart.options" data="barChart.data"></nvd3>
+          </div>
+        </div>
+        <div class="col-sm-6" ng-if="lineChart">
+          <div style="border: 2px solid #ddd; margin-top:15px;">
+            <div class="form-group">
+              <select class="form-control" style="width: 80px; position: absolute; right: 20px; bottom: 270px;" ng-options="option.name for option in lineChart.dimension.options track by option.value" ng-model="lineChart.dimension.selected" ng-change="changeDimensionFilter('line')">
+              </select>
+            </div>
+            <nvd3 options="lineChart.options" data="lineChart.data"></nvd3>
+          </div>
+        </div>
+      </div>
+    </div>
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/app/partials/header.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/header.html b/webapp/app/partials/header.html
index 0e37e63..fd22f9c 100644
--- a/webapp/app/partials/header.html
+++ b/webapp/app/partials/header.html
@@ -53,6 +53,9 @@
           <li class="{{activeTab=='admin'?'purple':'green'}}" ng-if="userService.hasRole('ROLE_ADMIN')">
             <a href="admin">System</a>
           </li>
+          <li class="{{activeTab=='dashboard'?'purple':'green'}}" ng-if="userService.isAuthorized()">
+            <a href="dashboard">Dashboard</a>
+          </li>
         </ul>
         <ul class="nav navbar-nav navbar-right">
           <li class="dropdown light-blue" ng-if="config.documents.length > 0">

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/app/routes.json
----------------------------------------------------------------------
diff --git a/webapp/app/routes.json b/webapp/app/routes.json
index 6729daf..65140a0 100644
--- a/webapp/app/routes.json
+++ b/webapp/app/routes.json
@@ -112,5 +112,13 @@
       "tab": "models",
       "controller": "ModelEditCtrl"
     }
+  },
+  {
+    "url": "/dashboard",
+    "params": {
+      "templateUrl": "partials/dashboard/dashboard.html",
+      "tab": "dashboard",
+      "controller": "DashboardCtrl"
+    }
   }
 ]

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/bower.json
----------------------------------------------------------------------
diff --git a/webapp/bower.json b/webapp/bower.json
index eac2cb3..f108862 100755
--- a/webapp/bower.json
+++ b/webapp/bower.json
@@ -17,7 +17,7 @@
     "angular-tree-control": "0.1.4",
     "angularLocalStorage": "~0.3.0",
     "messenger": "1.4.1",
-    "moment": "2.5.1",
+    "moment": "2.10.6",
     "d3": "3.4.4",
     "nvd3": "1.8.4",
     "angular-sweetalert": "~1.0.3",
@@ -35,7 +35,9 @@
     "angular-sanitize": "1.2.18",
     "angular-tree-control": "0.2.8",
     "angular-bootstrap-datetimepicker": "0.3.15",
-    "angular-nvd3": "1.0.9"
+    "angular-nvd3": "1.0.9",
+    "bootstrap-daterangepicker": "~1.3.23",
+    "moment-timezone" : "~0.5.5"
   },
   "devDependencies": {
     "less.js": "~1.4.0",
@@ -45,7 +47,7 @@
     "angular": "1.2.29",
     "nvd3": "1.8.4",
     "d3": "3.4.4",
-    "moment": "2.4.0",
+    "moment": "2.10.6",
     "angular-resource": "1.2.15",
     "angularLocalStorage": "0.1.7",
     "angular-cookies": "~1.2.0-rc.2",

http://git-wip-us.apache.org/repos/asf/kylin/blob/d608fc10/webapp/grunt.json
----------------------------------------------------------------------
diff --git a/webapp/grunt.json b/webapp/grunt.json
index 06720b2..b86f60b 100755
--- a/webapp/grunt.json
+++ b/webapp/grunt.json
@@ -48,6 +48,8 @@
         "app/components/angular-ui-select/dist/select.js",
         "app/components/angular-sanitize/angular-sanitize.js",
         "app/components/angular-nvd3/dist/angular-nvd3.js",
+        "app/components/moment-timezone/builds/moment-timezone-with-data.js",
+        "app/components/bootstrap-daterangepicker/daterangepicker.js",
         "tmp/js/scripts.js"
       ],
       "dest": "tmp/js/scripts.min.js"
@@ -73,6 +75,7 @@
         "app/components/angular-toggle-switch/angular-toggle-switch.css",
         "app/components/angular-ui-select/dist/select.css",
         "app/components/angular-bootstrap-datetimepicker/src/css/datetimepicker.css",
+        "app/components/bootstrap-daterangepicker/daterangepicker-bs3.css",
         "tmp/css/styles.css"
       ],
       "dest": "tmp/css/styles.min.css"


[05/18] kylin git commit: APACHE-KYLIN-2733: Introduce optimize job for adjusting cuboid set

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/ReducerNumSizing.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/ReducerNumSizing.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/ReducerNumSizing.java
deleted file mode 100644
index 5c0555a..0000000
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/ReducerNumSizing.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *  
- *     http://www.apache.org/licenses/LICENSE-2.0
- *  
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.engine.mr.steps;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.engine.mr.common.CubeStatsReader;
-import org.apache.kylin.job.exception.JobException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ReducerNumSizing {
-
-    private static final Logger logger = LoggerFactory.getLogger(ReducerNumSizing.class);
-
-    public static int getLayeredCubingReduceTaskNum(CubeSegment cubeSegment, double totalMapInputMB, int level) throws ClassNotFoundException, IOException, InterruptedException, JobException {
-        CubeDesc cubeDesc = cubeSegment.getCubeDesc();
-        KylinConfig kylinConfig = cubeDesc.getConfig();
-
-        double perReduceInputMB = kylinConfig.getDefaultHadoopJobReducerInputMB();
-        double reduceCountRatio = kylinConfig.getDefaultHadoopJobReducerCountRatio();
-        logger.info("Having per reduce MB " + perReduceInputMB + ", reduce count ratio " + reduceCountRatio + ", level " + level);
-
-        CubeStatsReader cubeStatsReader = new CubeStatsReader(cubeSegment, kylinConfig);
-
-        double parentLayerSizeEst, currentLayerSizeEst, adjustedCurrentLayerSizeEst;
-
-        if (level == -1) {
-            //merge case
-            double estimatedSize = cubeStatsReader.estimateCubeSize();
-            adjustedCurrentLayerSizeEst = estimatedSize > totalMapInputMB ? totalMapInputMB : estimatedSize;
-            logger.debug("estimated size {}, input size {}, adjustedCurrentLayerSizeEst: {}", estimatedSize, totalMapInputMB, adjustedCurrentLayerSizeEst);
-        } else if (level == 0) {
-            //base cuboid case TODO: the estimation could be very WRONG because it has no correction
-            adjustedCurrentLayerSizeEst = cubeStatsReader.estimateLayerSize(0);
-            logger.debug("adjustedCurrentLayerSizeEst: {}", adjustedCurrentLayerSizeEst);
-        } else {
-            parentLayerSizeEst = cubeStatsReader.estimateLayerSize(level - 1);
-            currentLayerSizeEst = cubeStatsReader.estimateLayerSize(level);
-            adjustedCurrentLayerSizeEst = totalMapInputMB / parentLayerSizeEst * currentLayerSizeEst;
-            logger.debug("totalMapInputMB: {}, parentLayerSizeEst: {}, currentLayerSizeEst: {}, adjustedCurrentLayerSizeEst: {}", totalMapInputMB, parentLayerSizeEst, currentLayerSizeEst, adjustedCurrentLayerSizeEst);
-        }
-
-        // number of reduce tasks
-        int numReduceTasks = (int) Math.round(adjustedCurrentLayerSizeEst / perReduceInputMB * reduceCountRatio + 0.99);
-
-        // adjust reducer number for cube which has DISTINCT_COUNT measures for better performance
-        if (cubeDesc.hasMemoryHungryMeasures()) {
-            logger.debug("Multiply reducer num by 4 to boost performance for memory hungry measures");
-            numReduceTasks = numReduceTasks * 4;
-        }
-
-        // at least 1 reducer by default
-        numReduceTasks = Math.max(kylinConfig.getHadoopJobMinReducerNumber(), numReduceTasks);
-        // no more than 500 reducer by default
-        numReduceTasks = Math.min(kylinConfig.getHadoopJobMaxReducerNumber(), numReduceTasks);
-
-        return numReduceTasks;
-    }
-
-    public static int getInmemCubingReduceTaskNum(CubeSegment cubeSeg) throws IOException {
-        KylinConfig kylinConfig = cubeSeg.getConfig();
-
-        Map<Long, Double> cubeSizeMap = new CubeStatsReader(cubeSeg, kylinConfig).getCuboidSizeMap();
-        double totalSizeInM = 0;
-        for (Double cuboidSize : cubeSizeMap.values()) {
-            totalSizeInM += cuboidSize;
-        }
-
-        double perReduceInputMB = kylinConfig.getDefaultHadoopJobReducerInputMB();
-
-        // number of reduce tasks
-        int numReduceTasks = (int) Math.round(totalSizeInM / perReduceInputMB);
-
-        // at least 1 reducer by default
-        numReduceTasks = Math.max(kylinConfig.getHadoopJobMinReducerNumber(), numReduceTasks);
-        // no more than 500 reducer by default
-        numReduceTasks = Math.min(kylinConfig.getHadoopJobMaxReducerNumber(), numReduceTasks);
-
-        logger.info("Having total map input MB " + Math.round(totalSizeInM));
-        logger.info("Having per reduce MB " + perReduceInputMB);
-        logger.info("Setting " + Reducer.Context.NUM_REDUCES + "=" + numReduceTasks);
-        return numReduceTasks;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterCheckpointStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterCheckpointStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterCheckpointStep.java
new file mode 100644
index 0000000..ed61b4a
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterCheckpointStep.java
@@ -0,0 +1,69 @@
+/*
+ * 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.mr.steps;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.common.CuboidStatsReaderUtil;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ */
+public class UpdateCubeInfoAfterCheckpointStep extends AbstractExecutable {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateCubeInfoAfterCheckpointStep.class);
+
+    public UpdateCubeInfoAfterCheckpointStep() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final CubeManager cubeManager = CubeManager.getInstance(context.getConfig());
+        final CubeInstance cube = cubeManager.getCube(CubingExecutableUtil.getCubeName(this.getParams()));
+
+        Set<Long> recommendCuboids = cube.getCuboidsRecommend();
+        try {
+            List<CubeSegment> newSegments = cube.getSegments(SegmentStatusEnum.READY_PENDING);
+            Map<Long, Long> recommendCuboidsWithStats = CuboidStatsReaderUtil
+                    .readCuboidStatsFromSegments(recommendCuboids, newSegments);
+            if (recommendCuboidsWithStats == null) {
+                throw new RuntimeException("Fail to get statistics info for recommended cuboids after optimization!!!");
+            }
+            cubeManager.promoteCheckpointOptimizeSegments(cube, recommendCuboidsWithStats,
+                    newSegments.toArray(new CubeSegment[newSegments.size()]));
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+        } catch (Exception e) {
+            logger.error("fail to update cube after build", e);
+            return ExecuteResult.createError(e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterOptimizeStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterOptimizeStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterOptimizeStep.java
new file mode 100644
index 0000000..13c4f40
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateCubeInfoAfterOptimizeStep.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr.steps;
+
+import java.io.IOException;
+
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.CubingJob;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ */
+public class UpdateCubeInfoAfterOptimizeStep extends AbstractExecutable {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateCubeInfoAfterOptimizeStep.class);
+
+    public UpdateCubeInfoAfterOptimizeStep() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final CubeManager cubeManager = CubeManager.getInstance(context.getConfig());
+        final CubeInstance cube = cubeManager.getCube(CubingExecutableUtil.getCubeName(this.getParams()));
+        final CubeSegment segment = cube.getSegmentById(CubingExecutableUtil.getSegmentId(this.getParams()));
+
+        CubeSegment originalSegment = cube.getOriginalSegmentToOptimize(segment);
+        long sourceCount = originalSegment.getInputRecords();
+        long sourceSizeBytes = originalSegment.getInputRecordsSize();
+
+        CubingJob cubingJob = (CubingJob) getManager().getJob(CubingExecutableUtil.getCubingJobId(this.getParams()));
+        long cubeSizeBytes = cubingJob.findCubeSizeBytes();
+
+        segment.setLastBuildJobID(CubingExecutableUtil.getCubingJobId(this.getParams()));
+        segment.setLastBuildTime(System.currentTimeMillis());
+        segment.setSizeKB(cubeSizeBytes / 1024);
+        segment.setInputRecords(sourceCount);
+        segment.setInputRecordsSize(sourceSizeBytes);
+
+        try {
+            cubeManager.promoteNewlyOptimizeSegments(cube, segment);
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+        } catch (IOException e) {
+            logger.error("fail to update cube after build", e);
+            return ExecuteResult.createError(e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateOldCuboidShardJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateOldCuboidShardJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateOldCuboidShardJob.java
new file mode 100644
index 0000000..0cd7264
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateOldCuboidShardJob.java
@@ -0,0 +1,105 @@
+/*
+ * 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.mr.steps;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+public class UpdateOldCuboidShardJob extends AbstractHadoopJob {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateOldCuboidShardJob.class);
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_SEGMENT_ID);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            parseOptions(options, args);
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            String segmentID = getOptionValue(OPTION_SEGMENT_ID);
+            Path input = new Path(getOptionValue(OPTION_INPUT_PATH));
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+
+            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            CubeSegment optSegment = cube.getSegmentById(segmentID);
+            CubeSegment originalSegment = cube.getOriginalSegmentToOptimize(optSegment);
+
+            logger.info("Starting: " + job.getJobName());
+
+            setJobClasspath(job, cube.getConfig());
+
+            // Mapper
+            job.setMapperClass(UpdateOldCuboidShardMapper.class);
+
+            // Reducer
+            job.setNumReduceTasks(0);
+
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(Text.class);
+
+            // Input
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+            FileInputFormat.setInputPaths(job, input);
+            // Output
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            FileOutputFormat.setOutputPath(job, output);
+
+            // set job configuration
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_ID, segmentID);
+            // add metadata to distributed cache
+            attachSegmentsMetadataWithDict(Lists.newArrayList(optSegment, originalSegment), job.getConfiguration());
+
+            this.deletePath(job.getConfiguration(), output);
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            logger.error("error in CuboidJob", e);
+            printUsage(options);
+            throw e;
+        } finally {
+            if (job != null)
+                cleanupTempConfFile(job.getConfiguration());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateOldCuboidShardMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateOldCuboidShardMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateOldCuboidShardMapper.java
new file mode 100644
index 0000000..58b553e
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UpdateOldCuboidShardMapper.java
@@ -0,0 +1,142 @@
+/*
+ * 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.mr.steps;
+
+import static org.apache.kylin.engine.mr.JobBuilderSupport.PathNameCuboidBase;
+import static org.apache.kylin.engine.mr.JobBuilderSupport.PathNameCuboidOld;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.MultipleOutputs;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.SplittedBytes;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.common.RowKeySplitter;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.kv.RowKeyEncoder;
+import org.apache.kylin.cube.kv.RowKeyEncoderProvider;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.engine.mr.KylinMapper;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateOldCuboidShardMapper extends KylinMapper<Text, Text, Text, Text> {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateOldCuboidShardMapper.class);
+
+    private MultipleOutputs mos;
+    private long baseCuboid;
+
+    private CubeDesc cubeDesc;
+    private RowKeySplitter rowKeySplitter;
+    private RowKeyEncoderProvider rowKeyEncoderProvider;
+
+    private Text outputKey = new Text();
+    private byte[] newKeyBodyBuf = new byte[RowConstants.ROWKEY_BUFFER_SIZE];
+    private ByteArray newKeyBuf = ByteArray.allocate(RowConstants.ROWKEY_BUFFER_SIZE);
+
+    @Override
+    protected void doSetup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+        mos = new MultipleOutputs(context);
+
+        String cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME);
+        String segmentID = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_ID);
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+
+        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
+        CubeSegment cubeSegment = cube.getSegmentById(segmentID);
+        CubeSegment oldSegment = cube.getOriginalSegmentToOptimize(cubeSegment);
+
+        cubeDesc = cube.getDescriptor();
+        baseCuboid = cube.getCuboidScheduler().getBaseCuboidId();
+
+        rowKeySplitter = new RowKeySplitter(oldSegment, 65, 256);
+        rowKeyEncoderProvider = new RowKeyEncoderProvider(cubeSegment);
+    }
+
+    @Override
+    public void doMap(Text key, Text value, Context context) throws IOException, InterruptedException {
+        long cuboidID = rowKeySplitter.split(key.getBytes());
+
+        Cuboid cuboid = Cuboid.findForMandatory(cubeDesc, cuboidID);
+        int fullKeySize = buildKey(cuboid, rowKeySplitter.getSplitBuffers());
+        outputKey.set(newKeyBuf.array(), 0, fullKeySize);
+
+        String baseOutputPath = PathNameCuboidOld;
+        if (cuboidID == baseCuboid) {
+            baseOutputPath = PathNameCuboidBase;
+        }
+        mos.write(outputKey, value, generateFileName(baseOutputPath));
+    }
+
+    private int buildKey(Cuboid cuboid, SplittedBytes[] splitBuffers) {
+        RowKeyEncoder rowkeyEncoder = rowKeyEncoderProvider.getRowkeyEncoder(cuboid);
+
+        int startIdx = rowKeySplitter.getBodySplitOffset(); // skip shard and cuboidId
+        int endIdx = startIdx + Long.bitCount(cuboid.getId());
+        int offset = 0;
+        for (int i = startIdx; i < endIdx; i++) {
+            System.arraycopy(splitBuffers[i].value, 0, newKeyBodyBuf, offset, splitBuffers[i].length);
+            offset += splitBuffers[i].length;
+        }
+
+        int fullKeySize = rowkeyEncoder.getBytesLength();
+        while (newKeyBuf.array().length < fullKeySize) {
+            newKeyBuf = new ByteArray(newKeyBuf.length() * 2);
+        }
+        newKeyBuf.setLength(fullKeySize);
+
+        rowkeyEncoder.encode(new ByteArray(newKeyBodyBuf, 0, offset), newKeyBuf);
+
+        return fullKeySize;
+    }
+
+    @Override
+    public void doCleanup(Context context) throws IOException, InterruptedException {
+        mos.close();
+
+        Path outputDirBase = new Path(context.getConfiguration().get(FileOutputFormat.OUTDIR), PathNameCuboidBase);
+        FileSystem fs = FileSystem.get(context.getConfiguration());
+        if (!fs.exists(outputDirBase)) {
+            fs.mkdirs(outputDirBase);
+            SequenceFile
+                    .createWriter(context.getConfiguration(),
+                            SequenceFile.Writer.file(new Path(outputDirBase, "part-m-00000")),
+                            SequenceFile.Writer.keyClass(Text.class), SequenceFile.Writer.valueClass(Text.class))
+                    .close();
+        }
+    }
+
+    private String generateFileName(String subDir) {
+        return subDir + "/part";
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
index 06cc988..bdf0633 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
@@ -233,7 +233,7 @@ public class SparkCubingByLayer extends AbstractApplication implements Serializa
         final String cuboidOutputPath = BatchCubingJobBuilder2.getCuboidOutputPathsByLevel(hdfsBaseLocation, level);
 
         IMROutput2.IMROutputFormat outputFormat = MRUtil.getBatchCubingOutputSide2(cubeSeg).getOuputFormat();
-        outputFormat.configureJobOutput(job, cuboidOutputPath, cubeSeg, level);
+        outputFormat.configureJobOutput(job, cuboidOutputPath, cubeSeg, cubeSeg.getCuboidScheduler(), level);
 
         prepareOutput(rdd, kylinConfig, cubeSeg, level).mapToPair(
                 new PairFunction<Tuple2<ByteArray, Object[]>, org.apache.hadoop.io.Text, org.apache.hadoop.io.Text>() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
index ad00706..77bd498 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
@@ -38,6 +38,7 @@ import org.apache.kylin.dimension.DimensionEncodingFactory;
 import org.apache.kylin.engine.EngineFactory;
 import org.apache.kylin.job.JobInstance;
 import org.apache.kylin.job.JoinedFlatTable;
+import org.apache.kylin.job.exception.JobException;
 import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
 import org.apache.kylin.metadata.model.ISourceAware;
 import org.apache.kylin.metadata.model.SegmentRange;
@@ -51,6 +52,7 @@ import org.apache.kylin.rest.exception.NotFoundException;
 import org.apache.kylin.rest.request.CubeRequest;
 import org.apache.kylin.rest.request.JobBuildRequest;
 import org.apache.kylin.rest.request.JobBuildRequest2;
+import org.apache.kylin.rest.request.JobOptimizeRequest;
 import org.apache.kylin.rest.response.EnvelopeResponse;
 import org.apache.kylin.rest.response.GeneralResponse;
 import org.apache.kylin.rest.response.HBaseResponse;
@@ -362,6 +364,67 @@ public class CubeController extends BasicController {
         }
     }
 
+    /**
+     * Send a optimize cube job
+     *
+     * @param cubeName Cube ID
+     * @return JobInstance of CheckpointExecutable
+     */
+    @RequestMapping(value = "/{cubeName}/optimize", method = { RequestMethod.PUT })
+    @ResponseBody
+    public JobInstance optimize(@PathVariable String cubeName, @RequestBody JobOptimizeRequest jobOptimizeRequest) {
+        try {
+            String submitter = SecurityContextHolder.getContext().getAuthentication().getName();
+            CubeInstance cube = jobService.getCubeManager().getCube(cubeName);
+
+            if (cube == null) {
+                throw new InternalErrorException("Cannot find cube " + cubeName);
+            }
+            logger.info("cuboid recommend:" + jobOptimizeRequest.getCuboidsRecommend());
+            return jobService.submitOptimizeJob(cube, jobOptimizeRequest.getCuboidsRecommend(), submitter).getFirst();
+        } catch (BadRequestException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw e;
+        } catch (JobException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new BadRequestException(e.getLocalizedMessage());
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        }
+    }
+
+    /**
+     * Send a optimize cube segment job
+     *
+     * @param cubeName Cube ID
+     * @param segmentID for segment to be optimized
+     */
+    @RequestMapping(value = "/{cubeName}/recover_segment_optimize/{segmentID}", method = { RequestMethod.PUT })
+    @ResponseBody
+    public JobInstance recoverSegmentOptimize(@PathVariable String cubeName, @PathVariable String segmentID) {
+        try {
+            String submitter = SecurityContextHolder.getContext().getAuthentication().getName();
+            CubeInstance cube = jobService.getCubeManager().getCube(cubeName);
+            if (cube == null) {
+                throw new InternalErrorException("Cannot find cube " + cubeName);
+            }
+
+            CubeSegment segment = cube.getSegmentById(segmentID);
+            if (segment == null) {
+                throw new InternalErrorException("Cannot find segment '" + segmentID + "'");
+            }
+
+            return jobService.submitRecoverSegmentOptimizeJob(segment, submitter);
+        } catch (JobException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new BadRequestException(e.getLocalizedMessage());
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        }
+    }
+
     @RequestMapping(value = "/{cubeName}/disable", method = { RequestMethod.PUT }, produces = { "application/json" })
     @ResponseBody
     public CubeInstance disableCube(@PathVariable String cubeName) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/server-base/src/main/java/org/apache/kylin/rest/request/JobOptimizeRequest.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/JobOptimizeRequest.java b/server-base/src/main/java/org/apache/kylin/rest/request/JobOptimizeRequest.java
new file mode 100644
index 0000000..51e8e7c
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/request/JobOptimizeRequest.java
@@ -0,0 +1,34 @@
+/*
+ * 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.rest.request;
+
+import java.util.Set;
+
+public class JobOptimizeRequest {
+
+    private Set<Long> cuboidsRecommend;
+
+    public Set<Long> getCuboidsRecommend() {
+        return cuboidsRecommend;
+    }
+
+    public void setCuboidsRecommend(Set<Long> cuboidsRecommend) {
+        this.cuboidsRecommend = cuboidsRecommend;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
index a18aaf3..d5805a1 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -72,6 +72,8 @@ import org.springframework.stereotype.Component;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 /**
  * Stateless & lightweight service facade of cube management functions.
@@ -525,6 +527,8 @@ public class CubeService extends BasicService implements InitializingBean {
 
         CubeUpdate update = new CubeUpdate(cube);
         update.setToRemoveSegs(cube.getSegments().toArray(new CubeSegment[cube.getSegments().size()]));
+        update.setCuboids(Maps.<Long, Long> newHashMap());
+        update.setCuboidsRecommend(Sets.<Long> newHashSet());
         CubeManager.getInstance(getConfig()).updateCube(update);
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
index 529f3b8..9f2d0d9 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
@@ -34,11 +34,13 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.directory.api.util.Strings;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.CubeUpdate;
 import org.apache.kylin.cube.model.CubeBuildTypeEnum;
 import org.apache.kylin.engine.EngineFactory;
+import org.apache.kylin.engine.mr.BatchOptimizeJobCheckpointBuilder;
 import org.apache.kylin.engine.mr.CubingJob;
 import org.apache.kylin.engine.mr.common.JobInfoConverter;
 import org.apache.kylin.engine.mr.steps.CubingExecutableUtil;
@@ -48,6 +50,7 @@ import org.apache.kylin.job.SchedulerFactory;
 import org.apache.kylin.job.constant.JobStatusEnum;
 import org.apache.kylin.job.constant.JobTimeFilterEnum;
 import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.job.exception.JobException;
 import org.apache.kylin.job.exception.SchedulerException;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.CheckpointExecutable;
@@ -72,6 +75,7 @@ import org.springframework.beans.factory.InitializingBean;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.beans.factory.annotation.Qualifier;
 import org.springframework.context.annotation.EnableAspectJAutoProxy;
+import org.springframework.security.core.context.SecurityContextHolder;
 import org.springframework.stereotype.Component;
 
 import com.google.common.base.Function;
@@ -273,6 +277,137 @@ public class JobService extends BasicService implements InitializingBean {
         return jobInstance;
     }
 
+    public Pair<JobInstance, List<JobInstance>> submitOptimizeJob(CubeInstance cube, Set<Long> cuboidsRecommend,
+            String submitter) throws IOException, JobException {
+
+        Pair<JobInstance, List<JobInstance>> result = submitOptimizeJobInternal(cube, cuboidsRecommend, submitter);
+        accessService.init(result.getFirst(), null);
+        accessService.inherit(result.getFirst(), cube);
+        for (JobInstance jobInstance : result.getSecond()) {
+            accessService.init(jobInstance, null);
+            accessService.inherit(jobInstance, cube);
+        }
+
+        return result;
+    }
+
+    private Pair<JobInstance, List<JobInstance>> submitOptimizeJobInternal(CubeInstance cube,
+            Set<Long> cuboidsRecommend, String submitter) throws IOException {
+        Message msg = MsgPicker.getMsg();
+
+        if (cube.getStatus() == RealizationStatusEnum.DESCBROKEN) {
+            throw new BadRequestException(String.format(msg.getBUILD_BROKEN_CUBE(), cube.getName()));
+        }
+
+        checkCubeDescSignature(cube);
+        checkAllowOptimization(cube, cuboidsRecommend);
+
+        CubeSegment[] optimizeSegments = null;
+        try {
+            /** Add optimize segments */
+            optimizeSegments = getCubeManager().optimizeSegments(cube, cuboidsRecommend);
+            List<JobInstance> optimizeJobInstances = Lists.newLinkedList();
+
+            /** Add optimize jobs */
+            List<AbstractExecutable> optimizeJobList = Lists.newArrayListWithExpectedSize(optimizeSegments.length);
+            for (CubeSegment optimizeSegment : optimizeSegments) {
+                DefaultChainedExecutable optimizeJob = EngineFactory.createBatchOptimizeJob(optimizeSegment, submitter);
+                getExecutableManager().addJob(optimizeJob);
+
+                optimizeJobList.add(optimizeJob);
+                optimizeJobInstances.add(getSingleJobInstance(optimizeJob));
+            }
+
+            /** Add checkpoint job for batch jobs */
+            CheckpointExecutable checkpointJob = new BatchOptimizeJobCheckpointBuilder(cube, submitter).build();
+            checkpointJob.addTaskListForCheck(optimizeJobList);
+
+            getExecutableManager().addJob(checkpointJob);
+
+            return new Pair(getCheckpointJobInstance(checkpointJob), optimizeJobInstances);
+        } catch (Exception e) {
+            if (optimizeSegments != null) {
+                logger.error("Job submission might failed for NEW segments {}, will clean the NEW segments from cube",
+                        optimizeSegments);
+                try {
+                    // Remove this segments
+                    CubeUpdate cubeBuilder = new CubeUpdate(cube);
+                    cubeBuilder.setToRemoveSegs(optimizeSegments);
+                    getCubeManager().updateCube(cubeBuilder);
+                } catch (Exception ee) {
+                    // swallow the exception
+                    logger.error("Clean New segments failed, ignoring it", e);
+                }
+            }
+            throw e;
+        }
+    }
+
+    public JobInstance submitRecoverSegmentOptimizeJob(CubeSegment segment, String submitter)
+            throws IOException, JobException {
+        CubeInstance cubeInstance = segment.getCubeInstance();
+
+        checkCubeDescSignature(cubeInstance);
+
+        String cubeName = cubeInstance.getName();
+        List<JobInstance> jobInstanceList = searchJobsByCubeName(cubeName, null,
+                Lists.newArrayList(JobStatusEnum.NEW, JobStatusEnum.PENDING, JobStatusEnum.ERROR),
+                JobTimeFilterEnum.ALL, JobSearchMode.CHECKPOINT_ONLY);
+        if (jobInstanceList.size() > 1) {
+            throw new IllegalStateException("Exist more than one CheckpointExecutable for cube " + cubeName);
+        } else if (jobInstanceList.size() == 0) {
+            throw new IllegalStateException("There's no CheckpointExecutable for cube " + cubeName);
+        }
+        CheckpointExecutable checkpointExecutable = (CheckpointExecutable) getExecutableManager()
+                .getJob(jobInstanceList.get(0).getId());
+
+        AbstractExecutable toBeReplaced = null;
+        for (AbstractExecutable taskForCheck : checkpointExecutable.getSubTasksForCheck()) {
+            if (taskForCheck instanceof CubingJob) {
+                CubingJob subCubingJob = (CubingJob) taskForCheck;
+                String segmentName = CubingExecutableUtil.getSegmentName(subCubingJob.getParams());
+                if (segmentName != null && segmentName.equals(segment.getName())) {
+                    String segmentID = CubingExecutableUtil.getSegmentId(subCubingJob.getParams());
+                    CubeSegment beingOptimizedSegment = cubeInstance.getSegmentById(segmentID);
+                    if (beingOptimizedSegment != null) { // beingOptimizedSegment exists & should not be recovered
+                        throw new IllegalStateException("Segment " + beingOptimizedSegment.getName() + "-"
+                                + beingOptimizedSegment.getUuid()
+                                + " still exists. Please delete it or discard the related optimize job first!!!");
+                    }
+                    toBeReplaced = taskForCheck;
+                    break;
+                }
+            }
+        }
+        if (toBeReplaced == null) {
+            throw new IllegalStateException("There's no CubingJob for segment " + segment.getName()
+                    + " in CheckpointExecutable " + checkpointExecutable.getName());
+        }
+
+        /** Add CubingJob for the related segment **/
+        CubeSegment optimizeSegment = getCubeManager().appendSegment(cubeInstance, segment.getTSRange());
+        CubeUpdate cubeBuilder = new CubeUpdate(cubeInstance);
+        cubeBuilder.setToAddSegs(optimizeSegment);
+        getCubeManager().updateCube(cubeBuilder);
+
+        DefaultChainedExecutable optimizeJob = EngineFactory.createBatchOptimizeJob(optimizeSegment, submitter);
+
+        getExecutableManager().addJob(optimizeJob);
+
+        JobInstance optimizeJobInstance = getSingleJobInstance(optimizeJob);
+        accessService.init(optimizeJobInstance, null);
+        accessService.inherit(optimizeJobInstance, cubeInstance);
+
+        /** Update the checkpoint job */
+        checkpointExecutable.getSubTasksForCheck().set(checkpointExecutable.getSubTasksForCheck().indexOf(toBeReplaced),
+                optimizeJob);
+
+        getExecutableManager().updateCheckpointJob(checkpointExecutable.getId(),
+                checkpointExecutable.getSubTasksForCheck());
+
+        return optimizeJobInstance;
+    }
+
     private void checkCubeDescSignature(CubeInstance cube) {
         Message msg = MsgPicker.getMsg();
 
@@ -281,8 +416,25 @@ public class JobService extends BasicService implements InitializingBean {
                     String.format(msg.getINCONSISTENT_CUBE_DESC_SIGNATURE(), cube.getDescriptor()));
     }
 
+    private void checkAllowOptimization(CubeInstance cube, Set<Long> cuboidsRecommend) {
+        long baseCuboid = cube.getCuboidScheduler().getBaseCuboidId();
+        if (!cuboidsRecommend.contains(baseCuboid)) {
+            throw new BadRequestException("The recommend cuboids should contain the base cuboid " + baseCuboid);
+        }
+        Set<Long> currentCuboidSet = cube.getCuboidScheduler().getAllCuboidIds();
+        if (currentCuboidSet.equals(cuboidsRecommend)) {
+            throw new BadRequestException(
+                    "The recommend cuboids are the same as the current cuboids. It's no need to do optimization.");
+        }
+    }
+
     public JobInstance getJobInstance(String uuid) {
-        return getSingleJobInstance(getExecutableManager().getJob(uuid));
+        AbstractExecutable job = getExecutableManager().getJob(uuid);
+        if (job instanceof CheckpointExecutable) {
+            return getCheckpointJobInstance(job);
+        } else {
+            return getSingleJobInstance(job);
+        }
     }
 
     public Output getOutput(String id) {
@@ -362,21 +514,90 @@ public class JobService extends BasicService implements InitializingBean {
             getExecutableManager().discardJob(job.getId());
             return job;
         }
-        CubeInstance cubeInstance = getCubeManager().getCube(job.getRelatedCube());
+
+        logger.info("Cancel job [" + job.getId() + "] trigger by "
+                + SecurityContextHolder.getContext().getAuthentication().getName());
+        if (job.getStatus() == JobStatusEnum.FINISHED) {
+            throw new IllegalStateException(
+                    "The job " + job.getId() + " has already been finished and cannot be discarded.");
+        }
+        if (job.getStatus() == JobStatusEnum.DISCARDED) {
+            return job;
+        }
+
+        AbstractExecutable executable = getExecutableManager().getJob(job.getId());
+        if (executable instanceof CubingJob) {
+            cancelCubingJobInner((CubingJob) executable);
+        } else if (executable instanceof CheckpointExecutable) {
+            cancelCheckpointJobInner((CheckpointExecutable) executable);
+        } else {
+            getExecutableManager().discardJob(executable.getId());
+        }
+        return job;
+    }
+
+    private void cancelCubingJobInner(CubingJob cubingJob) throws IOException {
+        CubeInstance cubeInstance = getCubeManager().getCube(CubingExecutableUtil.getCubeName(cubingJob.getParams()));
         // might not a cube job
-        final String segmentIds = job.getRelatedSegment();
-        for (String segmentId : StringUtils.split(segmentIds)) {
-            final CubeSegment segment = cubeInstance.getSegmentById(segmentId);
-            if (segment != null && (segment.getStatus() == SegmentStatusEnum.NEW || segment.getTSRange().end.v == 0)) {
-                // Remove this segments
+        final String segmentIds = CubingExecutableUtil.getSegmentId(cubingJob.getParams());
+        if (!StringUtils.isEmpty(segmentIds)) {
+            List<CubeSegment> toRemoveSegments = Lists.newLinkedList();
+            for (String segmentId : StringUtils.split(segmentIds)) {
+                final CubeSegment segment = cubeInstance.getSegmentById(segmentId);
+                if (segment != null
+                        && (segment.getStatus() == SegmentStatusEnum.NEW || segment.getTSRange().end.v == 0)) {
+                    // Remove this segment
+                    toRemoveSegments.add(segment);
+                }
+            }
+            if (!toRemoveSegments.isEmpty()) {
                 CubeUpdate cubeBuilder = new CubeUpdate(cubeInstance);
-                cubeBuilder.setToRemoveSegs(segment);
+                cubeBuilder.setToRemoveSegs(toRemoveSegments.toArray(new CubeSegment[toRemoveSegments.size()]));
                 getCubeManager().updateCube(cubeBuilder);
             }
         }
-        getExecutableManager().discardJob(job.getId());
+        getExecutableManager().discardJob(cubingJob.getId());
+    }
 
-        return job;
+    private void cancelCheckpointJobInner(CheckpointExecutable checkpointExecutable) throws IOException {
+        List<String> segmentIdList = Lists.newLinkedList();
+        List<String> jobIdList = Lists.newLinkedList();
+        jobIdList.add(checkpointExecutable.getId());
+        setRelatedIdList(checkpointExecutable, segmentIdList, jobIdList);
+
+        CubeInstance cubeInstance = getCubeManager()
+                .getCube(CubingExecutableUtil.getCubeName(checkpointExecutable.getParams()));
+        if (!segmentIdList.isEmpty()) {
+            List<CubeSegment> toRemoveSegments = Lists.newLinkedList();
+            for (String segmentId : segmentIdList) {
+                final CubeSegment segment = cubeInstance.getSegmentById(segmentId);
+                if (segment != null && segment.getStatus() != SegmentStatusEnum.READY) {
+                    toRemoveSegments.add(segment);
+                }
+            }
+
+            CubeUpdate cubeBuilder = new CubeUpdate(cubeInstance);
+            cubeBuilder.setToRemoveSegs(toRemoveSegments.toArray(new CubeSegment[toRemoveSegments.size()]));
+            cubeBuilder.setCuboidsRecommend(Sets.<Long> newHashSet()); //Set recommend cuboids to be null
+            getCubeManager().updateCube(cubeBuilder);
+        }
+
+        for (String jobId : jobIdList) {
+            getExecutableManager().discardJob(jobId);
+        }
+    }
+
+    private void setRelatedIdList(CheckpointExecutable checkpointExecutable, List<String> segmentIdList,
+            List<String> jobIdList) {
+        for (AbstractExecutable taskForCheck : checkpointExecutable.getSubTasksForCheck()) {
+            jobIdList.add(taskForCheck.getId());
+            if (taskForCheck instanceof CubingJob) {
+                segmentIdList.addAll(Lists
+                        .newArrayList(StringUtils.split(CubingExecutableUtil.getSegmentId(taskForCheck.getParams()))));
+            } else if (taskForCheck instanceof CheckpointExecutable) {
+                setRelatedIdList((CheckpointExecutable) taskForCheck, segmentIdList, jobIdList);
+            }
+        }
     }
 
     public JobInstance pauseJob(JobInstance job) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java
index c41df06..838112f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java
@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
@@ -67,6 +68,7 @@ public class CreateHTableJob extends AbstractHadoopJob {
     CubeInstance cube = null;
     CubeDesc cubeDesc = null;
     String segmentID = null;
+    String cuboidModeName = null;
     KylinConfig kylinConfig;
     Path partitionFilePath;
 
@@ -78,6 +80,7 @@ public class CreateHTableJob extends AbstractHadoopJob {
         options.addOption(OPTION_SEGMENT_ID);
         options.addOption(OPTION_PARTITION_FILE_PATH);
         options.addOption(OPTION_STATISTICS_ENABLED);
+        options.addOption(OPTION_CUBOID_MODE);
         parseOptions(options, args);
 
         partitionFilePath = new Path(getOptionValue(OPTION_PARTITION_FILE_PATH));
@@ -89,13 +92,27 @@ public class CreateHTableJob extends AbstractHadoopJob {
         cubeDesc = cube.getDescriptor();
         kylinConfig = cube.getConfig();
         segmentID = getOptionValue(OPTION_SEGMENT_ID);
+        cuboidModeName = getOptionValue(OPTION_CUBOID_MODE);
         CubeSegment cubeSegment = cube.getSegmentById(segmentID);
 
         Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
 
         byte[][] splitKeys;
         if (statsEnabled) {
-            final Map<Long, Double> cuboidSizeMap = new CubeStatsReader(cubeSegment, kylinConfig).getCuboidSizeMap();
+            Map<Long, Double> cuboidSizeMap = new CubeStatsReader(cubeSegment, null, kylinConfig).getCuboidSizeMap();
+            Set<Long> buildingCuboids = cube.getCuboidsByMode(cuboidModeName);
+            if (buildingCuboids != null && !buildingCuboids.isEmpty()) {
+                Map<Long, Double> optimizedCuboidSizeMap = Maps.newHashMapWithExpectedSize(buildingCuboids.size());
+                for (Long cuboid : buildingCuboids) {
+                    Double cuboidSize = cuboidSizeMap.get(cuboid);
+                    if (cuboidSize == null) {
+                        logger.warn(cuboid + "cuboid's size is null will replace by 0");
+                        cuboidSize = 0.0;
+                    }
+                    optimizedCuboidSizeMap.put(cuboid, cuboidSize);
+                }
+                cuboidSizeMap = optimizedCuboidSizeMap;
+            }
             splitKeys = getRegionSplitsFromCuboidStatistics(cuboidSizeMap, kylinConfig, cubeSegment, partitionFilePath.getParent());
         } else {
             splitKeys = getRegionSplits(conf, partitionFilePath);

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
index 31cb189..db3f7f4 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
@@ -31,13 +31,15 @@ import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidModeEnum;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
 import org.apache.kylin.engine.mr.IMROutput2;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.MapReduceUtil;
 import org.apache.kylin.engine.mr.steps.HiveToBaseCuboidMapper;
 import org.apache.kylin.engine.mr.steps.InMemCuboidMapper;
 import org.apache.kylin.engine.mr.steps.MergeCuboidJob;
 import org.apache.kylin.engine.mr.steps.NDCuboidMapper;
-import org.apache.kylin.engine.mr.steps.ReducerNumSizing;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -93,13 +95,15 @@ public class HBaseMROutput2Transition implements IMROutput2 {
         }
 
         @Override
-        public void configureJobOutput(Job job, String output, CubeSegment segment, int level) throws Exception {
+        public void configureJobOutput(Job job, String output, CubeSegment segment, CuboidScheduler cuboidScheduler,
+                int level) throws Exception {
             int reducerNum = 1;
             Class mapperClass = job.getMapperClass();
             if (mapperClass == HiveToBaseCuboidMapper.class || mapperClass == NDCuboidMapper.class) {
-                reducerNum = ReducerNumSizing.getLayeredCubingReduceTaskNum(segment, AbstractHadoopJob.getTotalMapInputMB(job), level);
+                reducerNum = MapReduceUtil.getLayeredCubingReduceTaskNum(segment, cuboidScheduler,
+                        AbstractHadoopJob.getTotalMapInputMB(job), level);
             } else if (mapperClass == InMemCuboidMapper.class) {
-                reducerNum = ReducerNumSizing.getInmemCubingReduceTaskNum(segment);
+                reducerNum = MapReduceUtil.getInmemCubingReduceTaskNum(segment, cuboidScheduler);
             }
             Path outputPath = new Path(output);
             FileOutputFormat.setOutputPath(job, outputPath);
@@ -149,7 +153,8 @@ public class HBaseMROutput2Transition implements IMROutput2 {
 
         @Override
         public void configureJobOutput(Job job, String output, CubeSegment segment) throws Exception {
-            int reducerNum = ReducerNumSizing.getLayeredCubingReduceTaskNum(segment, AbstractHadoopJob.getTotalMapInputMB(job), -1);
+            int reducerNum = MapReduceUtil.getLayeredCubingReduceTaskNum(segment, segment.getCuboidScheduler(),
+                    AbstractHadoopJob.getTotalMapInputMB(job), -1);
             job.setNumReduceTasks(reducerNum);
 
             Path outputPath = new Path(output);
@@ -185,4 +190,30 @@ public class HBaseMROutput2Transition implements IMROutput2 {
             throw new IllegalStateException("No merging segment's last build job ID equals " + jobID);
         }
     }
+
+    public IMRBatchOptimizeOutputSide2 getBatchOptimizeOutputSide(final CubeSegment seg) {
+        return new IMRBatchOptimizeOutputSide2() {
+            HBaseMRSteps steps = new HBaseMRSteps(seg);
+
+            @Override
+            public void addStepPhase2_CreateHTable(DefaultChainedExecutable jobFlow) {
+                jobFlow.addTask(steps.createCreateHTableStepWithStats(jobFlow.getId(), CuboidModeEnum.RECOMMEND));
+            }
+
+            @Override
+            public void addStepPhase3_BuildCube(DefaultChainedExecutable jobFlow) {
+                jobFlow.addTask(steps.createConvertCuboidToHfileStep(jobFlow.getId()));
+                jobFlow.addTask(steps.createBulkLoadStep(jobFlow.getId()));
+            }
+
+            public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow) {
+                steps.addOptimizeGarbageCollectionSteps(jobFlow);
+            }
+
+            @Override
+            public void addStepPhase5_Cleanup(DefaultChainedExecutable jobFlow) {
+                steps.addCheckpointGarbageCollectionSteps(jobFlow);
+            }
+        };
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/020c4e78/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
index 6f69e8c..13e7dc4 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidModeEnum;
 import org.apache.kylin.engine.mr.CubingJob;
 import org.apache.kylin.engine.mr.JobBuilderSupport;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
@@ -32,6 +33,7 @@ import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
 import org.apache.kylin.engine.mr.common.MapReduceExecutable;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 
 import com.google.common.base.Preconditions;
@@ -82,7 +84,15 @@ public class HBaseMRSteps extends JobBuilderSupport {
         return createCreateHTableStep(jobId, true);
     }
 
+    public HadoopShellExecutable createCreateHTableStepWithStats(String jobId, CuboidModeEnum cuboidMode) {
+        return createCreateHTableStep(jobId, true, cuboidMode);
+    }
+
     private HadoopShellExecutable createCreateHTableStep(String jobId, boolean withStats) {
+        return createCreateHTableStep(jobId, withStats, CuboidModeEnum.CURRENT);
+    }
+
+    private HadoopShellExecutable createCreateHTableStep(String jobId, boolean withStats, CuboidModeEnum cuboidMode) {
         HadoopShellExecutable createHtableStep = new HadoopShellExecutable();
         createHtableStep.setName(ExecutableConstants.STEP_NAME_CREATE_HBASE_TABLE);
         StringBuilder cmd = new StringBuilder();
@@ -90,6 +100,7 @@ public class HBaseMRSteps extends JobBuilderSupport {
         appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
         appendExecCmdParameters(cmd, BatchConstants.ARG_PARTITION, getRowkeyDistributionOutputPath(jobId) + "/part-r-00000");
         appendExecCmdParameters(cmd, BatchConstants.ARG_STATS_ENABLED, String.valueOf(withStats));
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBOID_MODE, cuboidMode.toString());
 
         createHtableStep.setJobParams(cmd.toString());
         createHtableStep.setJobClass(CreateHTableJob.class);
@@ -167,6 +178,35 @@ public class HBaseMRSteps extends JobBuilderSupport {
         return result;
     }
 
+    public MergeGCStep createOptimizeGCStep() {
+        MergeGCStep result = new MergeGCStep();
+        result.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION);
+        result.setOldHTables(getOptimizeHTables());
+        return result;
+    }
+
+    public List<CubeSegment> getOptimizeSegments() {
+        CubeInstance cube = (CubeInstance) seg.getRealization();
+        List<CubeSegment> newSegments = Lists.newArrayList(cube.getSegments(SegmentStatusEnum.READY_PENDING));
+        List<CubeSegment> oldSegments = Lists.newArrayListWithExpectedSize(newSegments.size());
+        for (CubeSegment segment : newSegments) {
+            oldSegments.add(cube.getOriginalSegmentToOptimize(segment));
+        }
+        return oldSegments;
+    }
+
+    public List<String> getOptimizeHTables() {
+        return getOldHTables(getOptimizeSegments());
+    }
+
+    public List<String> getOldHTables(final List<CubeSegment> oldSegments) {
+        final List<String> oldHTables = Lists.newArrayListWithExpectedSize(oldSegments.size());
+        for (CubeSegment segment : oldSegments) {
+            oldHTables.add(segment.getStorageLocationIdentifier());
+        }
+        return oldHTables;
+    }
+
     public List<String> getMergingHTables() {
         final List<CubeSegment> mergingSegments = ((CubeInstance) seg.getRealization()).getMergingSegments((CubeSegment) seg);
         Preconditions.checkState(mergingSegments.size() > 1, "there should be more than 2 segments to merge, target segment " + seg);
@@ -187,6 +227,18 @@ public class HBaseMRSteps extends JobBuilderSupport {
         return mergingHDFSPaths;
     }
 
+    public List<String> getOptimizeHDFSPaths() {
+        return getOldHDFSPaths(getOptimizeSegments());
+    }
+
+    public List<String> getOldHDFSPaths(final List<CubeSegment> oldSegments) {
+        final List<String> oldHDFSPaths = Lists.newArrayListWithExpectedSize(oldSegments.size());
+        for (CubeSegment oldSegment : oldSegments) {
+            oldHDFSPaths.add(getJobWorkingDir(oldSegment.getLastBuildJobID()));
+        }
+        return oldHDFSPaths;
+    }
+
     public String getHFilePath(String jobId) {
         return HBaseConnection.makeQualifiedPathInHBaseCluster(getJobWorkingDir(jobId) + "/" + seg.getRealization().getName() + "/hfile/");
     }
@@ -195,6 +247,36 @@ public class HBaseMRSteps extends JobBuilderSupport {
         return HBaseConnection.makeQualifiedPathInHBaseCluster(getJobWorkingDir(jobId) + "/" + seg.getRealization().getName() + "/rowkey_stats");
     }
 
+    public void addOptimizeGarbageCollectionSteps(DefaultChainedExecutable jobFlow) {
+        String jobId = jobFlow.getId();
+
+        List<String> toDeletePaths = new ArrayList<>();
+        toDeletePaths.add(getOptimizationRootPath(jobId));
+
+        HDFSPathGarbageCollectionStep step = new HDFSPathGarbageCollectionStep();
+        step.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION_HDFS);
+        step.setDeletePaths(toDeletePaths);
+        step.setJobId(jobId);
+
+        jobFlow.addTask(step);
+    }
+
+    public void addCheckpointGarbageCollectionSteps(DefaultChainedExecutable jobFlow) {
+        String jobId = jobFlow.getId();
+
+        jobFlow.addTask(createOptimizeGCStep());
+
+        List<String> toDeletePaths = new ArrayList<>();
+        toDeletePaths.addAll(getOptimizeHDFSPaths());
+
+        HDFSPathGarbageCollectionStep step = new HDFSPathGarbageCollectionStep();
+        step.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION_HDFS);
+        step.setDeletePaths(toDeletePaths);
+        step.setJobId(jobId);
+
+        jobFlow.addTask(step);
+    }
+
     public void addMergingGarbageCollectionSteps(DefaultChainedExecutable jobFlow) {
         String jobId = jobFlow.getId();
 


[18/18] kylin git commit: APACHE-KYLIN-2734: regard the imported hot cuboids as mandatory cuboids for cube planner phase one

Posted by li...@apache.org.
APACHE-KYLIN-2734: regard the imported hot cuboids as mandatory cuboids for cube planner phase one


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/950e3dba
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/950e3dba
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/950e3dba

Branch: refs/heads/ci-dong
Commit: 950e3dba0d5fd1a8f4df7f3c8944d7cfb3253b24
Parents: 0aec755
Author: Zhong <nj...@apache.org>
Authored: Fri Sep 1 09:42:17 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../cuboid/algorithm/CuboidRecommender.java     |  6 ++++
 .../org/apache/kylin/cube/model/CubeDesc.java   | 26 +++++++++++-----
 .../engine/mr/common/CuboidRecommenderUtil.java |  8 +++--
 .../mr/common/StatisticsDecisionUtil.java       | 32 +++++++++++++++-----
 .../mr/steps/FactDistinctColumnsMapper.java     |  9 +++++-
 5 files changed, 63 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/950e3dba/core-cube/src/main/java/org/apache/kylin/cube/cuboid/algorithm/CuboidRecommender.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/algorithm/CuboidRecommender.java b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/algorithm/CuboidRecommender.java
index 43b2318..057f7e8 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/algorithm/CuboidRecommender.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/algorithm/CuboidRecommender.java
@@ -75,6 +75,9 @@ public class CuboidRecommender {
         return instance;
     }
 
+    /**
+     * Get recommend cuboids with their row count stats with cache
+     */
     public Map<Long, Long> getRecommendCuboidList(final CuboidStats cuboidStats, final KylinConfig kylinConfig) {
         if (cuboidStats == null) {
             return null;
@@ -113,6 +116,9 @@ public class CuboidRecommender {
         return results;
     }
 
+    /**
+     * Get recommend cuboids with their row count stats without cache
+     */
     public Map<Long, Long> getRecommendCuboidList(CuboidStats cuboidStats, KylinConfig kylinConf,
             boolean ifForceRecommend) {
         long Threshold1 = 1L << kylinConf.getCubePlannerAgreedyAlgorithmAutoThreshold();

http://git-wip-us.apache.org/repos/asf/kylin/blob/950e3dba/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index 9c0a7cc..79116a8 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -189,6 +189,8 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
     @JsonInclude(JsonInclude.Include.NON_NULL)
     private List<Set<String>> mandatoryDimensionSetList = Collections.emptyList();
 
+    private Set<Long> mandatoryCuboids = Sets.newHashSet();
+
     private LinkedHashSet<TblColRef> allColumns = new LinkedHashSet<>();
     private LinkedHashSet<ColumnDesc> allColumnDescs = new LinkedHashSet<>();
     private LinkedHashSet<TblColRef> dimensionColumns = new LinkedHashSet<>();
@@ -458,6 +460,10 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
         this.mandatoryDimensionSetList = mandatoryDimensionSetList;
     }
 
+    public Set<Long> getMandatoryCuboids() {
+        return mandatoryCuboids;
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o)
@@ -651,24 +657,30 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
         initDictionaryDesc();
         amendAllColumns();
 
-        // check if mandatory dimension set list is valid
-        validateMandatoryDimensionSetList();
+        // initialize mandatory cuboids based on mandatoryDimensionSetList
+        initMandatoryCuboids();
     }
 
-    public void validateMandatoryDimensionSetList() {
-        Set<String> rowKeyColumns = Sets.newHashSet();
+    private void initMandatoryCuboids() {
+        Map<String, RowKeyColDesc> rowKeyColDescMap = Maps.newHashMap();
         for (RowKeyColDesc entry : getRowkey().getRowKeyColumns()) {
-            rowKeyColumns.add(entry.getColumn());
+            rowKeyColDescMap.put(entry.getColumn(), entry);
         }
 
         for (Set<String> mandatoryDimensionSet : this.mandatoryDimensionSetList) {
+            long cuboid = 0L;
             for (String columnName : mandatoryDimensionSet) {
-                if (!rowKeyColumns.contains(columnName)) {
-                    logger.info("Column " + columnName + " in " + mandatoryDimensionSet + " does not exist");
+                TblColRef tblColRef = model.findColumn(columnName);
+                RowKeyColDesc rowKeyColDesc = rowKeyColDescMap.get(tblColRef.getIdentity());
+                // check if mandatory dimension set list is valid
+                if (rowKeyColDesc == null) {
+                    logger.warn("Column " + columnName + " in " + mandatoryDimensionSet + " does not exist");
                     throw new IllegalStateException(
                             "Column " + columnName + " in " + mandatoryDimensionSet + " does not exist");
                 }
+                cuboid |= 1L << rowKeyColDesc.getBitIndex();
             }
+            mandatoryCuboids.add(cuboid);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/950e3dba/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidRecommenderUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidRecommenderUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidRecommenderUtil.java
index 649eeb6..05458b6 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidRecommenderUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidRecommenderUtil.java
@@ -20,6 +20,7 @@ package org.apache.kylin.engine.mr.common;
 
 import java.io.IOException;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeInstance;
@@ -53,10 +54,13 @@ public class CuboidRecommenderUtil {
             return null;
         }
 
+        Set<Long> mandatoryCuboids = segment.getCubeDesc().getMandatoryCuboids();
+
         String key = cube.getName();
         CuboidStats cuboidStats = new CuboidStats.Builder(key, baseCuboid, cubeStatsReader.getCuboidRowEstimatesHLL(),
-                cubeStatsReader.getCuboidSizeMap()).build();
-        return CuboidRecommender.getInstance().getRecommendCuboidList(cuboidStats, segment.getConfig(), false);
+                cubeStatsReader.getCuboidSizeMap()).setMandatoryCuboids(mandatoryCuboids).build();
+        return CuboidRecommender.getInstance().getRecommendCuboidList(cuboidStats, segment.getConfig(),
+                !mandatoryCuboids.isEmpty());
     }
 
     /** Trigger cube planner phase two for optimization */

http://git-wip-us.apache.org/repos/asf/kylin/blob/950e3dba/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/StatisticsDecisionUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/StatisticsDecisionUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/StatisticsDecisionUtil.java
index a5a1ba8..4efcb96 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/StatisticsDecisionUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/StatisticsDecisionUtil.java
@@ -88,15 +88,9 @@ public class StatisticsDecisionUtil {
         cubingJob.setAlgorithm(alg);
     }
 
+    // For triggering cube planner phase one
     public static void optimizeCubingPlan(CubeSegment segment) throws IOException {
-        CubeInstance cube = segment.getCubeInstance();
-        
-        if (cube.getConfig().isCubePlannerEnabled() == false)
-            return;
-        
-        List<CubeSegment> readySegments = cube.getSegments(SegmentStatusEnum.READY);
-        if (readySegments.size() == 0 || (cube.getConfig().isCubePlannerEnabledForExistingCube()
-                && readySegments.size() == 1 && (readySegments.get(0).getSegRange().equals(segment.getSegRange())))) {
+        if (isAbleToOptimizeCubingPlan(segment)) {
             logger.info("It's able to trigger cuboid planner algorithm.");
         } else {
             return;
@@ -107,8 +101,30 @@ public class StatisticsDecisionUtil {
             return;
         }
 
+        CubeInstance cube = segment.getCubeInstance();
         CubeUpdate cubeBuilder = new CubeUpdate(cube);
         cubeBuilder.setCuboids(recommendCuboidsWithStats);
         CubeManager.getInstance(cube.getConfig()).updateCube(cubeBuilder);
     }
+
+    public static boolean isAbleToOptimizeCubingPlan(CubeSegment segment) {
+        CubeInstance cube = segment.getCubeInstance();
+        if (!cube.getConfig().isCubePlannerEnabled())
+            return false;
+
+        if (cube.getSegments(SegmentStatusEnum.READY_PENDING).size() > 0) {
+            logger.info("Has read pending segments and will not enable cube planner.");
+            return false;
+        }
+        List<CubeSegment> readySegments = cube.getSegments(SegmentStatusEnum.READY);
+        List<CubeSegment> newSegments = cube.getSegments(SegmentStatusEnum.NEW);
+        if (newSegments.size() <= 1 && //
+                (readySegments.size() == 0 || //
+                        (cube.getConfig().isCubePlannerEnabledForExistingCube() && readySegments.size() == 1
+                                && readySegments.get(0).getSegRange().equals(segment.getSegRange())))) {
+            return true;
+        } else {
+            return false;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/950e3dba/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java
index e9fd3bd..ace16a5 100755
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java
@@ -29,6 +29,7 @@ import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.cube.cuboid.CuboidUtil;
 import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.StatisticsDecisionUtil;
 import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.hllc.HLLCounter;
 import org.apache.kylin.measure.hllc.RegisterType;
@@ -37,6 +38,7 @@ import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Sets;
 import com.google.common.hash.HashFunction;
 import com.google.common.hash.Hasher;
 import com.google.common.hash.Hashing;
@@ -86,7 +88,12 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB
             samplingPercentage = Integer.parseInt(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT));
             nRowKey = cubeDesc.getRowkey().getRowKeyColumns().length;
 
-            Set<Long> cuboidIdSet = cubeSeg.getCuboidScheduler().getAllCuboidIds();
+            Set<Long> cuboidIdSet = Sets.newHashSet(cubeSeg.getCuboidScheduler().getAllCuboidIds());
+            if (StatisticsDecisionUtil.isAbleToOptimizeCubingPlan(cubeSeg)) {
+                // For cube planner, for every prebuilt cuboid, its related row count stats should be calculated
+                // If the precondition for trigger cube planner phase one is satisfied, we need to calculate row count stats for mandatory cuboids.
+                cuboidIdSet.addAll(cubeSeg.getCubeDesc().getMandatoryCuboids());
+            }
             cuboidIds = cuboidIdSet.toArray(new Long[cuboidIdSet.size()]);
             allCuboidsBitSet = CuboidUtil.getCuboidBitSet(cuboidIds, nRowKey);
 


[08/18] kylin git commit: APACHE-KYLIN-2732: Introduce base cuboid as a new input for cubing job

Posted by li...@apache.org.
APACHE-KYLIN-2732: Introduce base cuboid as a new input for cubing job

Signed-off-by: Zhong <nj...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/e83a2e5d
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/e83a2e5d
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/e83a2e5d

Branch: refs/heads/ci-dong
Commit: e83a2e5dc4a33beba7d0d9108057df070df97596
Parents: 07b7f82
Author: Wang Ken <mi...@ebay.com>
Authored: Mon Aug 28 11:34:17 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Nov 23 13:31:34 2017 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/cube/CubeInstance.java     |  33 +++
 .../cube/cuboid/TreeCuboidSchedulerManager.java | 102 +++++++++
 .../inmemcubing/AbstractInMemCubeBuilder.java   |  13 +-
 .../ConsumeBlockingQueueController.java         |  84 ++++++++
 .../cube/inmemcubing/DoggedCubeBuilder.java     | 117 ++--------
 .../cube/inmemcubing/InMemCubeBuilder.java      |  87 ++------
 .../InMemCubeBuilderInputConverter.java         | 149 -------------
 .../kylin/cube/inmemcubing/InputConverter.java  |  69 ++++++
 .../cube/inmemcubing/InputConverterUnit.java    |  33 +++
 .../InputConverterUnitForBaseCuboid.java        |  49 +++++
 .../InputConverterUnitForRawData.java           | 159 ++++++++++++++
 .../RecordConsumeBlockingQueueController.java   |  91 ++++++++
 .../org/apache/kylin/gridtable/GTRecord.java    |   5 +
 .../engine/mr/common/AbstractHadoopJob.java     |  12 +-
 .../kylin/engine/mr/common/BatchConstants.java  |   1 +
 .../engine/mr/steps/InMemCuboidMapper.java      | 129 +++--------
 .../engine/mr/steps/InMemCuboidMapperBase.java  | 216 +++++++++++++++++++
 .../ITDoggedCubeBuilderStressTest.java          |   3 +-
 .../inmemcubing/ITDoggedCubeBuilderTest.java    |   3 +-
 .../inmemcubing/ITInMemCubeBuilderTest.java     |  14 +-
 20 files changed, 944 insertions(+), 425 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
index cc56727..f6eceb6 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
@@ -18,6 +18,9 @@
 
 package org.apache.kylin.cube;
 
+import static org.apache.kylin.cube.cuboid.CuboidModeEnum.CURRENT;
+import static org.apache.kylin.cube.cuboid.CuboidModeEnum.RECOMMEND;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
@@ -29,6 +32,7 @@ import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
 import org.apache.kylin.common.util.CompressionUtils;
 import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.cube.cuboid.CuboidModeEnum;
 import org.apache.kylin.cube.cuboid.CuboidScheduler;
 import org.apache.kylin.cube.cuboid.TreeCuboidScheduler;
 import org.apache.kylin.cube.model.CubeDesc;
@@ -327,6 +331,35 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
         this.createTimeUTC = createTimeUTC;
     }
 
+    public Set<Long> getCuboidsByMode(String cuboidModeName) {
+        return getCuboidsByMode(cuboidModeName == null ? null : CuboidModeEnum.getByModeName(cuboidModeName));
+    }
+
+    public Set<Long> getCuboidsByMode(CuboidModeEnum cuboidMode) {
+        if (cuboidMode == null || cuboidMode == CURRENT) {
+            return getCuboidScheduler().getAllCuboidIds();
+        }
+        Set<Long> cuboidsRecommend = getCuboidsRecommend();
+        if (cuboidsRecommend == null || cuboidMode == RECOMMEND) {
+            return cuboidsRecommend;
+        }
+        Set<Long> currentCuboids = getCuboidScheduler().getAllCuboidIds();
+        switch (cuboidMode) {
+        case RECOMMEND_EXISTING:
+            cuboidsRecommend.retainAll(currentCuboids);
+            return cuboidsRecommend;
+        case RECOMMEND_MISSING:
+            cuboidsRecommend.removeAll(currentCuboids);
+            return cuboidsRecommend;
+        case RECOMMEND_MISSING_WITH_BASE:
+            cuboidsRecommend.removeAll(currentCuboids);
+            currentCuboids.add(getCuboidScheduler().getBaseCuboidId());
+            return cuboidsRecommend;
+        default:
+            return null;
+        }
+    }
+
     public Map<Long, Long> getCuboids() {
         if (cuboidBytes == null)
             return null;

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java
new file mode 100644
index 0000000..5e8d965
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/TreeCuboidSchedulerManager.java
@@ -0,0 +1,102 @@
+/*
+ * 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.cube.cuboid;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class TreeCuboidSchedulerManager {
+    private static ConcurrentMap<String, TreeCuboidScheduler> cache = Maps.newConcurrentMap();
+
+    private class TreeCuboidSchedulerSyncListener extends Broadcaster.Listener {
+        @Override
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+            cache.clear();
+        }
+
+        @Override
+        public void onEntityChange(Broadcaster broadcaster, String entity, Broadcaster.Event event, String cacheKey)
+                throws IOException {
+            cache.remove(cacheKey);
+        }
+    }
+
+    public TreeCuboidSchedulerManager() {
+        Broadcaster.getInstance(KylinConfig.getInstanceFromEnv())
+                .registerListener(new TreeCuboidSchedulerSyncListener(), "cube");
+    }
+
+    private static TreeCuboidSchedulerManager instance = new TreeCuboidSchedulerManager();
+
+    public static TreeCuboidSchedulerManager getInstance() {
+        return instance;
+    }
+
+    /**
+     *
+     * @param cubeName
+     * @return null if the cube has no pre-built cuboids
+     */
+    public static TreeCuboidScheduler getTreeCuboidScheduler(String cubeName) {
+        TreeCuboidScheduler result = cache.get(cubeName);
+        if (result == null) {
+            CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cubeInstance = cubeManager.getCube(cubeName);
+            if (cubeInstance == null) {
+                return null;
+            }
+            TreeCuboidScheduler treeCuboidScheduler = getTreeCuboidScheduler(cubeInstance.getDescriptor(),
+                    cubeManager.getCube(cubeName).getCuboids());
+            if (treeCuboidScheduler == null) {
+                return null;
+            }
+            cache.put(cubeName, treeCuboidScheduler);
+            result = treeCuboidScheduler;
+        }
+        return result;
+    }
+
+    public static TreeCuboidScheduler getTreeCuboidScheduler(CubeDesc cubeDesc, Map<Long, Long> cuboidsWithRowCnt) {
+        if (cuboidsWithRowCnt == null || cuboidsWithRowCnt.isEmpty()) {
+            return null;
+        }
+        return getTreeCuboidScheduler(cubeDesc, Lists.newArrayList(cuboidsWithRowCnt.keySet()), cuboidsWithRowCnt);
+    }
+
+    public static TreeCuboidScheduler getTreeCuboidScheduler(CubeDesc cubeDesc, List<Long> cuboidIds,
+            Map<Long, Long> cuboidsWithRowCnt) {
+        if (cuboidIds == null || cuboidsWithRowCnt == null) {
+            return null;
+        }
+        TreeCuboidScheduler treeCuboidScheduler = new TreeCuboidScheduler(cubeDesc, cuboidIds,
+                new TreeCuboidScheduler.CuboidCostComparator(cuboidsWithRowCnt));
+        return treeCuboidScheduler;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
index 952926c..df1fa7a 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
@@ -19,7 +19,6 @@
 package org.apache.kylin.cube.inmemcubing;
 
 import java.io.IOException;
-import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
 
@@ -78,12 +77,17 @@ abstract public class AbstractInMemCubeBuilder {
         return this.reserveMemoryMB;
     }
 
-    public Runnable buildAsRunnable(final BlockingQueue<List<String>> input, final ICuboidWriter output) {
+    public Runnable buildAsRunnable(final BlockingQueue<String[]> input, final ICuboidWriter output) {
+        return buildAsRunnable(input, new InputConverterUnitForRawData(cubeDesc, flatDesc, dictionaryMap), output);
+    }
+
+    public <T> Runnable buildAsRunnable(final BlockingQueue<T> input, final InputConverterUnit<T> inputConverterUnit,
+            final ICuboidWriter output) {
         return new Runnable() {
             @Override
             public void run() {
                 try {
-                    build(input, output);
+                    build(input, inputConverterUnit, output);
                 } catch (IOException e) {
                     throw new RuntimeException(e);
                 }
@@ -91,7 +95,8 @@ abstract public class AbstractInMemCubeBuilder {
         };
     }
 
-    abstract public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException;
+    abstract public <T> void build(BlockingQueue<T> input, InputConverterUnit<T> inputConverterUnit,
+            ICuboidWriter output) throws IOException;
 
     protected void outputCuboid(long cuboidId, GridTable gridTable, ICuboidWriter output) throws IOException {
         long startTime = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConsumeBlockingQueueController.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConsumeBlockingQueueController.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConsumeBlockingQueueController.java
new file mode 100644
index 0000000..a9e55f7
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConsumeBlockingQueueController.java
@@ -0,0 +1,84 @@
+/*
+ * 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.cube.inmemcubing;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.Lists;
+
+public class ConsumeBlockingQueueController<T> implements Iterator<T> {
+    public final static int DEFAULT_BATCH_SIZE = 1000;
+
+    private volatile boolean hasException = false;
+    private final BlockingQueue<T> input;
+    private final int batchSize;
+    private final List<T> batchBuffer;
+    private Iterator<T> internalIT;
+
+    private AtomicInteger outputRowCount = new AtomicInteger();
+
+    public ConsumeBlockingQueueController(BlockingQueue<T> input, int batchSize) {
+        this.input = input;
+        this.batchSize = batchSize;
+        this.batchBuffer = Lists.newArrayListWithExpectedSize(batchSize);
+        this.internalIT = batchBuffer.iterator();
+    }
+
+    @Override
+    public boolean hasNext() {
+        if (hasException) {
+            return false;
+        }
+        if (internalIT.hasNext()) {
+            return true;
+        } else {
+            batchBuffer.clear();
+            try {
+                batchBuffer.add(input.take());
+                outputRowCount.incrementAndGet();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+            outputRowCount.addAndGet(input.drainTo(batchBuffer, batchSize - 1));
+            internalIT = batchBuffer.iterator();
+        }
+        return true;
+    }
+
+    @Override
+    public T next() {
+        return internalIT.next();
+    }
+
+    @Override
+    public void remove() {
+        throw new UnsupportedOperationException();
+    }
+
+    public void findException() {
+        hasException = true;
+    }
+
+    public long getOutputRowCount() {
+        return outputRowCount.get();
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java
index dd92a2b..ccd7137 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java
@@ -20,21 +20,17 @@ package org.apache.kylin.cube.inmemcubing;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.PriorityQueue;
-import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentNavigableMap;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.ImmutableBitSet;
-import org.apache.kylin.common.util.MemoryBudgetController;
 import org.apache.kylin.cube.cuboid.CuboidScheduler;
 import org.apache.kylin.gridtable.GTRecord;
 import org.apache.kylin.gridtable.GTScanRequestBuilder;
@@ -55,7 +51,7 @@ public class DoggedCubeBuilder extends AbstractInMemCubeBuilder {
     private static Logger logger = LoggerFactory.getLogger(DoggedCubeBuilder.class);
 
     private int splitRowThreshold = Integer.MAX_VALUE;
-    private int unitRows = 1000;
+    private int unitRows = ConsumeBlockingQueueController.DEFAULT_BATCH_SIZE;
 
     public DoggedCubeBuilder(CuboidScheduler cuboidScheduler, IJoinedFlatTableDesc flatDesc,
             Map<TblColRef, Dictionary<String>> dictionaryMap) {
@@ -72,8 +68,9 @@ public class DoggedCubeBuilder extends AbstractInMemCubeBuilder {
     }
 
     @Override
-    public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException {
-        new BuildOnce().build(input, output);
+    public <T> void build(BlockingQueue<T> input, InputConverterUnit<T> inputConverterUnit, ICuboidWriter output)
+            throws IOException {
+        new BuildOnce().build(input, inputConverterUnit, output);
     }
 
     private class BuildOnce {
@@ -81,7 +78,8 @@ public class DoggedCubeBuilder extends AbstractInMemCubeBuilder {
         BuildOnce() {
         }
 
-        public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException {
+        public <T> void build(BlockingQueue<T> input, InputConverterUnit<T> inputConverterUnit, ICuboidWriter output)
+                throws IOException {
             final List<SplitThread> splits = new ArrayList<SplitThread>();
             final Merger merger = new Merger();
 
@@ -89,32 +87,23 @@ public class DoggedCubeBuilder extends AbstractInMemCubeBuilder {
             logger.info("Dogged Cube Build start");
 
             try {
-                SplitThread last = null;
-                boolean eof = false;
+                while (true) {
+                    SplitThread last = new SplitThread(splits.size() + 1, RecordConsumeBlockingQueueController
+                            .getQueueController(inputConverterUnit, input, unitRows));
+                    splits.add(last);
 
-                while (!eof) {
+                    last.start();
+                    logger.info("Split #" + splits.size() + " kickoff");
 
-                    if (last != null && shouldCutSplit(splits)) {
-                        cutSplit(last);
-                        last = null;
-                    }
+                    // Build splits sequentially
+                    last.join();
 
                     checkException(splits);
-
-                    if (last == null) {
-                        last = new SplitThread();
-                        splits.add(last);
-                        last.start();
-                        logger.info("Split #" + splits.size() + " kickoff");
+                    if (last.inputController.ifEnd()) {
+                        break;
                     }
-
-                    eof = feedSomeInput(input, last, unitRows);
                 }
 
-                for (SplitThread split : splits) {
-                    split.join();
-                }
-                checkException(splits);
                 logger.info("Dogged Cube Build splits complete, took " + (System.currentTimeMillis() - start) + " ms");
 
                 merger.mergeAndOutput(splits, output);
@@ -202,81 +191,18 @@ public class DoggedCubeBuilder extends AbstractInMemCubeBuilder {
                 throw new IOException(errors.size() + " exceptions during in-mem cube build, cause set to the first, check log for more", errors.get(0));
             }
         }
-
-        private boolean feedSomeInput(BlockingQueue<List<String>> input, SplitThread split, int n) {
-            try {
-                int i = 0;
-                while (i < n) {
-                    List<String> record = input.take();
-                    i++;
-
-                    while (split.inputQueue.offer(record, 1, TimeUnit.SECONDS) == false) {
-                        if (split.exception != null)
-                            return true; // got some error
-                    }
-                    split.inputRowCount++;
-
-                    if (record == null || record.isEmpty()) {
-                        return true;
-                    }
-                }
-                return false;
-
-            } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                throw new RuntimeException(e);
-            }
-        }
-
-        private void cutSplit(SplitThread last) {
-            try {
-                // signal the end of input
-                while (last.isAlive()) {
-                    if (last.inputQueue.offer(Collections.<String> emptyList())) {
-                        break;
-                    }
-                    Thread.sleep(1000);
-                }
-
-                // wait cuboid build done
-                last.join();
-
-            } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                throw new RuntimeException(e);
-            }
-        }
-
-        private boolean shouldCutSplit(List<SplitThread> splits) {
-            int systemAvailMB = MemoryBudgetController.getSystemAvailMB();
-            int nSplit = splits.size();
-            long splitRowCount = nSplit == 0 ? 0 : splits.get(nSplit - 1).inputRowCount;
-
-            logger.info(splitRowCount + " records went into split #" + nSplit + "; " + systemAvailMB + " MB left, " + reserveMemoryMB + " MB threshold");
-
-            if (splitRowCount >= splitRowThreshold) {
-                logger.info("Split cut due to hitting splitRowThreshold " + splitRowThreshold);
-                return true;
-            }
-
-            if (systemAvailMB <= reserveMemoryMB * 1.5) {
-                logger.info("Split cut due to hitting memory threshold, system avail " + systemAvailMB + " MB <= reserve " + reserveMemoryMB + "*1.5 MB");
-                return true;
-            }
-
-            return false;
-        }
     }
 
     private class SplitThread extends Thread {
-        final BlockingQueue<List<String>> inputQueue = new ArrayBlockingQueue<List<String>>(16);
+        final RecordConsumeBlockingQueueController<?> inputController;
         final InMemCubeBuilder builder;
 
         ConcurrentNavigableMap<Long, CuboidResult> buildResult;
-        long inputRowCount = 0;
         RuntimeException exception;
 
-        public SplitThread() {
+        public SplitThread(final int num, final RecordConsumeBlockingQueueController<?> inputController) {
+            super("SplitThread" + num);
+            this.inputController = inputController;
             this.builder = new InMemCubeBuilder(cuboidScheduler, flatDesc, dictionaryMap);
             this.builder.setConcurrentThreads(taskThreadCount);
             this.builder.setReserveMemoryMB(reserveMemoryMB);
@@ -285,12 +211,13 @@ public class DoggedCubeBuilder extends AbstractInMemCubeBuilder {
         @Override
         public void run() {
             try {
-                buildResult = builder.build(inputQueue);
+                buildResult = builder.build(inputController);
             } catch (Exception e) {
                 if (e instanceof RuntimeException)
                     this.exception = (RuntimeException) e;
                 else
                     this.exception = new RuntimeException(e);
+                inputController.findException();
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
index 684c26b..f63b53f 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
@@ -87,6 +87,7 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
     private CuboidResult baseResult;
     private Object[] totalSumForSanityCheck;
     private ICuboidCollector resultCollector;
+    private boolean ifBaseCuboidCollected = true;
 
     public InMemCubeBuilder(CuboidScheduler cuboidScheduler, IJoinedFlatTableDesc flatDesc,
             Map<TblColRef, Dictionary<String>> dictionaryMap) {
@@ -121,8 +122,10 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
     }
 
     @Override
-    public void build(BlockingQueue<List<String>> input, ICuboidWriter output) throws IOException {
-        ConcurrentNavigableMap<Long, CuboidResult> result = build(input);
+    public <T> void build(BlockingQueue<T> input, InputConverterUnit<T> inputConverterUnit, ICuboidWriter output)
+            throws IOException {
+        ConcurrentNavigableMap<Long, CuboidResult> result = build(
+                RecordConsumeBlockingQueueController.getQueueController(inputConverterUnit, input));
         try {
             for (CuboidResult cuboidResult : result.values()) {
                 outputCuboid(cuboidResult.cuboidId, cuboidResult.table, output);
@@ -133,7 +136,11 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
         }
     }
 
-    public ConcurrentNavigableMap<Long, CuboidResult> build(BlockingQueue<List<String>> input) throws IOException {
+    public <T> ConcurrentNavigableMap<Long, CuboidResult> build(RecordConsumeBlockingQueueController<T> input)
+            throws IOException {
+        if (input.inputConverterUnit instanceof InputConverterUnitForBaseCuboid) {
+            ifBaseCuboidCollected = false;
+        }
         final ConcurrentNavigableMap<Long, CuboidResult> result = new ConcurrentSkipListMap<Long, CuboidResult>();
         build(input, new ICuboidCollector() {
             @Override
@@ -150,7 +157,8 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
         void collect(CuboidResult result);
     }
 
-    private void build(BlockingQueue<List<String>> input, ICuboidCollector collector) throws IOException {
+    private <T> void build(RecordConsumeBlockingQueueController<T> input, ICuboidCollector collector)
+            throws IOException {
         long startTime = System.currentTimeMillis();
         logger.info("In Mem Cube Build start, " + cubeDesc.getName());
 
@@ -326,7 +334,7 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
         memBudget = new MemoryBudgetController(budget);
     }
 
-    private CuboidResult createBaseCuboid(BlockingQueue<List<String>> input) throws IOException {
+    private <T> CuboidResult createBaseCuboid(RecordConsumeBlockingQueueController<T> input) throws IOException {
         long startTime = System.currentTimeMillis();
         logger.info("Calculating base cuboid " + baseCuboidId);
 
@@ -356,10 +364,15 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
         int mbEstimateBaseAggrCache = (int) (aggregationScanner.getEstimateSizeOfAggrCache() / MemoryBudgetController.ONE_MB);
         logger.info("Wild estimate of base aggr cache is " + mbEstimateBaseAggrCache + " MB");
 
-        return updateCuboidResult(baseCuboidId, baseCuboid, count, timeSpent, 0);
+        return updateCuboidResult(baseCuboidId, baseCuboid, count, timeSpent, 0, ifBaseCuboidCollected);
     }
 
     private CuboidResult updateCuboidResult(long cuboidId, GridTable table, int nRows, long timeSpent, int aggrCacheMB) {
+        return updateCuboidResult(cuboidId, table, nRows, timeSpent, aggrCacheMB, true);
+    }
+
+    private CuboidResult updateCuboidResult(long cuboidId, GridTable table, int nRows, long timeSpent, int aggrCacheMB,
+            boolean ifCollect) {
         if (aggrCacheMB <= 0 && baseResult != null) {
             aggrCacheMB = (int) Math.round(//
                     (DERIVE_AGGR_CACHE_CONSTANT_FACTOR + DERIVE_AGGR_CACHE_VARIABLE_FACTOR * nRows / baseResult.nRows) //
@@ -369,7 +382,9 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
         CuboidResult result = new CuboidResult(cuboidId, table, nRows, timeSpent, aggrCacheMB);
         taskCuboidCompleted.incrementAndGet();
 
-        resultCollector.collect(result);
+        if (ifCollect) {
+            resultCollector.collect(result);
+        }
         return result;
     }
 
@@ -508,62 +523,4 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
             return comp < 0 ? -1 : (comp > 0 ? 1 : 0);
         }
     }
-
-    // ============================================================================
-
-    private class InputConverter implements IGTScanner {
-        GTInfo info;
-        GTRecord record;
-        BlockingQueue<List<String>> input;
-        final InMemCubeBuilderInputConverter inMemCubeBuilderInputConverter;
-
-        public InputConverter(GTInfo info, BlockingQueue<List<String>> input) {
-            this.info = info;
-            this.input = input;
-            this.record = new GTRecord(info);
-            this.inMemCubeBuilderInputConverter = new InMemCubeBuilderInputConverter(cubeDesc, flatDesc, dictionaryMap, info);
-        }
-
-        @Override
-        public Iterator<GTRecord> iterator() {
-            return new Iterator<GTRecord>() {
-
-                List<String> currentObject = null;
-
-                @Override
-                public boolean hasNext() {
-                    try {
-                        currentObject = input.take();
-                    } catch (InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                        throw new RuntimeException(e);
-                    }
-                    return currentObject != null && currentObject.size() > 0;
-                }
-
-                @Override
-                public GTRecord next() {
-                    if (currentObject.size() == 0)
-                        throw new IllegalStateException();
-
-                    inMemCubeBuilderInputConverter.convert(currentObject, record);
-                    return record;
-                }
-
-                @Override
-                public void remove() {
-                    throw new UnsupportedOperationException();
-                }
-            };
-        }
-
-        @Override
-        public void close() throws IOException {
-        }
-
-        @Override
-        public GTInfo getInfo() {
-            return info;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
deleted file mode 100644
index 6dd20d8..0000000
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilderInputConverter.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-package org.apache.kylin.cube.inmemcubing;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.CubeJoinedFlatTableEnrich;
-import org.apache.kylin.gridtable.GTInfo;
-import org.apache.kylin.gridtable.GTRecord;
-import org.apache.kylin.measure.MeasureIngester;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.ParameterDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-
-/**
- */
-public class InMemCubeBuilderInputConverter {
-
-    @SuppressWarnings("unused")
-    private static final Logger logger = LoggerFactory.getLogger(InMemCubeBuilderInputConverter.class);
-    
-    public static final byte[] HIVE_NULL = Bytes.toBytes("\\N");
-
-    private final CubeJoinedFlatTableEnrich flatDesc;
-    private final MeasureDesc[] measureDescs;
-    private final MeasureIngester<?>[] measureIngesters;
-    private final int measureCount;
-    private final Map<TblColRef, Dictionary<String>> dictionaryMap;
-    private final GTInfo gtInfo;
-    protected List<byte[]> nullBytes;
-
-    public InMemCubeBuilderInputConverter(CubeDesc cubeDesc, IJoinedFlatTableDesc flatDesc, Map<TblColRef, Dictionary<String>> dictionaryMap, GTInfo gtInfo) {
-        this.gtInfo = gtInfo;
-        this.flatDesc = new CubeJoinedFlatTableEnrich(flatDesc, cubeDesc);
-        this.measureCount = cubeDesc.getMeasures().size();
-        this.measureDescs = cubeDesc.getMeasures().toArray(new MeasureDesc[measureCount]);
-        this.measureIngesters = MeasureIngester.create(cubeDesc.getMeasures());
-        this.dictionaryMap = dictionaryMap;
-        initNullBytes(cubeDesc);
-    }
-
-    public final GTRecord convert(List<String> row) {
-        final GTRecord record = new GTRecord(gtInfo);
-        convert(row, record);
-        return record;
-    }
-
-    public final void convert(List<String> row, GTRecord record) {
-        Object[] dimensions = buildKey(row);
-        Object[] metricsValues = buildValue(row);
-        Object[] recordValues = new Object[dimensions.length + metricsValues.length];
-        System.arraycopy(dimensions, 0, recordValues, 0, dimensions.length);
-        System.arraycopy(metricsValues, 0, recordValues, dimensions.length, metricsValues.length);
-        record.setValues(recordValues);
-    }
-
-    private Object[] buildKey(List<String> row) {
-        int keySize = flatDesc.getRowKeyColumnIndexes().length;
-        Object[] key = new Object[keySize];
-
-        for (int i = 0; i < keySize; i++) {
-            key[i] = row.get(flatDesc.getRowKeyColumnIndexes()[i]);
-            if (key[i] != null && isNull(Bytes.toBytes((String) key[i]))) {
-                key[i] = null;
-            }
-        }
-
-        return key;
-    }
-
-    private Object[] buildValue(List<String> row) {
-        Object[] values = new Object[measureCount];
-        for (int i = 0; i < measureCount; i++) {
-            values[i] = buildValueOf(i, row);
-        }
-        return values;
-    }
-
-    private Object buildValueOf(int idxOfMeasure, List<String> row) {
-        MeasureDesc measure = measureDescs[idxOfMeasure];
-        FunctionDesc function = measure.getFunction();
-        int[] colIdxOnFlatTable = flatDesc.getMeasureColumnIndexes()[idxOfMeasure];
-
-        int paramCount = function.getParameterCount();
-        String[] inputToMeasure = new String[paramCount];
-
-        // pick up parameter values
-        ParameterDesc param = function.getParameter();
-        int paramColIdx = 0; // index among parameters of column type
-        for (int i = 0; i < paramCount; i++, param = param.getNextParameter()) {
-            String value;
-            if (function.isCount()) {
-                value = "1";
-            } else if (param.isColumnType()) {
-                value = row.get(colIdxOnFlatTable[paramColIdx++]);
-            } else {
-                value = param.getValue();
-            }
-            inputToMeasure[i] = value;
-        }
-
-        return measureIngesters[idxOfMeasure].valueOf(inputToMeasure, measure, dictionaryMap);
-    }
-
-    private void initNullBytes(CubeDesc cubeDesc) {
-        nullBytes = Lists.newArrayList();
-        nullBytes.add(HIVE_NULL);
-        String[] nullStrings = cubeDesc.getNullStrings();
-        if (nullStrings != null) {
-            for (String s : nullStrings) {
-                nullBytes.add(Bytes.toBytes(s));
-            }
-        }
-    }
-
-    private boolean isNull(byte[] v) {
-        for (byte[] nullByte : nullBytes) {
-            if (Bytes.equals(v, nullByte))
-                return true;
-        }
-        return false;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverter.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverter.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverter.java
new file mode 100644
index 0000000..664f784
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverter.java
@@ -0,0 +1,69 @@
+/*
+ * 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.cube.inmemcubing;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.IGTScanner;
+
+public class InputConverter<T> implements IGTScanner {
+    private GTInfo info;
+    private GTRecord record;
+    private RecordConsumeBlockingQueueController<T> inputController;
+
+    public InputConverter(GTInfo info, RecordConsumeBlockingQueueController<T> inputController) {
+        this.info = info;
+        this.inputController = inputController;
+        this.record = new GTRecord(info);
+    }
+
+    @Override
+    public Iterator<GTRecord> iterator() {
+        return new Iterator<GTRecord>() {
+
+            @Override
+            public boolean hasNext() {
+                return inputController.hasNext();
+            }
+
+            @Override
+            public GTRecord next() {
+                inputController.inputConverterUnit.convert(inputController.next(), record);
+                return record;
+            }
+
+            @Override
+            public void remove() {
+                throw new UnsupportedOperationException();
+            }
+        };
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+    @Override
+    public GTInfo getInfo() {
+        return info;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnit.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnit.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnit.java
new file mode 100644
index 0000000..fe32937
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnit.java
@@ -0,0 +1,33 @@
+/*
+ * 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.cube.inmemcubing;
+
+import org.apache.kylin.gridtable.GTRecord;
+
+public interface InputConverterUnit<T> {
+    public void convert(T currentObject, GTRecord record);
+
+    public boolean ifEnd(T currentObject);
+
+    public boolean ifCut(T currentObject);
+
+    public T getEmptyUnit();
+
+    public T getCutUnit();
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForBaseCuboid.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForBaseCuboid.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForBaseCuboid.java
new file mode 100644
index 0000000..9110a87
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForBaseCuboid.java
@@ -0,0 +1,49 @@
+/*
+ * 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.cube.inmemcubing;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.gridtable.GTRecord;
+
+public class InputConverterUnitForBaseCuboid implements InputConverterUnit<ByteArray> {
+
+    public static final ByteArray EMPTY_ROW = new ByteArray();
+    public static final ByteArray CUT_ROW = new ByteArray(0);
+
+    public void convert(ByteArray currentObject, GTRecord record) {
+        record.loadColumns(currentObject.asBuffer());
+    }
+
+    public boolean ifEnd(ByteArray currentObject) {
+        return currentObject == EMPTY_ROW;
+    }
+
+    public ByteArray getEmptyUnit() {
+        return EMPTY_ROW;
+    }
+
+    public ByteArray getCutUnit() {
+        return CUT_ROW;
+    }
+
+    @Override
+    public boolean ifCut(ByteArray currentObject) {
+        return currentObject == CUT_ROW;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForRawData.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForRawData.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForRawData.java
new file mode 100644
index 0000000..f6548b2
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InputConverterUnitForRawData.java
@@ -0,0 +1,159 @@
+/*
+ * 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.cube.inmemcubing;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableEnrich;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ */
+public class InputConverterUnitForRawData implements InputConverterUnit<String[]> {
+
+    @SuppressWarnings("unused")
+    private static final Logger logger = LoggerFactory.getLogger(InputConverterUnitForRawData.class);
+    
+    public static final byte[] HIVE_NULL = Bytes.toBytes("\\N");
+    public static final String[] EMPTY_ROW = new String[0];
+    public static final String[] CUT_ROW = { "" };
+
+    private final CubeJoinedFlatTableEnrich flatDesc;
+    private final MeasureDesc[] measureDescs;
+    private final MeasureIngester<?>[] measureIngesters;
+    private final int measureCount;
+    private final Map<TblColRef, Dictionary<String>> dictionaryMap;
+    protected List<byte[]> nullBytes;
+
+    public InputConverterUnitForRawData(CubeDesc cubeDesc, IJoinedFlatTableDesc flatDesc,
+            Map<TblColRef, Dictionary<String>> dictionaryMap) {
+        this.flatDesc = new CubeJoinedFlatTableEnrich(flatDesc, cubeDesc);
+        this.measureCount = cubeDesc.getMeasures().size();
+        this.measureDescs = cubeDesc.getMeasures().toArray(new MeasureDesc[measureCount]);
+        this.measureIngesters = MeasureIngester.create(cubeDesc.getMeasures());
+        this.dictionaryMap = dictionaryMap;
+        initNullBytes(cubeDesc);
+    }
+
+    public final void convert(String[] row, GTRecord record) {
+        Object[] dimensions = buildKey(row);
+        Object[] metricsValues = buildValue(row);
+        Object[] recordValues = new Object[dimensions.length + metricsValues.length];
+        System.arraycopy(dimensions, 0, recordValues, 0, dimensions.length);
+        System.arraycopy(metricsValues, 0, recordValues, dimensions.length, metricsValues.length);
+        record.setValues(recordValues);
+    }
+
+    public boolean ifEnd(String[] currentObject) {
+        return currentObject == EMPTY_ROW;
+    }
+
+    public boolean ifCut(String[] currentObject) {
+        return currentObject == CUT_ROW;
+    }
+
+    public String[] getEmptyUnit() {
+        return EMPTY_ROW;
+    }
+
+    public String[] getCutUnit() {
+        return CUT_ROW;
+    }
+
+    private Object[] buildKey(String[] row) {
+        int keySize = flatDesc.getRowKeyColumnIndexes().length;
+        Object[] key = new Object[keySize];
+
+        for (int i = 0; i < keySize; i++) {
+            key[i] = row[flatDesc.getRowKeyColumnIndexes()[i]];
+            if (key[i] != null && isNull(Bytes.toBytes((String) key[i]))) {
+                key[i] = null;
+            }
+        }
+
+        return key;
+    }
+
+    private Object[] buildValue(String[] row) {
+        Object[] values = new Object[measureCount];
+        for (int i = 0; i < measureCount; i++) {
+            values[i] = buildValueOf(i, row);
+        }
+        return values;
+    }
+
+    private Object buildValueOf(int idxOfMeasure, String[] row) {
+        MeasureDesc measure = measureDescs[idxOfMeasure];
+        FunctionDesc function = measure.getFunction();
+        int[] colIdxOnFlatTable = flatDesc.getMeasureColumnIndexes()[idxOfMeasure];
+
+        int paramCount = function.getParameterCount();
+        String[] inputToMeasure = new String[paramCount];
+
+        // pick up parameter values
+        ParameterDesc param = function.getParameter();
+        int paramColIdx = 0; // index among parameters of column type
+        for (int i = 0; i < paramCount; i++, param = param.getNextParameter()) {
+            String value;
+            if (function.isCount()) {
+                value = "1";
+            } else if (param.isColumnType()) {
+                value = row[colIdxOnFlatTable[paramColIdx++]];
+            } else {
+                value = param.getValue();
+            }
+            inputToMeasure[i] = value;
+        }
+
+        return measureIngesters[idxOfMeasure].valueOf(inputToMeasure, measure, dictionaryMap);
+    }
+
+    private void initNullBytes(CubeDesc cubeDesc) {
+        nullBytes = Lists.newArrayList();
+        nullBytes.add(HIVE_NULL);
+        String[] nullStrings = cubeDesc.getNullStrings();
+        if (nullStrings != null) {
+            for (String s : nullStrings) {
+                nullBytes.add(Bytes.toBytes(s));
+            }
+        }
+    }
+
+    private boolean isNull(byte[] v) {
+        for (byte[] nullByte : nullBytes) {
+            if (Bytes.equals(v, nullByte))
+                return true;
+        }
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/RecordConsumeBlockingQueueController.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/RecordConsumeBlockingQueueController.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/RecordConsumeBlockingQueueController.java
new file mode 100644
index 0000000..49cbe1f
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/RecordConsumeBlockingQueueController.java
@@ -0,0 +1,91 @@
+/*
+ * 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.cube.inmemcubing;
+
+import java.util.concurrent.BlockingQueue;
+
+public class RecordConsumeBlockingQueueController<T> extends ConsumeBlockingQueueController<T> {
+
+    public final InputConverterUnit<T> inputConverterUnit;
+
+    private RecordConsumeBlockingQueueController(InputConverterUnit<T> inputConverterUnit, BlockingQueue<T> input, int batchSize) {
+        super(input, batchSize);
+        this.inputConverterUnit = inputConverterUnit;
+    }
+   
+    private T currentObject = null;
+    private volatile boolean ifEnd = false;
+    private volatile boolean cut = false;
+    private long outputRowCountCut = 0L;
+
+    @Override
+    public boolean hasNext() {
+        if (currentObject != null) {
+            return hasNext(currentObject);
+        }
+        if (!super.hasNext()) {
+            return false;
+        }
+        currentObject = super.next();
+        return hasNext(currentObject);
+    }
+
+    @Override
+    public T next() {
+        if (ifEnd())
+            throw new IllegalStateException();
+
+        T result = currentObject;
+        currentObject = null;
+        return result;
+    }
+
+    public boolean ifEnd() {
+        return ifEnd;
+    }
+
+    private boolean hasNext(T object) {
+        if (inputConverterUnit.ifEnd(object)) {
+            ifEnd = true;
+            return false;
+        }else if(cut){
+            return false;
+        }else if(inputConverterUnit.ifCut(object)){
+            return false;
+        }
+        return true;
+    }
+    
+    public static <T> RecordConsumeBlockingQueueController<T> getQueueController(InputConverterUnit<T> inputConverterUnit, BlockingQueue<T> input){
+        return new RecordConsumeBlockingQueueController<>(inputConverterUnit, input, DEFAULT_BATCH_SIZE);
+    }
+    
+    public static <T> RecordConsumeBlockingQueueController<T> getQueueController(InputConverterUnit<T> inputConverterUnit, BlockingQueue<T> input, int batchSize){
+        return new RecordConsumeBlockingQueueController<>(inputConverterUnit, input, batchSize);
+    }
+
+    public void forceCutPipe() {
+        cut = true;
+        outputRowCountCut = getOutputRowCount();
+    }
+
+    public long getOutputRowCountAfterCut() {
+        return getOutputRowCount() - outputRowCountCut;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java
index ba3b1c4..36bd095 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java
@@ -251,6 +251,11 @@ public class GTRecord implements Comparable<GTRecord> {
         loadColumns(info.colBlocks[c], buf);
     }
 
+    /** change pointers to point to data in given buffer, UNLIKE deserialize */
+    public void loadColumns(ByteBuffer buf) {
+        loadColumns(info.colAll, buf);
+    }
+
     /**
      * Change pointers to point to data in given buffer, UNLIKE deserialize
      * @param selectedCols positions of column to load

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
index 1756251..fd212be 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
@@ -506,9 +506,19 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
     }
 
     protected void attachSegmentMetadataWithDict(CubeSegment segment, Configuration conf) throws IOException {
+        attachSegmentMetadata(segment, conf, true, true);
+    }
+
+    protected void attachSegmentMetadata(CubeSegment segment, Configuration conf, boolean ifDictIncluded,
+            boolean ifStatsIncluded) throws IOException {
         Set<String> dumpList = new LinkedHashSet<>();
         dumpList.addAll(JobRelatedMetaUtil.collectCubeMetadata(segment.getCubeInstance()));
-        dumpList.addAll(segment.getDictionaryPaths());
+        if (ifDictIncluded) {
+            dumpList.addAll(segment.getDictionaryPaths());
+        }
+        if (ifStatsIncluded) {
+            dumpList.add(segment.getStatisticsResourcePath());
+        }
         dumpKylinPropsAndMetadata(segment.getProject(), dumpList, segment.getConfig(), conf);
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
index 25a67f9..aaf2654 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
@@ -29,6 +29,7 @@ public interface BatchConstants {
      * ConFiGuration entry names for MR jobs
      */
 
+    String CFG_CUBOID_MODE = "cuboid.mode";
     String CFG_CUBE_NAME = "cube.name";
     String CFG_CUBE_SEGMENT_NAME = "cube.segment.name";
     String CFG_CUBE_SEGMENT_ID = "cube.segment.id";

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
index a7b3923..859e126 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
@@ -19,133 +19,60 @@
 package org.apache.kylin.engine.mr.steps;
 
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.common.util.MemoryBudgetController;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.inmemcubing.AbstractInMemCubeBuilder;
 import org.apache.kylin.cube.inmemcubing.DoggedCubeBuilder;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.engine.EngineFactory;
+import org.apache.kylin.cube.inmemcubing.InputConverterUnit;
+import org.apache.kylin.cube.inmemcubing.InputConverterUnitForRawData;
 import org.apache.kylin.engine.mr.ByteArrayWritable;
-import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat;
-import org.apache.kylin.engine.mr.KylinMapper;
+import org.apache.kylin.engine.mr.IMRInput;
 import org.apache.kylin.engine.mr.MRUtil;
-import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
-import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Maps;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-/**
- */
-public class InMemCuboidMapper<KEYIN> extends KylinMapper<KEYIN, Object, ByteArrayWritable, ByteArrayWritable> {
-
-    private static final Logger logger = LoggerFactory.getLogger(InMemCuboidMapper.class);
+public class InMemCuboidMapper<KEYIN>
+        extends InMemCuboidMapperBase<KEYIN, Object, ByteArrayWritable, ByteArrayWritable, String[]> {
 
-    private CubeInstance cube;
-    private CubeDesc cubeDesc;
-    private CubeSegment cubeSegment;
-    private IMRTableInputFormat flatTableInputFormat;
 
-    private BlockingQueue<List<String>> queue = new ArrayBlockingQueue<List<String>>(64);
-    private Future<?> future;
+    private IMRInput.IMRTableInputFormat flatTableInputFormat;
 
     @Override
     protected void doSetup(Context context) throws IOException {
-        super.bindCurrentConfiguration(context.getConfiguration());
+        super.doSetup(context);
 
-        Configuration conf = context.getConfiguration();
-
-        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-        String cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
-        cube = CubeManager.getInstance(config).getCube(cubeName);
-        cubeDesc = cube.getDescriptor();
-        String segmentID = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_ID);
-        cubeSegment = cube.getSegmentById(segmentID);
         flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSegment).getFlatTableInputFormat();
-        IJoinedFlatTableDesc flatDesc = EngineFactory.getJoinedFlatTableDesc(cubeSegment);
-
-        Map<TblColRef, Dictionary<String>> dictionaryMap = Maps.newHashMap();
-
-        // dictionary
-        for (TblColRef col : cubeDesc.getAllColumnsHaveDictionary()) {
-            Dictionary<?> dict = cubeSegment.getDictionary(col);
-            if (dict == null) {
-                logger.warn("Dictionary for " + col + " was not found.");
-            }
-
-            dictionaryMap.put(col, cubeSegment.getDictionary(col));
-        }
-
-        int taskCount = config.getCubeAlgorithmInMemConcurrentThreads();
-        DoggedCubeBuilder cubeBuilder = new DoggedCubeBuilder(cubeSegment.getCuboidScheduler(), flatDesc,
-                dictionaryMap);
-        cubeBuilder.setReserveMemoryMB(calculateReserveMB(context.getConfiguration()));
-        cubeBuilder.setConcurrentThreads(taskCount);
-
-        ExecutorService executorService = Executors.newSingleThreadExecutor();
-        future = executorService.submit(cubeBuilder.buildAsRunnable(queue, new MapContextGTRecordWriter(context, cubeDesc, cubeSegment)));
-
     }
 
-    private int calculateReserveMB(Configuration configuration) {
-        int sysAvailMB = MemoryBudgetController.getSystemAvailMB();
-        int mrReserve = configuration.getInt("mapreduce.task.io.sort.mb", 100);
-        int sysReserve = Math.max(sysAvailMB / 10, 100);
-        int reserveMB = mrReserve + sysReserve;
-        logger.info("Reserve " + reserveMB + " MB = " + mrReserve + " (MR reserve) + " + sysReserve + " (SYS reserve)");
-        return reserveMB;
+    @Override
+    protected InputConverterUnit<String[]> getInputConverterUnit() {
+        Preconditions.checkNotNull(cubeDesc);
+        Preconditions.checkNotNull(dictionaryMap);
+        return new InputConverterUnitForRawData(cubeDesc, flatDesc, dictionaryMap);
     }
 
     @Override
-    public void doMap(KEYIN key, Object record, Context context) throws IOException, InterruptedException {
-        // put each row to the queue
-        Collection<String[]> rowCollection = flatTableInputFormat.parseMapperInput(record);
-
-        for(String[] row: rowCollection) {
-            List<String> rowAsList = Arrays.asList(row);
-            while (!future.isDone()) {
-                if (queue.offer(rowAsList, 1, TimeUnit.SECONDS)) {
-                    break;
-                }
-            }
-        }
+    protected String[] getRecordFromKeyValue(KEYIN key, Object value) {
+        return flatTableInputFormat.parseMapperInput(value).iterator().next();
     }
 
     @Override
-    protected void doCleanup(Context context) throws IOException, InterruptedException {
-        logger.info("Totally handled " + mapCounter + " records!");
-
-        while (!future.isDone()) {
-            if (queue.offer(Collections.<String> emptyList(), 1, TimeUnit.SECONDS)) {
-                break;
-            }
-        }
-
-        try {
-            future.get();
-        } catch (Exception e) {
-            throw new IOException("Failed to build cube in mapper " + context.getTaskAttemptID().getTaskID().getId(), e);
-        }
-        queue.clear();
+    protected Future getCubingThreadFuture(Context context, Map<TblColRef, Dictionary<String>> dictionaryMap,
+            int reserveMemoryMB, CuboidScheduler cuboidScheduler, InputConverterUnit<String[]> inputConverterUnit) {
+        AbstractInMemCubeBuilder cubeBuilder = new DoggedCubeBuilder(cuboidScheduler, flatDesc, dictionaryMap);
+        cubeBuilder.setReserveMemoryMB(reserveMemoryMB);
+        cubeBuilder.setConcurrentThreads(taskThreadCount);
+
+        ExecutorService executorService = Executors.newSingleThreadExecutor(
+                new ThreadFactoryBuilder().setDaemon(true).setNameFormat("inmemory-cube-building-mapper-%d").build());
+        return executorService.submit(cubeBuilder.buildAsRunnable(queue, inputConverterUnit,
+                new MapContextGTRecordWriter(context, cubeDesc, cubeSegment)));
     }
-
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java
new file mode 100644
index 0000000..43f95e5
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapperBase.java
@@ -0,0 +1,216 @@
+/*
+ * 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.mr.steps;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.common.util.MemoryBudgetController;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.cuboid.DefaultCuboidScheduler;
+import org.apache.kylin.cube.cuboid.TreeCuboidSchedulerManager;
+import org.apache.kylin.cube.inmemcubing.ConsumeBlockingQueueController;
+import org.apache.kylin.cube.inmemcubing.InputConverterUnit;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableEnrich;
+import org.apache.kylin.engine.EngineFactory;
+import org.apache.kylin.engine.mr.KylinMapper;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.CuboidStatsReaderUtil;
+import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Maps;
+
+/**
+ */
+public abstract class InMemCuboidMapperBase<KEYIN, VALUEIN, KEYOUT, VALUEOUT, T> extends KylinMapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
+
+    private static final Logger logger = LoggerFactory.getLogger(InMemCuboidMapperBase.class);
+
+    private int reserveMemoryMB;
+    private int nSplit = 1;
+    private int countOfLastSplit = 0;
+    private int counter = 0;
+    private int splitRowThreshold = Integer.MAX_VALUE;
+    private int unitRows = ConsumeBlockingQueueController.DEFAULT_BATCH_SIZE;
+
+    protected CubeInstance cube;
+    protected CubeDesc cubeDesc;
+    protected CubeSegment cubeSegment;
+    protected Map<TblColRef, Dictionary<String>> dictionaryMap;
+    protected IJoinedFlatTableDesc flatDesc;
+
+    protected int taskThreadCount;
+    protected BlockingQueue<T> queue = new LinkedBlockingQueue<>(2000);
+    protected InputConverterUnit<T> inputConverterUnit;
+    private Future<?> future;
+
+    protected abstract InputConverterUnit<T> getInputConverterUnit();
+
+    protected abstract Future getCubingThreadFuture(Context context, Map<TblColRef, Dictionary<String>> dictionaryMap, int reserveMemoryMB, //
+                                                    CuboidScheduler cuboidScheduler, InputConverterUnit<T> inputConverterUnit);
+
+    protected abstract T getRecordFromKeyValue(KEYIN key, VALUEIN value);
+
+    @Override
+    protected void doSetup(Context context) throws IOException {
+        super.bindCurrentConfiguration(context.getConfiguration());
+
+        Configuration conf = context.getConfiguration();
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+        String cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
+        cube = CubeManager.getInstance(config).getCube(cubeName);
+        cubeDesc = cube.getDescriptor();
+        String segmentID = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_ID);
+        cubeSegment = cube.getSegmentById(segmentID);
+        flatDesc = new CubeJoinedFlatTableEnrich(EngineFactory.getJoinedFlatTableDesc(cubeSegment), cubeDesc);
+
+        dictionaryMap = Maps.newHashMap();
+
+        // dictionary
+        for (TblColRef col : cubeDesc.getAllColumnsHaveDictionary()) {
+            Dictionary<?> dict = cubeSegment.getDictionary(col);
+            if (dict == null) {
+                logger.warn("Dictionary for " + col + " was not found.");
+            }
+
+            dictionaryMap.put(col, cubeSegment.getDictionary(col));
+        }
+
+        // check memory more often if a single row is big
+        if (cubeDesc.hasMemoryHungryMeasures()) {
+            unitRows /= 10;
+        }
+
+        String cuboidModeName = conf.get(BatchConstants.CFG_CUBOID_MODE);
+        CuboidScheduler cuboidScheduler = TreeCuboidSchedulerManager.getTreeCuboidScheduler(cubeDesc, //
+                CuboidStatsReaderUtil.readCuboidStatsFromSegment(cube.getCuboidsByMode(cuboidModeName), cubeSegment));
+        if (cuboidScheduler == null) {
+            cuboidScheduler = new DefaultCuboidScheduler(cubeDesc);
+        }
+
+        taskThreadCount = config.getCubeAlgorithmInMemConcurrentThreads();
+        reserveMemoryMB = calculateReserveMB(conf);
+        inputConverterUnit = getInputConverterUnit();
+        future = getCubingThreadFuture(context, dictionaryMap, reserveMemoryMB, cuboidScheduler, inputConverterUnit);
+    }
+
+    private int calculateReserveMB(Configuration configuration) {
+        int sysAvailMB = MemoryBudgetController.getSystemAvailMB();
+        int mrReserve = configuration.getInt("mapreduce.task.io.sort.mb", 100);
+        int sysReserve = Math.max(sysAvailMB / 10, 100);
+        int reserveMB = mrReserve + sysReserve;
+        logger.info("Reserve " + reserveMB + " MB = " + mrReserve + " (MR reserve) + " + sysReserve + " (SYS reserve)");
+        return reserveMB;
+    }
+
+    @Override
+    public void doMap(KEYIN key, VALUEIN value, Context context) throws IOException, InterruptedException {
+        // put each row to the queue
+        T row = getRecordFromKeyValue(key, value);
+
+        if (offer(context, row, 1, TimeUnit.MINUTES, 60)) {
+            counter++;
+            countOfLastSplit++;
+            if (counter % BatchConstants.NORMAL_RECORD_LOG_THRESHOLD == 0) {
+                logger.info("Handled " + counter + " records, internal queue size = " + queue.size());
+            }
+        } else {
+            throw new IOException("Failed to offer row to internal queue due to queue full!");
+        }
+
+        if (counter % unitRows == 0 && shouldCutSplit(nSplit, countOfLastSplit)) {
+            if (offer(context, inputConverterUnit.getCutUnit(), 1, TimeUnit.MINUTES, 60)) {
+                countOfLastSplit = 0;
+            } else {
+                throw new IOException("Failed to offer row to internal queue due to queue full!");
+            }
+            nSplit++;
+        }
+    }
+
+    @Override
+    protected void doCleanup(Context context) throws IOException, InterruptedException {
+        logger.info("Totally handled " + mapCounter + " records!");
+
+        while (!future.isDone()) {
+            if (queue.offer(inputConverterUnit.getEmptyUnit(), 1, TimeUnit.SECONDS)) {
+                break;
+            }
+        }
+
+        futureGet(context);
+        queue.clear();
+    }
+
+    private boolean shouldCutSplit(int nSplit, long splitRowCount) {
+        int systemAvailMB = MemoryBudgetController.getSystemAvailMB();
+
+        logger.info(splitRowCount + " records went into split #" + nSplit + "; " + systemAvailMB + " MB left, " + reserveMemoryMB + " MB threshold");
+
+        if (splitRowCount >= splitRowThreshold) {
+            logger.info("Split cut due to hitting splitRowThreshold " + splitRowThreshold);
+            return true;
+        }
+
+        if (systemAvailMB <= reserveMemoryMB) {
+            logger.info("Split cut due to hitting memory threshold, system avail " + systemAvailMB + " MB <= reserve " + reserveMemoryMB + " MB");
+            return true;
+        }
+
+        return false;
+    }
+
+    private boolean offer(Context context, T row, long timeout, TimeUnit unit, int nRound) throws IOException, InterruptedException {
+        while (nRound > 0) {
+            if (queue.offer(row, timeout, unit)) {
+                return true;
+            }
+            if (future.isDone()) {
+                futureGet(context);
+                throw new IOException("Failed to build cube in mapper due to cubing thread exit unexpectedly");
+            }
+            nRound--;
+        }
+        return false;
+    }
+
+    private void futureGet(Context context) throws IOException {
+        try {
+            future.get();
+        } catch (Exception e) {
+            throw new IOException("Failed to build cube in mapper " + context.getTaskAttemptID().getTaskID().getId(), e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderStressTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderStressTest.java b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderStressTest.java
index 5e5b16a..fa2d792 100644
--- a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderStressTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderStressTest.java
@@ -19,7 +19,6 @@
 package org.apache.kylin.cube.inmemcubing;
 
 import java.io.IOException;
-import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.ExecutorService;
@@ -79,7 +78,7 @@ public class ITDoggedCubeBuilderStressTest extends LocalFileMetadataTestCase {
     @Test
     public void test() throws Exception {
 
-        ArrayBlockingQueue<List<String>> queue = new ArrayBlockingQueue<List<String>>(1000);
+        ArrayBlockingQueue<String[]> queue = new ArrayBlockingQueue<String[]>(1000);
         ExecutorService executorService = Executors.newSingleThreadExecutor();
         long randSeed = System.currentTimeMillis();
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderTest.java b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderTest.java
index 1e10d79..0338da8 100644
--- a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderTest.java
@@ -26,7 +26,6 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
-import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.ExecutorService;
@@ -83,7 +82,7 @@ public class ITDoggedCubeBuilderTest extends LocalFileMetadataTestCase {
     @Test
     public void test() throws Exception {
 
-        ArrayBlockingQueue<List<String>> queue = new ArrayBlockingQueue<List<String>>(1000);
+        ArrayBlockingQueue<String[]> queue = new ArrayBlockingQueue<String[]>(1000);
         ExecutorService executorService = Executors.newSingleThreadExecutor();
         long randSeed = System.currentTimeMillis();
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/e83a2e5d/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java
index 49d267a..ad754cd 100644
--- a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java
@@ -113,7 +113,7 @@ public class ITInMemCubeBuilderTest extends LocalFileMetadataTestCase {
         //DoggedCubeBuilder cubeBuilder = new DoggedCubeBuilder(cube.getDescriptor(), dictionaryMap);
         cubeBuilder.setConcurrentThreads(nThreads);
 
-        ArrayBlockingQueue<List<String>> queue = new ArrayBlockingQueue<List<String>>(1000);
+        ArrayBlockingQueue<String[]> queue = new ArrayBlockingQueue<String[]>(1000);
         ExecutorService executorService = Executors.newSingleThreadExecutor();
 
         try {
@@ -144,11 +144,13 @@ public class ITInMemCubeBuilderTest extends LocalFileMetadataTestCase {
         }
     }
 
-    static void feedData(final CubeInstance cube, final String flatTable, ArrayBlockingQueue<List<String>> queue, int count) throws IOException, InterruptedException {
+    static void feedData(final CubeInstance cube, final String flatTable, ArrayBlockingQueue<String[]> queue, int count)
+            throws IOException, InterruptedException {
         feedData(cube, flatTable, queue, count, 0);
     }
 
-    static void feedData(final CubeInstance cube, final String flatTable, ArrayBlockingQueue<List<String>> queue, int count, long randSeed) throws IOException, InterruptedException {
+    static void feedData(final CubeInstance cube, final String flatTable, ArrayBlockingQueue<String[]> queue, int count,
+            long randSeed) throws IOException, InterruptedException {
         IJoinedFlatTableDesc flatDesc = EngineFactory.getJoinedFlatTableDesc(cube.getDescriptor());
         int nColumns = flatDesc.getAllColumns().size();
 
@@ -177,14 +179,14 @@ public class ITInMemCubeBuilderTest extends LocalFileMetadataTestCase {
 
         // output with random data
         for (; count > 0; count--) {
-            ArrayList<String> row = new ArrayList<String>(nColumns);
+            String[] row = new String[nColumns];
             for (int i = 0; i < nColumns; i++) {
                 String[] candidates = distincts.get(i);
-                row.add(candidates[rand.nextInt(candidates.length)]);
+                row[i] = candidates[rand.nextInt(candidates.length)];
             }
             queue.put(row);
         }
-        queue.put(new ArrayList<String>(0));
+        queue.put(InputConverterUnitForRawData.EMPTY_ROW);
     }
 
     static Map<TblColRef, Dictionary<String>> getDictionaryMap(CubeInstance cube, String flatTable) throws IOException {