You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by ma...@apache.org on 2015/11/30 04:16:26 UTC

[5/7] incubator-kylin git commit: whitelist

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
index abec436..87b182d 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
@@ -18,127 +18,51 @@
 
 package org.apache.kylin.cube.model;
 
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
-import org.apache.commons.lang.StringUtils;
-import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.metadata.model.TblColRef;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Objects;
 
 /**
  */
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class RowKeyDesc {
 
-    public static class HierarchyMask {
-        public long fullMask;
-        public long[] allMasks;
-    }
-
-    public static class AggrGroupMask {
-        public AggrGroupMask(int size) {
-            groupOneBitMasks = new long[size];
-        }
-
-        public long groupMask;
-        public long groupOneBitMasks[];
-        public long uniqueMask;
-        public long leftoverMask;
-    }
-
     @JsonProperty("rowkey_columns")
     private RowKeyColDesc[] rowkeyColumns;
-    @JsonProperty("aggregation_groups")
-    private String[][] aggregationGroups;
 
     // computed content
+    private long fullMask;
     private CubeDesc cubeDesc;
     private Map<TblColRef, RowKeyColDesc> columnMap;
 
-    private long fullMask;
-    private long mandatoryColumnMask;
-    private AggrGroupMask[] aggrGroupMasks;
-    private long aggrGroupFullMask;
-    private long tailMask;
-
-    private List<HierarchyMask> hierarchyMasks;
-
     public RowKeyColDesc[] getRowKeyColumns() {
         return rowkeyColumns;
     }
 
-    // search a specific row key col
-    public int getRowKeyIndexByColumnName(String columnName) {
-        if (this.rowkeyColumns == null)
-            return -1;
-
-        for (int i = 0; i < this.rowkeyColumns.length; ++i) {
-            RowKeyColDesc desc = this.rowkeyColumns[i];
-            if (desc.getColumn().equalsIgnoreCase(columnName)) {
-                return i;
-            }
-        }
-        return -1;
-    }
-
-    public int getNCuboidBuildLevels() {
-        // N aggregation columns requires N levels of cuboid build
-        // - N columns requires N-1 levels build
-        // - zero tail cuboid needs one more additional level
-        Set<String> aggDims = new HashSet<String>();
-        for (String[] aggrGroup : aggregationGroups) {
-            for (String dim : aggrGroup) {
-                aggDims.add(dim);
-            }
-        }
-        return aggDims.size();
-    }
-
-    public String[][] getAggregationGroups() {
-        return aggregationGroups;
-    }
-
-    public CubeDesc getCubeRef() {
-        return cubeDesc;
-    }
-
-    public void setCubeRef(CubeDesc cubeRef) {
+    //    public int getNCuboidBuildLevels() {
+    //        // N aggregation columns requires N levels of cuboid build
+    //        // - N columns requires N-1 levels build
+    //        // - zero tail cuboid needs one more additional level
+    //        Set<String> aggDims = new HashSet<String>();
+    //        for (String[] aggrGroup : aggregationGroups) {
+    //            for (String dim : aggrGroup) {
+    //                aggDims.add(dim);
+    //            }
+    //        }
+    //        return aggDims.size();
+    //    }
+
+    public void setCubeDesc(CubeDesc cubeRef) {
         this.cubeDesc = cubeRef;
     }
 
-    public long getFullMask() {
-        return fullMask;
-    }
-
-    public long getMandatoryColumnMask() {
-        return mandatoryColumnMask;
-    }
-
-    public long getAggrGroupFullMask() {
-        return aggrGroupFullMask;
-    }
-
-    public AggrGroupMask[] getAggrGroupMasks() {
-        return aggrGroupMasks;
-    }
-
-    public List<HierarchyMask> getHierarchyMasks() {
-        return hierarchyMasks;
-    }
-
-    public long getTailMask() {
-        return tailMask;
-    }
-
     public int getColumnBitIndex(TblColRef col) {
         return getColDesc(col).getBitIndex();
     }
@@ -147,10 +71,6 @@ public class RowKeyDesc {
         return getColDesc(col).getLength();
     }
 
-    public String getDictionary(TblColRef col) {
-        return getColDesc(col).getDictionary();
-    }
-
     private RowKeyColDesc getColDesc(TblColRef col) {
         RowKeyColDesc desc = columnMap.get(col);
         if (desc == null)
@@ -158,40 +78,37 @@ public class RowKeyDesc {
         return desc;
     }
 
-    public boolean isUseDictionary(int index) {
-        String useDictionary = rowkeyColumns[index].getDictionary();
-        return useDictionary(useDictionary);
-    }
-
     public boolean isUseDictionary(TblColRef col) {
-        String useDictionary = getDictionary(col);
-        return useDictionary(useDictionary);
+        return getColDesc(col).isUsingDictionary();
     }
 
-    private boolean useDictionary(String useDictionary) {
-        return !StringUtils.isBlank(useDictionary) && !"false".equals(useDictionary);
-    }
+    public void init(CubeDesc cubeDesc) {
 
-    public void init(CubeDesc cube) {
-        setCubeRef(cube);
-        Map<String, TblColRef> colNameAbbr = cube.buildColumnNameAbbreviation();
+        setCubeDesc(cubeDesc);
+        Map<String, TblColRef> colNameAbbr = cubeDesc.buildColumnNameAbbreviation();
 
         buildRowKey(colNameAbbr);
-        buildAggregationGroups(colNameAbbr);
-        buildHierarchyMasks();
+    }
+
+    public RowKeyColDesc[] getRowkeyColumns() {
+        return rowkeyColumns;
+    }
+
+    public void setRowkeyColumns(RowKeyColDesc[] rowkeyColumns) {
+        this.rowkeyColumns = rowkeyColumns;
     }
 
     @Override
     public String toString() {
-        return "RowKeyDesc [rowkeyColumns=" + Arrays.toString(rowkeyColumns) + ", aggregationGroups=" + Arrays.toString(aggregationGroups) + "]";
+        return Objects.toStringHelper(this).add("RowKeyColumns", Arrays.toString(rowkeyColumns)).toString();
     }
 
     private void buildRowKey(Map<String, TblColRef> colNameAbbr) {
         columnMap = new HashMap<TblColRef, RowKeyColDesc>();
-        mandatoryColumnMask = 0;
 
         for (int i = 0; i < rowkeyColumns.length; i++) {
             RowKeyColDesc rowKeyColDesc = rowkeyColumns[i];
+            rowKeyColDesc.init();
             String column = rowKeyColDesc.getColumn();
             rowKeyColDesc.setColumn(column.toUpperCase());
             rowKeyColDesc.setBitIndex(rowkeyColumns.length - i - 1);
@@ -201,95 +118,17 @@ public class RowKeyDesc {
             }
 
             columnMap.put(rowKeyColDesc.getColRef(), rowKeyColDesc);
-
-            if (rowKeyColDesc.isMandatory()) {
-                mandatoryColumnMask |= 1L << rowKeyColDesc.getBitIndex();
-            }
-        }
-    }
-
-    private void buildAggregationGroups(Map<String, TblColRef> colNameAbbr) {
-        if (aggregationGroups == null) {
-            aggregationGroups = new String[0][];
-        }
-
-        for (int i = 0; i < aggregationGroups.length; i++) {
-            StringUtil.toUpperCaseArray(aggregationGroups[i], this.aggregationGroups[i]);
         }
 
+        this.fullMask = 0L;
         for (int i = 0; i < this.rowkeyColumns.length; i++) {
             int index = rowkeyColumns[i].getBitIndex();
             this.fullMask |= 1L << index;
         }
-
-        this.aggrGroupMasks = new AggrGroupMask[aggregationGroups.length];
-        for (int i = 0; i < this.aggregationGroups.length; i++) {
-            String[] aggGrp = this.aggregationGroups[i];
-            AggrGroupMask mask = new AggrGroupMask(aggGrp.length);
-
-            for (int j = 0; j < aggGrp.length; j++) {
-                TblColRef aggCol = colNameAbbr.get(aggGrp[j].toUpperCase());
-                if (aggCol == null) {
-                    throw new IllegalArgumentException("Can't find aggregation column " + aggGrp[j] + " in  cube " + this.cubeDesc.getName());
-                }
-                Integer index = getColumnBitIndex(aggCol);
-                mask.groupMask |= 1L << index;
-                mask.groupOneBitMasks[j] = 1L << index;
-                this.aggrGroupFullMask |= 1L << index;
-            }
-            this.aggrGroupMasks[i] = mask;
-        }
-
-        this.tailMask = fullMask ^ mandatoryColumnMask ^ aggrGroupFullMask;
-
-        // unique mask = (bits in this group) - (bits in following groups)
-        // leftover mask = (tail bits) + (bits in following groups) - (bits in
-        // this group)
-        for (int i = 0; i < aggrGroupMasks.length; i++) {
-            AggrGroupMask mask = aggrGroupMasks[i];
-
-            mask.uniqueMask = mask.groupMask;
-            for (int j = i + 1; j < aggrGroupMasks.length; j++) {
-                mask.uniqueMask &= ~aggrGroupMasks[j].groupMask;
-            }
-
-            mask.leftoverMask = tailMask;
-            for (int j = i + 1; j < aggrGroupMasks.length; j++) {
-                mask.leftoverMask |= aggrGroupMasks[j].groupMask;
-            }
-            mask.leftoverMask &= ~mask.groupMask;
-        }
     }
 
-    private void buildHierarchyMasks() {
-        this.hierarchyMasks = new ArrayList<HierarchyMask>();
-
-        for (DimensionDesc dimension : this.cubeDesc.getDimensions()) {
-            HierarchyDesc[] hierarchies = dimension.getHierarchy();
-            if (hierarchies == null || hierarchies.length == 0)
-                continue;
-
-            HierarchyMask mask = new HierarchyMask();
-            ArrayList<Long> allMaskList = new ArrayList<Long>();
-            for (int i = 0; i < hierarchies.length; i++) {
-                TblColRef hColumn = hierarchies[i].getColumnRef();
-                Integer index = getColumnBitIndex(hColumn);
-                long bit = 1L << index;
-
-                if ((tailMask & bit) > 0)
-                    continue; // ignore levels in tail, they don't participate
-                              // aggregation group combination anyway
-
-                mask.fullMask |= bit;
-                allMaskList.add(mask.fullMask);
-            }
-
-            mask.allMasks = new long[allMaskList.size()];
-            for (int i = 0; i < allMaskList.size(); i++)
-                mask.allMasks[i] = allMaskList.get(i);
-
-            this.hierarchyMasks.add(mask);
-        }
+    public long getFullMask() {
+        return this.fullMask;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java
new file mode 100644
index 0000000..63b0fc4
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java
@@ -0,0 +1,30 @@
+/*
+ * 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.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class SelectRule {
+    @JsonProperty("hierarchy_dims")
+    public String[][] hierarchy_dims;
+    @JsonProperty("mandatory_dims")
+    public String[] mandatory_dims;
+    @JsonProperty("joint_dims")
+    public String[][] joint_dims;
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeBuildTypeEnum.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeBuildTypeEnum.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeBuildTypeEnum.java
new file mode 100644
index 0000000..2544913
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeBuildTypeEnum.java
@@ -0,0 +1,39 @@
+/*
+ * 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.model.v2;
+
+/**
+ * @author xduo
+ * 
+ */
+public enum CubeBuildTypeEnum {
+    /**
+     * rebuild a segment or incremental build
+     */
+    BUILD,
+    /**
+     * merge segments
+     */
+    MERGE,
+
+    /**
+     * refresh segments
+     */
+    REFRESH
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeDesc.java
new file mode 100644
index 0000000..04e4c65
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeDesc.java
@@ -0,0 +1,867 @@
+/*
+ * 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.model.v2;
+
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.persistence.RootPersistentEntity;
+import org.apache.kylin.common.util.Array;
+import org.apache.kylin.common.util.CaseInsensitiveStringMap;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.metadata.MetadataConstants;
+import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.DataModelDesc;
+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.MeasureDesc;
+import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ */
+@SuppressWarnings("serial")
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class CubeDesc extends RootPersistentEntity {
+
+    public static enum DeriveType {
+        LOOKUP, PK_FK
+    }
+
+    public static class DeriveInfo {
+        public DeriveType type;
+        public DimensionDesc dimension;
+        public TblColRef[] columns;
+        public boolean isOneToOne; // only used when ref from derived to host
+
+        DeriveInfo(DeriveType type, DimensionDesc dimension, TblColRef[] columns, boolean isOneToOne) {
+            this.type = type;
+            this.dimension = dimension;
+            this.columns = columns;
+            this.isOneToOne = isOneToOne;
+        }
+
+        @Override
+        public String toString() {
+            return "DeriveInfo [type=" + type + ", dimension=" + dimension + ", columns=" + Arrays.toString(columns) + ", isOneToOne=" + isOneToOne + "]";
+        }
+
+    }
+
+    private KylinConfig config;
+    private DataModelDesc model;
+
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("model_name")
+    private String modelName;
+    @JsonProperty("description")
+    private String description;
+    @JsonProperty("null_string")
+    private String[] nullStrings;
+    @JsonProperty("dimensions")
+    private List<DimensionDesc> dimensions;
+    @JsonProperty("measures")
+    private List<MeasureDesc> measures;
+    @JsonProperty("rowkey")
+    private RowKeyDesc rowkey;
+    @JsonProperty("hbase_mapping")
+    private HBaseMappingDesc hbaseMapping;
+    @JsonProperty("signature")
+    private String signature;
+    @JsonProperty("notify_list")
+    private List<String> notifyList;
+    @JsonProperty("status_need_notify")
+    private List<String> statusNeedNotify = Collections.emptyList();
+    @JsonProperty("auto_merge_time_ranges")
+    private long[] autoMergeTimeRanges;
+    @JsonProperty("retention_range")
+    private long retentionRange = 0;
+
+    @JsonProperty("engine_type")
+    private int engineType = IEngineAware.ID_MR_V1;
+    @JsonProperty("storage_type")
+    private int storageType = IStorageAware.ID_HBASE;
+
+    private Map<String, Map<String, TblColRef>> columnMap = new HashMap<String, Map<String, TblColRef>>();
+    private LinkedHashSet<TblColRef> allColumns = new LinkedHashSet<TblColRef>();
+    private LinkedHashSet<TblColRef> dimensionColumns = new LinkedHashSet<TblColRef>();
+
+    private LinkedHashSet<TblColRef> measureDisplayColumns = new LinkedHashSet<TblColRef>();
+    private Map<TblColRef, DeriveInfo> derivedToHostMap = Maps.newHashMap();
+    private Map<Array<TblColRef>, List<DeriveInfo>> hostToDerivedMap = Maps.newHashMap();
+
+    public boolean isEnableSharding() {
+        //in the future may extend to other storage that is shard-able
+        return storageType == IStorageAware.ID_SHARDED_HBASE;
+    }
+
+    /**
+     * Error messages during resolving json metadata
+     */
+    private List<String> errors = new ArrayList<String>();
+
+    /**
+     * @return all columns this cube can support, including derived
+     */
+    public Set<TblColRef> listAllColumns() {
+        return allColumns;
+    }
+
+    /**
+     * @return dimension columns including derived, BUT NOT measures
+     */
+    public Set<TblColRef> listDimensionColumnsIncludingDerived() {
+        return dimensionColumns;
+    }
+
+    /**
+     * @return dimension columns excluding derived and measures
+     */
+    public List<TblColRef> listDimensionColumnsExcludingDerived() {
+        List<TblColRef> result = new ArrayList<TblColRef>();
+        for (TblColRef col : dimensionColumns) {
+            if (isDerived(col) == false)
+                result.add(col);
+        }
+        return result;
+    }
+
+    /**
+     * Find FunctionDesc by Full Expression.
+     *
+     * @return
+     */
+    public FunctionDesc findFunctionOnCube(FunctionDesc manualFunc) {
+        for (MeasureDesc m : measures) {
+            if (m.getFunction().equals(manualFunc))
+                return m.getFunction();
+        }
+        return null;
+    }
+
+    public TblColRef findColumnRef(String table, String column) {
+        Map<String, TblColRef> cols = columnMap.get(table);
+        if (cols == null)
+            return null;
+        else
+            return cols.get(column);
+    }
+
+    public DimensionDesc findDimensionByColumn(TblColRef col) {
+        for (DimensionDesc dim : dimensions) {
+            if (ArrayUtils.contains(dim.getColumnRefs(), col))
+                return dim;
+        }
+        return null;
+    }
+
+    public DimensionDesc findDimensionByTable(String lookupTableName) {
+        lookupTableName = lookupTableName.toUpperCase();
+        for (DimensionDesc dim : dimensions)
+            if (dim.getTable() != null && dim.getTable().equals(lookupTableName))
+                return dim;
+        return null;
+    }
+
+    public DimensionDesc findDimensionByName(String dimName) {
+        dimName = dimName.toUpperCase();
+        for (DimensionDesc dim : dimensions) {
+            if (dimName.equals(dim.getName()))
+                return dim;
+        }
+        return null;
+    }
+
+    /**
+     * Get all functions from each measure.
+     *
+     * @return
+     */
+    public List<FunctionDesc> listAllFunctions() {
+        List<FunctionDesc> functions = new ArrayList<FunctionDesc>();
+        for (MeasureDesc m : measures) {
+            functions.add(m.getFunction());
+        }
+        return functions;
+    }
+
+    public boolean isDerived(TblColRef col) {
+        return derivedToHostMap.containsKey(col);
+    }
+
+    public DeriveInfo getHostInfo(TblColRef derived) {
+        return derivedToHostMap.get(derived);
+    }
+
+    public Map<Array<TblColRef>, List<DeriveInfo>> getHostToDerivedInfo(List<TblColRef> rowCols, Collection<TblColRef> wantedCols) {
+        Map<Array<TblColRef>, List<DeriveInfo>> result = new HashMap<Array<TblColRef>, List<DeriveInfo>>();
+        for (Entry<Array<TblColRef>, List<DeriveInfo>> entry : hostToDerivedMap.entrySet()) {
+            Array<TblColRef> hostCols = entry.getKey();
+            boolean hostOnRow = rowCols.containsAll(Arrays.asList(hostCols.data));
+            if (!hostOnRow)
+                continue;
+
+            List<DeriveInfo> wantedInfo = new ArrayList<DeriveInfo>();
+            for (DeriveInfo info : entry.getValue()) {
+                if (wantedCols == null || Collections.disjoint(wantedCols, Arrays.asList(info.columns)) == false) // has
+                    // any
+                    // wanted
+                    // columns?
+                    wantedInfo.add(info);
+            }
+
+            if (wantedInfo.size() > 0)
+                result.put(hostCols, wantedInfo);
+        }
+        return result;
+    }
+
+    public String getResourcePath() {
+        return getCubeDescResourcePath(name);
+    }
+
+    public static String getCubeDescResourcePath(String descName) {
+        return ResourceStore.CUBE_DESC_RESOURCE_ROOT + "/" + descName + MetadataConstants.FILE_SURFIX;
+    }
+
+    // ============================================================================
+
+    public HBaseMappingDesc getHBaseMapping() {
+        return hbaseMapping;
+    }
+
+    public void setHBaseMapping(HBaseMappingDesc hbaseMapping) {
+        this.hbaseMapping = hbaseMapping;
+    }
+
+    public KylinConfig getConfig() {
+        return config;
+    }
+
+    public void setConfig(KylinConfig config) {
+        this.config = config;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getModelName() {
+        return modelName;
+    }
+
+    public void setModelName(String modelName) {
+        this.modelName = modelName;
+    }
+
+    public DataModelDesc getModel() {
+        return model;
+    }
+
+    public void setModel(DataModelDesc model) {
+        this.model = model;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    public String getFactTable() {
+        return model.getFactTable();
+    }
+
+    public TableDesc getFactTableDesc() {
+        return model.getFactTableDesc();
+    }
+
+    public String[] getNullStrings() {
+        return nullStrings;
+    }
+
+    public List<DimensionDesc> getDimensions() {
+        return dimensions;
+    }
+
+    public void setDimensions(List<DimensionDesc> dimensions) {
+        this.dimensions = dimensions;
+    }
+
+    public List<MeasureDesc> getMeasures() {
+        return measures;
+    }
+
+    public void setMeasures(List<MeasureDesc> measures) {
+        this.measures = measures;
+    }
+
+    public RowKeyDesc getRowkey() {
+        return rowkey;
+    }
+
+    public void setRowkey(RowKeyDesc rowkey) {
+        this.rowkey = rowkey;
+    }
+
+    public String getSignature() {
+        return signature;
+    }
+
+    public void setSignature(String signature) {
+        this.signature = signature;
+    }
+
+    public List<String> getNotifyList() {
+        return notifyList;
+    }
+
+    public void setNotifyList(List<String> notifyList) {
+        this.notifyList = notifyList;
+    }
+
+    public List<String> getStatusNeedNotify() {
+        return statusNeedNotify;
+    }
+
+    public void setStatusNeedNotify(List<String> statusNeedNotify) {
+        this.statusNeedNotify = statusNeedNotify;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        CubeDesc cubeDesc = (CubeDesc) o;
+
+        if (!name.equals(cubeDesc.name))
+            return false;
+        if (!getFactTable().equals(cubeDesc.getFactTable()))
+            return false;
+
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = 0;
+        result = 31 * result + name.hashCode();
+        result = 31 * result + getFactTable().hashCode();
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "CubeDesc [name=" + name + "]";
+    }
+
+    public boolean checkSignature() {
+        if (StringUtils.isBlank(getSignature())) {
+            return true;
+        }
+        return calculateSignature().equals(getSignature());
+    }
+    
+    public String calculateSignature() {
+        MessageDigest md = null;
+        try {
+            md = MessageDigest.getInstance("MD5");
+            StringBuilder sigString = new StringBuilder();
+            sigString.append(this.name).append("|").append(this.getFactTable()).append("|").append(JsonUtil.writeValueAsString(this.model.getPartitionDesc())).append("|").append(JsonUtil.writeValueAsString(this.dimensions)).append("|").append(JsonUtil.writeValueAsString(this.measures)).append("|").append(JsonUtil.writeValueAsString(this.rowkey)).append("|").append(JsonUtil.writeValueAsString(this.hbaseMapping));
+
+            byte[] signature = md.digest(sigString.toString().getBytes());
+            return new String(Base64.encodeBase64(signature));
+        } catch (NoSuchAlgorithmException | JsonProcessingException e) {
+            throw new RuntimeException("Failed to calculate signature");
+        }
+    }
+
+    public Map<String, TblColRef> buildColumnNameAbbreviation() {
+        Map<String, TblColRef> r = new CaseInsensitiveStringMap<TblColRef>();
+        for (TblColRef col : listDimensionColumnsExcludingDerived()) {
+            r.put(col.getName(), col);
+        }
+        return r;
+    }
+
+    public void init(KylinConfig config, Map<String, TableDesc> tables) {
+        this.errors.clear();
+        this.config = config;
+
+        if (this.modelName == null || this.modelName.length() == 0) {
+            this.addError("The cubeDesc '" + this.getName() + "' doesn't have data model specified.");
+        }
+
+        this.model = MetadataManager.getInstance(config).getDataModelDesc(this.modelName);
+
+        if (this.model == null) {
+            this.addError("No data model found with name '" + modelName + "'.");
+        }
+
+        for (DimensionDesc dim : dimensions) {
+            dim.init(this, tables);
+        }
+
+        sortDimAndMeasure();
+        initDimensionColumns();
+        initMeasureColumns();
+
+        rowkey.init(this);
+        if (hbaseMapping != null) {
+            hbaseMapping.init(this);
+        }
+
+        initMeasureReferenceToColumnFamily();
+
+        // check all dimension columns are presented on rowkey
+        List<TblColRef> dimCols = listDimensionColumnsExcludingDerived();
+        if (rowkey.getRowKeyColumns().length != dimCols.size()) {
+            addError("RowKey columns count (" + rowkey.getRowKeyColumns().length + ") does not match dimension columns count (" + dimCols.size() + "). ");
+        }
+    }
+
+    private void initDimensionColumns() {
+        for (DimensionDesc dim : dimensions) {
+            JoinDesc join = dim.getJoin();
+
+            // init dimension columns
+            ArrayList<TblColRef> dimCols = Lists.newArrayList();
+            String[] colStrs = dim.getColumn();
+
+            // when column is omitted, special case
+            if (colStrs == null && dim.isDerived() || ArrayUtils.contains(colStrs, "{FK}")) {
+                for (TblColRef col : join.getForeignKeyColumns()) {
+                    dimCols.add(initDimensionColRef(col));
+                }
+            }
+            // normal case
+            else {
+                if (colStrs == null || colStrs.length == 0)
+                    throw new IllegalStateException("Dimension column must not be blank " + dim);
+
+                for (String colStr : colStrs) {
+                    dimCols.add(initDimensionColRef(dim, colStr));
+                }
+
+                // fill back column ref in hierarchy
+                if (dim.isHierarchy()) {
+                    for (int i = 0; i < dimCols.size(); i++)
+                        dim.getHierarchy()[i].setColumnRef(dimCols.get(i));
+                }
+            }
+
+            TblColRef[] dimColArray = (TblColRef[]) dimCols.toArray(new TblColRef[dimCols.size()]);
+            dim.setColumnRefs(dimColArray);
+
+            // init derived columns
+            TblColRef[] hostCols = dimColArray;
+            if (dim.isDerived()) {
+                String[] derived = dim.getDerived();
+                String[][] split = splitDerivedColumnAndExtra(derived);
+                String[] derivedNames = split[0];
+                String[] derivedExtra = split[1];
+                TblColRef[] derivedCols = new TblColRef[derivedNames.length];
+                for (int i = 0; i < derivedNames.length; i++) {
+                    derivedCols[i] = initDimensionColRef(dim, derivedNames[i]);
+                }
+                initDerivedMap(hostCols, DeriveType.LOOKUP, dim, derivedCols, derivedExtra);
+            }
+
+            // PK-FK derive the other side
+            if (join != null) {
+                TblColRef[] fk = join.getForeignKeyColumns();
+                TblColRef[] pk = join.getPrimaryKeyColumns();
+
+                allColumns.addAll(Arrays.asList(fk));
+                allColumns.addAll(Arrays.asList(pk));
+                for (int i = 0; i < fk.length; i++) {
+                    int find = ArrayUtils.indexOf(hostCols, fk[i]);
+                    if (find >= 0) {
+                        TblColRef derivedCol = initDimensionColRef(pk[i]);
+                        initDerivedMap(hostCols[find], DeriveType.PK_FK, dim, derivedCol);
+                    }
+                }
+                /** disable this code as we don't need fk be derived from pk
+                 for (int i = 0; i < pk.length; i++) {
+                 int find = ArrayUtils.indexOf(hostCols, pk[i]);
+                 if (find >= 0) {
+                 TblColRef derivedCol = initDimensionColRef(fk[i]);
+                 initDerivedMap(hostCols[find], DeriveType.PK_FK, dim, derivedCol);
+                 }
+                 }
+                 */
+            }
+        }
+    }
+
+    private String[][] splitDerivedColumnAndExtra(String[] derived) {
+        String[] cols = new String[derived.length];
+        String[] extra = new String[derived.length];
+        for (int i = 0; i < derived.length; i++) {
+            String str = derived[i];
+            int cut = str.indexOf(":");
+            if (cut >= 0) {
+                cols[i] = str.substring(0, cut);
+                extra[i] = str.substring(cut + 1).trim();
+            } else {
+                cols[i] = str;
+                extra[i] = "";
+            }
+        }
+        return new String[][] { cols, extra };
+    }
+
+    private void initDerivedMap(TblColRef hostCol, DeriveType type, DimensionDesc dimension, TblColRef derivedCol) {
+        initDerivedMap(new TblColRef[] { hostCol }, type, dimension, new TblColRef[] { derivedCol }, null);
+    }
+
+    private void initDerivedMap(TblColRef[] hostCols, DeriveType type, DimensionDesc dimension, TblColRef[] derivedCols, String[] extra) {
+        if (hostCols.length == 0 || derivedCols.length == 0)
+            throw new IllegalStateException("host/derived columns must not be empty");
+
+        // Although FK derives PK automatically, user unaware of this can declare PK as derived dimension explicitly.
+        // In that case, derivedCols[] will contain a FK which is transformed from the PK by initDimensionColRef().
+        // Must drop FK from derivedCols[] before continue.
+        for (int i = 0; i < derivedCols.length; i++) {
+            if (ArrayUtils.contains(hostCols, derivedCols[i])) {
+                derivedCols = (TblColRef[]) ArrayUtils.remove(derivedCols, i);
+                extra = (String[]) ArrayUtils.remove(extra, i);
+                i--;
+            }
+        }
+
+        Array<TblColRef> hostColArray = new Array<TblColRef>(hostCols);
+        List<DeriveInfo> infoList = hostToDerivedMap.get(hostColArray);
+        if (infoList == null) {
+            hostToDerivedMap.put(hostColArray, infoList = new ArrayList<DeriveInfo>());
+        }
+        infoList.add(new DeriveInfo(type, dimension, derivedCols, false));
+
+        for (int i = 0; i < derivedCols.length; i++) {
+            TblColRef derivedCol = derivedCols[i];
+            boolean isOneToOne = type == DeriveType.PK_FK || ArrayUtils.contains(hostCols, derivedCol) || (extra != null && extra[i].contains("1-1"));
+            derivedToHostMap.put(derivedCol, new DeriveInfo(type, dimension, hostCols, isOneToOne));
+        }
+    }
+
+    private TblColRef initDimensionColRef(DimensionDesc dim, String colName) {
+        TableDesc table = dim.getTableDesc();
+        ColumnDesc col = table.findColumnByName(colName);
+        if (col == null)
+            throw new IllegalArgumentException("No column '" + colName + "' found in table " + table);
+
+        TblColRef ref = new TblColRef(col);
+
+        // always use FK instead PK, FK could be shared by more than one lookup tables
+        JoinDesc join = dim.getJoin();
+        if (join != null) {
+            int idx = ArrayUtils.indexOf(join.getPrimaryKeyColumns(), ref);
+            if (idx >= 0) {
+                ref = join.getForeignKeyColumns()[idx];
+            }
+        }
+        return initDimensionColRef(ref);
+    }
+
+    private TblColRef initDimensionColRef(TblColRef ref) {
+        TblColRef existing = findColumnRef(ref.getTable(), ref.getName());
+        if (existing != null) {
+            return existing;
+        }
+
+        allColumns.add(ref);
+        dimensionColumns.add(ref);
+
+        Map<String, TblColRef> cols = columnMap.get(ref.getTable());
+        if (cols == null) {
+            columnMap.put(ref.getTable(), cols = new HashMap<String, TblColRef>());
+        }
+        cols.put(ref.getName(), ref);
+        return ref;
+    }
+
+    private void initMeasureColumns() {
+        if (measures == null || measures.isEmpty()) {
+            return;
+        }
+
+        TableDesc factTable = getFactTableDesc();
+        for (MeasureDesc m : measures) {
+            m.setName(m.getName().toUpperCase());
+
+            if (m.getDependentMeasureRef() != null) {
+                m.setDependentMeasureRef(m.getDependentMeasureRef().toUpperCase());
+            }
+
+            FunctionDesc f = m.getFunction();
+            f.setExpression(f.getExpression().toUpperCase());
+            f.initReturnDataType();
+
+            ParameterDesc p = f.getParameter();
+            p.normalizeColumnValue();
+
+            ArrayList<TblColRef> colRefs = Lists.newArrayList();
+            if (p.isColumnType()) {
+                for (String cName : p.getValue().split("\\s*,\\s*")) {
+                    ColumnDesc sourceColumn = factTable.findColumnByName(cName);
+                    TblColRef colRef = new TblColRef(sourceColumn);
+                    colRefs.add(colRef);
+                    allColumns.add(colRef);
+                }
+            }
+
+            // for topN
+            if (StringUtils.isNotEmpty(p.getDisplayColumn())) {
+                ColumnDesc sourceColumn = factTable.findColumnByName(p.getDisplayColumn());
+                TblColRef colRef = new TblColRef(sourceColumn);
+                colRefs.add(colRef);
+                measureDisplayColumns.add(colRef);
+                allColumns.add(colRef);
+            }
+
+            if (colRefs.isEmpty() == false)
+                p.setColRefs(colRefs);
+
+            // verify holistic count distinct as a dependent measure
+            if (m.getFunction().isHolisticCountDistinct() && StringUtils.isBlank(m.getDependentMeasureRef())) {
+                throw new IllegalStateException(m + " is a holistic count distinct but it has no DependentMeasureRef defined!");
+            }
+        }
+    }
+
+    private void initMeasureReferenceToColumnFamily() {
+        if (measures == null || measures.size() == 0)
+            return;
+
+        Map<String, MeasureDesc> measureLookup = new HashMap<String, MeasureDesc>();
+        for (MeasureDesc m : measures)
+            measureLookup.put(m.getName(), m);
+        Map<String, Integer> measureIndexLookup = new HashMap<String, Integer>();
+        for (int i = 0; i < measures.size(); i++)
+            measureIndexLookup.put(measures.get(i).getName(), i);
+
+        for (HBaseColumnFamilyDesc cf : getHBaseMapping().getColumnFamily()) {
+            for (HBaseColumnDesc c : cf.getColumns()) {
+                String[] colMeasureRefs = c.getMeasureRefs();
+                MeasureDesc[] measureDescs = new MeasureDesc[colMeasureRefs.length];
+                int[] measureIndex = new int[colMeasureRefs.length];
+                for (int i = 0; i < colMeasureRefs.length; i++) {
+                    measureDescs[i] = measureLookup.get(colMeasureRefs[i]);
+                    measureIndex[i] = measureIndexLookup.get(colMeasureRefs[i]);
+                }
+                c.setMeasures(measureDescs);
+                c.setMeasureIndex(measureIndex);
+                c.setColumnFamilyName(cf.getName());
+            }
+        }
+    }
+
+    private void sortDimAndMeasure() {
+        sortDimensionsByID();
+        sortMeasuresByID();
+        for (DimensionDesc dim : dimensions) {
+            sortHierarchiesByLevel(dim.getHierarchy());
+        }
+    }
+
+    private void sortDimensionsByID() {
+        Collections.sort(dimensions, new Comparator<DimensionDesc>() {
+            @Override
+            public int compare(DimensionDesc d1, DimensionDesc d2) {
+                Integer id1 = d1.getId();
+                Integer id2 = d2.getId();
+                return id1.compareTo(id2);
+            }
+        });
+    }
+
+    private void sortMeasuresByID() {
+        if (measures == null) {
+            measures = Lists.newArrayList();
+        }
+
+//        Collections.sort(measures, new Comparator<MeasureDesc>() {
+//            @Override
+//            public int compare(MeasureDesc m1, MeasureDesc m2) {
+//                Integer id1 = m1.getId();
+//                Integer id2 = m2.getId();
+//                return id1.compareTo(id2);
+//            }
+//        });
+    }
+
+    private void sortHierarchiesByLevel(HierarchyDesc[] hierarchies) {
+        if (hierarchies != null) {
+            Arrays.sort(hierarchies, new Comparator<HierarchyDesc>() {
+                @Override
+                public int compare(HierarchyDesc h1, HierarchyDesc h2) {
+                    Integer level1 = Integer.parseInt(h1.getLevel());
+                    Integer level2 = Integer.parseInt(h2.getLevel());
+                    return level1.compareTo(level2);
+                }
+            });
+        }
+    }
+
+    public boolean hasHolisticCountDistinctMeasures() {
+        for (MeasureDesc measure : measures) {
+            if (measure.getFunction().isHolisticCountDistinct()) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    public long getRetentionRange() {
+        return retentionRange;
+    }
+
+    public void setRetentionRange(long retentionRange) {
+        this.retentionRange = retentionRange;
+    }
+
+    public long[] getAutoMergeTimeRanges() {
+        return autoMergeTimeRanges;
+    }
+
+    public void setAutoMergeTimeRanges(long[] autoMergeTimeRanges) {
+        this.autoMergeTimeRanges = autoMergeTimeRanges;
+    }
+
+    /**
+     * Add error info and thrown exception out
+     *
+     * @param message
+     */
+    public void addError(String message) {
+        addError(message, false);
+    }
+
+    /**
+     * @param message error message
+     * @param silent  if throw exception
+     */
+    public void addError(String message, boolean silent) {
+        if (!silent) {
+            throw new IllegalStateException(message);
+        } else {
+            this.errors.add(message);
+        }
+    }
+
+    public List<String> getError() {
+        return this.errors;
+    }
+
+    public HBaseMappingDesc getHbaseMapping() {
+        return hbaseMapping;
+    }
+
+    public void setHbaseMapping(HBaseMappingDesc hbaseMapping) {
+        this.hbaseMapping = hbaseMapping;
+    }
+
+    public void setNullStrings(String[] nullStrings) {
+        this.nullStrings = nullStrings;
+    }
+
+    public int getStorageType() {
+        return storageType;
+    }
+
+    void setStorageType(int storageType) {
+        this.storageType = storageType;
+    }
+
+    public int getEngineType() {
+        return engineType;
+    }
+
+    void setEngineType(int engineType) {
+        this.engineType = engineType;
+    }
+
+    public List<TblColRef> getAllColumnsNeedDictionary() {
+        List<TblColRef> result = Lists.newArrayList();
+
+        for (RowKeyColDesc rowKeyColDesc : rowkey.getRowKeyColumns()) {
+            TblColRef colRef = rowKeyColDesc.getColRef();
+            if (rowkey.isUseDictionary(colRef)) {
+                result.add(colRef);
+            }
+        }
+
+        for (TblColRef colRef : measureDisplayColumns) {
+            if (!result.contains(colRef))
+                result.add(colRef);
+        }
+        return result;
+    }
+
+    public LinkedHashSet<TblColRef> getMeasureDisplayColumns() {
+        return measureDisplayColumns;
+    }
+
+
+    public boolean hasMeasureUsingDictionary() {
+        for (MeasureDesc measureDesc : this.getMeasures()) {
+            if (measureDesc.getFunction().isTopN())
+                return true;
+        }
+
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/DimensionDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/DimensionDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/DimensionDesc.java
new file mode 100644
index 0000000..3432d08
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/DimensionDesc.java
@@ -0,0 +1,239 @@
+/*
+ * 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.model.v2;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.metadata.model.JoinDesc;
+import org.apache.kylin.metadata.model.LookupDesc;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class DimensionDesc {
+
+    @JsonProperty("id")
+    private int id;
+
+    @JsonProperty("name")
+    private String name;
+
+    @JsonProperty("hierarchy")
+    private boolean isHierarchy;
+    @JsonProperty("table")
+    private String table;
+    @JsonProperty("column")
+    private String[] column;
+    @JsonProperty("derived")
+    private String[] derived;
+
+    private TableDesc tableDesc;
+    private JoinDesc join;
+    private HierarchyDesc[] hierarchy;
+
+    // computed
+    private TblColRef[] columnRefs;
+    private TblColRef[] derivedColRefs;
+
+    public void init(CubeDesc cubeDesc, Map<String, TableDesc> tables) {
+        if (name != null)
+            name = name.toUpperCase();
+
+        if (table != null)
+            table = table.toUpperCase();
+
+        tableDesc = tables.get(this.getTable());
+        if (tableDesc == null)
+            throw new IllegalStateException("Can't find table " + table + " for dimension " + name);
+
+        join = null;
+        for (LookupDesc lookup : cubeDesc.getModel().getLookups()) {
+            if (lookup.getTable().equalsIgnoreCase(this.getTable())) {
+                join = lookup.getJoin();
+                break;
+            }
+        }
+
+        if (isHierarchy && this.column.length > 0) {
+            List<HierarchyDesc> hierarchyList = new ArrayList<HierarchyDesc>(3);
+            for (int i = 0, n = this.column.length; i < n; i++) {
+                String aColumn = this.column[i];
+                HierarchyDesc aHierarchy = new HierarchyDesc();
+                aHierarchy.setLevel(String.valueOf(i + 1));
+                aHierarchy.setColumn(aColumn);
+                hierarchyList.add(aHierarchy);
+            }
+
+            this.hierarchy = hierarchyList.toArray(new HierarchyDesc[hierarchyList.size()]);
+        }
+
+        if (hierarchy != null && hierarchy.length == 0)
+            hierarchy = null;
+        if (derived != null && derived.length == 0)
+            derived = null;
+
+        if (hierarchy != null) {
+            for (HierarchyDesc h : hierarchy)
+                h.setColumn(h.getColumn().toUpperCase());
+        }
+
+        if (derived != null) {
+            StringUtil.toUpperCaseArray(derived, derived);
+        }
+
+        if (derived != null && join == null) {
+            throw new IllegalStateException("Derived can only be defined on lookup table, cube " + cubeDesc + ", " + this);
+        }
+    }
+
+    public boolean isHierarchyColumn(TblColRef col) {
+        if (hierarchy == null)
+            return false;
+
+        for (HierarchyDesc hier : hierarchy) {
+            if (hier.getColumnRef().equals(col))
+                return true;
+        }
+        return false;
+    }
+
+    public boolean isDerived() {
+        return derived != null;
+    }
+
+    public boolean isHierarchy() {
+        return isHierarchy;
+    }
+
+    public void setHierarchy(boolean isHierarchy) {
+        this.isHierarchy = isHierarchy;
+    }
+
+    public String getTable() {
+        return table;
+    }
+
+    public void setTable(String table) {
+        this.table = table;
+    }
+
+    public int getId() {
+        return id;
+    }
+
+    public void setId(int id) {
+        this.id = id;
+    }
+
+    public JoinDesc getJoin() {
+        return join;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public TblColRef[] getColumnRefs() {
+        return this.columnRefs;
+    }
+
+    public void setColumnRefs(TblColRef[] colRefs) {
+        this.columnRefs = colRefs;
+    }
+
+    public String[] getColumn() {
+        return this.column;
+    }
+
+    public void setColumn(String[] column) {
+        this.column = column;
+    }
+
+    public HierarchyDesc[] getHierarchy() {
+        return hierarchy;
+    }
+
+    public void setHierarchy(HierarchyDesc[] hierarchy) {
+        this.hierarchy = hierarchy;
+    }
+
+    public String[] getDerived() {
+        return derived;
+    }
+
+    public void setDerived(String[] derived) {
+        this.derived = derived;
+    }
+
+    public TblColRef[] getDerivedColRefs() {
+        return derivedColRefs;
+    }
+
+    public void setDerivedColRefs(TblColRef[] derivedColRefs) {
+        this.derivedColRefs = derivedColRefs;
+    }
+
+    public TableDesc getTableDesc() {
+        return this.tableDesc;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        DimensionDesc that = (DimensionDesc) o;
+
+        if (id != that.id)
+            return false;
+        if (!name.equals(that.name))
+            return false;
+
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = id;
+        result = 31 * result + name.hashCode();
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "DimensionDesc [name=" + name + ", join=" + join + ", hierarchy=" + Arrays.toString(hierarchy) + ", table=" + table + ", column=" + Arrays.toString(column) + ", derived=" + Arrays.toString(derived) + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnDesc.java
new file mode 100644
index 0000000..1912e63
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnDesc.java
@@ -0,0 +1,138 @@
+/*
+ * 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.model.v2;
+
+import java.util.Arrays;
+
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class HBaseColumnDesc {
+
+    @JsonProperty("qualifier")
+    private String qualifier;
+    @JsonProperty("measure_refs")
+    private String[] measureRefs;
+
+    // these two will be assembled at runtime
+    private MeasureDesc[] measures;
+    private int[] measureIndex; // the index on CubeDesc.getMeasures()
+    private String columnFamilyName;
+
+    public String getQualifier() {
+        return qualifier;
+    }
+
+    public void setQualifier(String qualifier) {
+        this.qualifier = qualifier;
+    }
+
+    public String[] getMeasureRefs() {
+        return measureRefs;
+    }
+
+    public void setMeasureRefs(String[] measureRefs) {
+        this.measureRefs = measureRefs;
+    }
+    
+    public int[] getMeasureIndex() {
+        return measureIndex;
+    }
+    
+    public void setMeasureIndex(int[] index) {
+        this.measureIndex = index;
+    }
+
+    public MeasureDesc[] getMeasures() {
+        return measures;
+    }
+
+    public void setMeasures(MeasureDesc[] measures) {
+        this.measures = measures;
+    }
+
+    public String getColumnFamilyName() {
+        return columnFamilyName;
+    }
+
+    public void setColumnFamilyName(String columnFamilyName) {
+        this.columnFamilyName = columnFamilyName;
+    }
+
+    public int findMeasure(FunctionDesc function) {
+        for (int i = 0; i < measures.length; i++) {
+            if (measures[i].getFunction().equals(function)) {
+                return i;
+            }
+        }
+        return -1;
+    }
+
+    public boolean containsMeasure(String refName) {
+        for (String ref : measureRefs) {
+            if (ref.equals(refName))
+                return true;
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((columnFamilyName == null) ? 0 : columnFamilyName.hashCode());
+        result = prime * result + ((qualifier == null) ? 0 : qualifier.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        HBaseColumnDesc other = (HBaseColumnDesc) obj;
+        if (columnFamilyName == null) {
+            if (other.columnFamilyName != null)
+                return false;
+        } else if (!columnFamilyName.equals(other.columnFamilyName))
+            return false;
+        if (qualifier == null) {
+            if (other.qualifier != null)
+                return false;
+        } else if (!qualifier.equals(other.qualifier))
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "HBaseColumnDesc [qualifier=" + qualifier + ", measureRefs=" + Arrays.toString(measureRefs) + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnFamilyDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnFamilyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnFamilyDesc.java
new file mode 100644
index 0000000..73012f3
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnFamilyDesc.java
@@ -0,0 +1,58 @@
+/*
+ * 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.model.v2;
+
+import java.util.Arrays;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class HBaseColumnFamilyDesc {
+
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("columns")
+    private HBaseColumnDesc[] columns;
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public HBaseColumnDesc[] getColumns() {
+        return columns;
+    }
+
+    public void setColumns(HBaseColumnDesc[] columns) {
+        this.columns = columns;
+    }
+
+    @Override
+    public String toString() {
+        return "HBaseColumnFamilyDesc [name=" + name + ", columns=" + Arrays.toString(columns) + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseMappingDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseMappingDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseMappingDesc.java
new file mode 100644
index 0000000..0a15d5f
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseMappingDesc.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.cube.model.v2;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedList;
+
+import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class HBaseMappingDesc {
+
+    @JsonProperty("column_family")
+    private HBaseColumnFamilyDesc[] columnFamily;
+
+    // point to the cube instance which contain this HBaseMappingDesc instance.
+    private CubeDesc cubeRef;
+
+    public Collection<HBaseColumnDesc> findHBaseColumnByFunction(FunctionDesc function) {
+        Collection<HBaseColumnDesc> result = new LinkedList<HBaseColumnDesc>();
+        HBaseMappingDesc hbaseMapping = cubeRef.getHBaseMapping();
+        if (hbaseMapping == null || hbaseMapping.getColumnFamily() == null) {
+            return result;
+        }
+        for (HBaseColumnFamilyDesc cf : hbaseMapping.getColumnFamily()) {
+            for (HBaseColumnDesc c : cf.getColumns()) {
+                for (MeasureDesc m : c.getMeasures()) {
+                    if (m.getFunction().equals(function)) {
+                        result.add(c);
+                    }
+                }
+            }
+        }
+        return result;
+    }
+
+    public CubeDesc getCubeRef() {
+        return cubeRef;
+    }
+
+    public void setCubeRef(CubeDesc cubeRef) {
+        this.cubeRef = cubeRef;
+    }
+
+    public HBaseColumnFamilyDesc[] getColumnFamily() {
+        return columnFamily;
+    }
+
+    public void setColumnFamily(HBaseColumnFamilyDesc[] columnFamily) {
+        this.columnFamily = columnFamily;
+    }
+
+    public void init(CubeDesc cubeDesc) {
+        cubeRef = cubeDesc;
+
+        for (HBaseColumnFamilyDesc cf : columnFamily) {
+            cf.setName(cf.getName().toUpperCase());
+
+            for (HBaseColumnDesc c : cf.getColumns()) {
+                c.setQualifier(c.getQualifier().toUpperCase());
+                StringUtil.toUpperCaseArray(c.getMeasureRefs(), c.getMeasureRefs());
+            }
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "HBaseMappingDesc [columnFamily=" + Arrays.toString(columnFamily) + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HierarchyDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HierarchyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HierarchyDesc.java
new file mode 100644
index 0000000..9dcf05b
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HierarchyDesc.java
@@ -0,0 +1,68 @@
+/*
+ * 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.model.v2;
+
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class HierarchyDesc {
+
+    @JsonProperty("level")
+    private String level;
+    @JsonProperty("column")
+    private String column;
+
+    private TblColRef columnRef;
+
+    public String getLevel() {
+        return level;
+    }
+
+    public void setLevel(String level) {
+        this.level = level;
+    }
+
+    public TblColRef getColumnRef() {
+        return columnRef;
+    }
+
+    public void setColumnRef(TblColRef column) {
+        this.columnRef = column;
+    }
+
+    public String getColumn() {
+        return column;
+    }
+
+    public void setColumn(String columnName) {
+        this.column = columnName;
+    }
+
+    @Override
+    public String toString() {
+        return "HierarchyDesc [level=" + level + ", column=" + column + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyColDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyColDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyColDesc.java
new file mode 100644
index 0000000..86e4a53
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyColDesc.java
@@ -0,0 +1,92 @@
+/*
+ * 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.model.v2;
+
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * @author yangli9
+ * 
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class RowKeyColDesc {
+
+    @JsonProperty("column")
+    private String column;
+    @JsonProperty("length")
+    private int length;
+    @JsonProperty("dictionary")
+    private String dictionary;
+    @JsonProperty("mandatory")
+    private boolean mandatory = false;
+
+    // computed
+    private int bitIndex;
+    private TblColRef colRef;
+
+    public String getDictionary() {
+        return dictionary;
+    }
+
+    public String getColumn() {
+        return column;
+    }
+
+    void setColumn(String column) {
+        this.column = column;
+    }
+
+    public int getLength() {
+        return length;
+    }
+
+    public boolean isMandatory() {
+        return mandatory;
+    }
+
+    public int getBitIndex() {
+        return bitIndex;
+    }
+
+    void setBitIndex(int index) {
+        this.bitIndex = index;
+    }
+
+    public TblColRef getColRef() {
+        return colRef;
+    }
+
+    void setColRef(TblColRef colRef) {
+        this.colRef = colRef;
+    }
+
+    public void setDictionary(String dictionary) {
+        this.dictionary = dictionary;
+    }
+
+    @Override
+    public String toString() {
+        return "RowKeyColDesc [column=" + column + ", length=" + length + ", dictionary=" + dictionary + ", mandatory=" + mandatory + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyDesc.java
new file mode 100644
index 0000000..95249ba
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyDesc.java
@@ -0,0 +1,295 @@
+/*
+ * 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.model.v2;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class RowKeyDesc {
+
+    public static class HierarchyMask {
+        public long fullMask;
+        public long[] allMasks;
+    }
+
+    public static class AggrGroupMask {
+        public AggrGroupMask(int size) {
+            groupOneBitMasks = new long[size];
+        }
+
+        public long groupMask;
+        public long groupOneBitMasks[];
+        public long uniqueMask;
+        public long leftoverMask;
+    }
+
+    @JsonProperty("rowkey_columns")
+    private RowKeyColDesc[] rowkeyColumns;
+    @JsonProperty("aggregation_groups")
+    private String[][] aggregationGroups;
+
+    // computed content
+    private CubeDesc cubeDesc;
+    private Map<TblColRef, RowKeyColDesc> columnMap;
+
+    private long fullMask;
+    private long mandatoryColumnMask;
+    private AggrGroupMask[] aggrGroupMasks;
+    private long aggrGroupFullMask;
+    private long tailMask;
+
+    private List<HierarchyMask> hierarchyMasks;
+
+    public RowKeyColDesc[] getRowKeyColumns() {
+        return rowkeyColumns;
+    }
+
+    // search a specific row key col
+    public int getRowKeyIndexByColumnName(String columnName) {
+        if (this.rowkeyColumns == null)
+            return -1;
+
+        for (int i = 0; i < this.rowkeyColumns.length; ++i) {
+            RowKeyColDesc desc = this.rowkeyColumns[i];
+            if (desc.getColumn().equalsIgnoreCase(columnName)) {
+                return i;
+            }
+        }
+        return -1;
+    }
+
+    public int getNCuboidBuildLevels() {
+        // N aggregation columns requires N levels of cuboid build
+        // - N columns requires N-1 levels build
+        // - zero tail cuboid needs one more additional level
+        Set<String> aggDims = new HashSet<String>();
+        for (String[] aggrGroup : aggregationGroups) {
+            for (String dim : aggrGroup) {
+                aggDims.add(dim);
+            }
+        }
+        return aggDims.size();
+    }
+
+    public String[][] getAggregationGroups() {
+        return aggregationGroups;
+    }
+
+    public CubeDesc getCubeRef() {
+        return cubeDesc;
+    }
+
+    public void setCubeRef(CubeDesc cubeRef) {
+        this.cubeDesc = cubeRef;
+    }
+
+    public long getFullMask() {
+        return fullMask;
+    }
+
+    public long getMandatoryColumnMask() {
+        return mandatoryColumnMask;
+    }
+
+    public long getAggrGroupFullMask() {
+        return aggrGroupFullMask;
+    }
+
+    public AggrGroupMask[] getAggrGroupMasks() {
+        return aggrGroupMasks;
+    }
+
+    public List<HierarchyMask> getHierarchyMasks() {
+        return hierarchyMasks;
+    }
+
+    public long getTailMask() {
+        return tailMask;
+    }
+
+    public int getColumnBitIndex(TblColRef col) {
+        return getColDesc(col).getBitIndex();
+    }
+
+    public int getColumnLength(TblColRef col) {
+        return getColDesc(col).getLength();
+    }
+
+    public String getDictionary(TblColRef col) {
+        return getColDesc(col).getDictionary();
+    }
+
+    private RowKeyColDesc getColDesc(TblColRef col) {
+        RowKeyColDesc desc = columnMap.get(col);
+        if (desc == null)
+            throw new NullPointerException("Column " + col + " does not exist in row key desc");
+        return desc;
+    }
+
+    public boolean isUseDictionary(int index) {
+        String useDictionary = rowkeyColumns[index].getDictionary();
+        return useDictionary(useDictionary);
+    }
+
+    public boolean isUseDictionary(TblColRef col) {
+        String useDictionary = getDictionary(col);
+        return useDictionary(useDictionary);
+    }
+
+    private boolean useDictionary(String useDictionary) {
+        return !StringUtils.isBlank(useDictionary) && !"false".equals(useDictionary);
+    }
+
+    public void init(CubeDesc cube) {
+        setCubeRef(cube);
+        Map<String, TblColRef> colNameAbbr = cube.buildColumnNameAbbreviation();
+
+        buildRowKey(colNameAbbr);
+        buildAggregationGroups(colNameAbbr);
+        buildHierarchyMasks();
+    }
+
+    @Override
+    public String toString() {
+        return "RowKeyDesc [rowkeyColumns=" + Arrays.toString(rowkeyColumns) + ", aggregationGroups=" + Arrays.toString(aggregationGroups) + "]";
+    }
+
+    private void buildRowKey(Map<String, TblColRef> colNameAbbr) {
+        columnMap = new HashMap<TblColRef, RowKeyColDesc>();
+        mandatoryColumnMask = 0;
+
+        for (int i = 0; i < rowkeyColumns.length; i++) {
+            RowKeyColDesc rowKeyColDesc = rowkeyColumns[i];
+            String column = rowKeyColDesc.getColumn();
+            rowKeyColDesc.setColumn(column.toUpperCase());
+            rowKeyColDesc.setBitIndex(rowkeyColumns.length - i - 1);
+            rowKeyColDesc.setColRef(colNameAbbr.get(column));
+            if (rowKeyColDesc.getColRef() == null) {
+                throw new IllegalArgumentException("Cannot find rowkey column " + column + " in cube " + cubeDesc);
+            }
+
+            columnMap.put(rowKeyColDesc.getColRef(), rowKeyColDesc);
+
+            if (rowKeyColDesc.isMandatory()) {
+                mandatoryColumnMask |= 1L << rowKeyColDesc.getBitIndex();
+            }
+        }
+    }
+
+    private void buildAggregationGroups(Map<String, TblColRef> colNameAbbr) {
+        if (aggregationGroups == null) {
+            aggregationGroups = new String[0][];
+        }
+
+        for (int i = 0; i < aggregationGroups.length; i++) {
+            StringUtil.toUpperCaseArray(aggregationGroups[i], this.aggregationGroups[i]);
+        }
+
+        for (int i = 0; i < this.rowkeyColumns.length; i++) {
+            int index = rowkeyColumns[i].getBitIndex();
+            this.fullMask |= 1L << index;
+        }
+
+        this.aggrGroupMasks = new AggrGroupMask[aggregationGroups.length];
+        for (int i = 0; i < this.aggregationGroups.length; i++) {
+            String[] aggGrp = this.aggregationGroups[i];
+            AggrGroupMask mask = new AggrGroupMask(aggGrp.length);
+
+            for (int j = 0; j < aggGrp.length; j++) {
+                TblColRef aggCol = colNameAbbr.get(aggGrp[j].toUpperCase());
+                if (aggCol == null) {
+                    throw new IllegalArgumentException("Can't find aggregation column " + aggGrp[j] + " in  cube " + this.cubeDesc.getName());
+                }
+                Integer index = getColumnBitIndex(aggCol);
+                mask.groupMask |= 1L << index;
+                mask.groupOneBitMasks[j] = 1L << index;
+                this.aggrGroupFullMask |= 1L << index;
+            }
+            this.aggrGroupMasks[i] = mask;
+        }
+
+        this.tailMask = fullMask ^ mandatoryColumnMask ^ aggrGroupFullMask;
+
+        // unique mask = (bits in this group) - (bits in following groups)
+        // leftover mask = (tail bits) + (bits in following groups) - (bits in
+        // this group)
+        for (int i = 0; i < aggrGroupMasks.length; i++) {
+            AggrGroupMask mask = aggrGroupMasks[i];
+
+            mask.uniqueMask = mask.groupMask;
+            for (int j = i + 1; j < aggrGroupMasks.length; j++) {
+                mask.uniqueMask &= ~aggrGroupMasks[j].groupMask;
+            }
+
+            mask.leftoverMask = tailMask;
+            for (int j = i + 1; j < aggrGroupMasks.length; j++) {
+                mask.leftoverMask |= aggrGroupMasks[j].groupMask;
+            }
+            mask.leftoverMask &= ~mask.groupMask;
+        }
+    }
+
+    private void buildHierarchyMasks() {
+        this.hierarchyMasks = new ArrayList<HierarchyMask>();
+
+        for (DimensionDesc dimension : this.cubeDesc.getDimensions()) {
+            HierarchyDesc[] hierarchies = dimension.getHierarchy();
+            if (hierarchies == null || hierarchies.length == 0)
+                continue;
+
+            HierarchyMask mask = new HierarchyMask();
+            ArrayList<Long> allMaskList = new ArrayList<Long>();
+            for (int i = 0; i < hierarchies.length; i++) {
+                TblColRef hColumn = hierarchies[i].getColumnRef();
+                Integer index = getColumnBitIndex(hColumn);
+                long bit = 1L << index;
+
+                if ((tailMask & bit) > 0)
+                    continue; // ignore levels in tail, they don't participate
+                              // aggregation group combination anyway
+
+                mask.fullMask |= bit;
+                allMaskList.add(mask.fullMask);
+            }
+
+            mask.allMasks = new long[allMaskList.size()];
+            for (int i = 0; i < allMaskList.size(); i++)
+                mask.allMasks[i] = allMaskList.get(i);
+
+            this.hierarchyMasks.add(mask);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java
index 2f8fc7b..7d7710c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java
@@ -19,9 +19,8 @@
 package org.apache.kylin.cube.model.validation;
 
 import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.validation.rule.AggregationGroupSizeRule;
+import org.apache.kylin.cube.model.validation.rule.AggregationGroupRule;
 import org.apache.kylin.cube.model.validation.rule.FunctionRule;
-import org.apache.kylin.cube.model.validation.rule.MandatoryColumnRule;
 import org.apache.kylin.cube.model.validation.rule.RowKeyAttrRule;
 
 /**
@@ -32,16 +31,14 @@ import org.apache.kylin.cube.model.validation.rule.RowKeyAttrRule;
  */
 public class CubeMetadataValidator {
     @SuppressWarnings("unchecked")
-    private IValidatorRule<CubeDesc>[] rules = new IValidatorRule[] { new FunctionRule(), new AggregationGroupSizeRule(), new MandatoryColumnRule(), new RowKeyAttrRule() };
+    private IValidatorRule<CubeDesc>[] rules = new IValidatorRule[] { new FunctionRule(), new AggregationGroupRule(), new RowKeyAttrRule() };
 
     public ValidateContext validate(CubeDesc cube) {
         return validate(cube, false);
     }
 
     /**
-     * @param cubeDesc
-     * @param inject
-     *            inject error into cube desc
+     * @param inject    inject error into cube desc
      * @return
      */
     public ValidateContext validate(CubeDesc cube, boolean inject) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/ValidateContext.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/ValidateContext.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/ValidateContext.java
index e33ec19..7f1f617 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/ValidateContext.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/ValidateContext.java
@@ -25,9 +25,6 @@ import java.util.List;
 
 /**
  * Context. Supply all dependent objects for validator
- * 
- * @author jianliu
- * 
  */
 public class ValidateContext {
     private List<Result> results = new ArrayList<ValidateContext.Result>();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2d5f725b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupOverlapRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupOverlapRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupOverlapRule.java
new file mode 100644
index 0000000..f63e5f1
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupOverlapRule.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.cube.model.validation.rule;
+
+import org.apache.kylin.cube.model.AggregationGroup;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.validation.IValidatorRule;
+import org.apache.kylin.cube.model.validation.ResultLevel;
+import org.apache.kylin.cube.model.validation.ValidateContext;
+
+/**
+ * find forbid overlaps in each AggregationGroup
+ *  the include dims in AggregationGroup must contain all mandatory, hierarchy and joint
+ */
+public class AggregationGroupOverlapRule implements IValidatorRule<CubeDesc> {
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * org.apache.kylin.metadata.validation.IValidatorRule#validate(java.lang.Object
+     * , org.apache.kylin.metadata.validation.ValidateContext)
+     */
+    @Override
+    public void validate(CubeDesc cube, ValidateContext context) {
+
+        int index = 0;
+        for (AggregationGroup agg : cube.getAggregationGroups()) {
+            
+            if ((agg.getMandatoryColumnMask() & agg.getHierarchyDimsMask()) != 0) {
+                context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " mandatory dims overlap with hierarchy dims");
+            }
+            if ((agg.getMandatoryColumnMask() & agg.getJointDimsMask()) != 0) {
+                context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " mandatory dims overlap with joint dims");
+            }
+
+            int jointDimNum = 0;
+            for (Long joint : agg.getJointDims()) {
+                jointDimNum += Long.bitCount(joint);
+                if (jointDimNum < 2) {
+                    context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " require at least 2 dims in a joint");
+                }
+
+                int overlapHierarchies = 0;
+                for (AggregationGroup.HierarchyMask mask : agg.getHierarchyMasks()) {
+                    long share = (joint & mask.fullMask);
+                    if (share != 0) {
+                        overlapHierarchies++;
+                    }
+                    if (Long.bitCount(share) > 1) {
+                        context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " joint columns overlap with more than 1 dim in same hierarchy");
+                    }
+                }
+
+                if (overlapHierarchies > 1) {
+                    context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " joint columns overlap with more than 1 hierarchies");
+                }
+            }
+
+            if (jointDimNum != Long.bitCount(agg.getJointDimsMask())) {
+                context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " a dim exist in more than 1 joint");
+            }
+
+            index++;
+        }
+
+    }
+
+}