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/12/04 10:02:36 UTC

[4/7] kylin git commit: KYLIN-242 refactor cuboid and cuboid scheduer to support white list by addming joint semantic

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java
new file mode 100644
index 0000000..17f5b2d
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java
@@ -0,0 +1,185 @@
+/*
+ * 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 java.util.Arrays;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.kylin.common.KylinConfig;
+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 AggregationGroupRule implements IValidatorRule<CubeDesc> {
+
+    @Override
+    public void validate(CubeDesc cube, ValidateContext context) {
+        inner(cube, context);
+    }
+
+    private int count(String[][] input) {
+        if (input == null) {
+            return 0;
+        } else {
+            int count = 0;
+            for (String[] x : input) {
+                count += count(x);
+            }
+            return count;
+        }
+    }
+
+    private int count(String[] input) {
+        return input == null ? 0 : input.length;
+    }
+
+    private void inner(CubeDesc cube, ValidateContext context) {
+        int maxSize = getMaxAgrGroupSize();
+
+        int index = 0;
+        for (AggregationGroup agg : cube.getAggregationGroups()) {
+            if (agg.getIncludes() == null) {
+                context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " includes field not set");
+                continue;
+            }
+
+            if (agg.getSelectRule() == null) {
+                context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " select rule field not set");
+                continue;
+            }
+
+            Set<String> includeDims = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            if (agg.getIncludes() != null) {
+                for (String include : agg.getIncludes()) {
+                    includeDims.add(include);
+                }
+            }
+
+            Set<String> mandatoryDims = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            if (agg.getSelectRule().mandatory_dims != null) {
+                for (String m : agg.getSelectRule().mandatory_dims) {
+                    mandatoryDims.add(m);
+                }
+            }
+
+            Set<String> hierarchyDims = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            if (agg.getSelectRule().hierarchy_dims != null) {
+                for (String[] ss : agg.getSelectRule().hierarchy_dims) {
+                    for (String s : ss)
+                        hierarchyDims.add(s);
+                }
+            }
+
+            Set<String> jointDims = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            if (agg.getSelectRule().joint_dims != null) {
+                for (String[] ss : agg.getSelectRule().joint_dims) {
+                    for (String s : ss)
+                        jointDims.add(s);
+                }
+            }
+
+            if (!includeDims.containsAll(mandatoryDims) || !includeDims.containsAll(hierarchyDims) || !includeDims.containsAll(jointDims)) {
+                context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " Include dims not containing all the used dims");
+                continue;
+            }
+
+            Set<String> normalDims = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            normalDims.addAll(includeDims);
+            normalDims.removeAll(mandatoryDims);
+            normalDims.removeAll(hierarchyDims);
+            normalDims.removeAll(jointDims);
+
+            int normalDimSize = normalDims.size();
+            int hierarchySize = count(agg.getSelectRule().hierarchy_dims);
+            int jointSize = count(agg.getSelectRule().joint_dims);
+
+            if (normalDimSize + hierarchySize + jointSize > maxSize) {
+                context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " has too many dimensions");
+                continue;
+            }
+
+            if (CollectionUtils.containsAny(mandatoryDims, hierarchyDims)) {
+                context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " mandatory dims overlap with hierarchy dims");
+                continue;
+            }
+            if (CollectionUtils.containsAny(mandatoryDims, jointDims)) {
+                context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " mandatory dims overlap with joint dims");
+                continue;
+            }
+
+            int jointDimNum = 0;
+            if (agg.getSelectRule().joint_dims != null) {
+                for (String[] joints : agg.getSelectRule().joint_dims) {
+
+                    Set<String> oneJoint = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+                    for (String s : joints) {
+                        oneJoint.add(s);
+                    }
+
+                    if (oneJoint.size() < 2) {
+                        context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " require at least 2 dims in a joint");
+                        continue;
+                    }
+                    jointDimNum += oneJoint.size();
+
+                    int overlapHierarchies = 0;
+                    if (agg.getSelectRule().hierarchy_dims != null) {
+                        for (String[] oneHierarchy : agg.getSelectRule().hierarchy_dims) {
+                            Set<String> share = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+                            share.addAll(CollectionUtils.intersection(oneJoint, Arrays.asList(oneHierarchy)));
+
+                            if (!share.isEmpty()) {
+                                overlapHierarchies++;
+                            }
+                            if (share.size() > 1) {
+                                context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " joint columns overlap with more than 1 dim in same hierarchy");
+                                continue;
+                            }
+                        }
+
+                        if (overlapHierarchies > 1) {
+                            context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " joint columns overlap with more than 1 hierarchies");
+                            continue;
+                        }
+                    }
+                }
+
+                if (jointDimNum != jointDims.size()) {
+                    context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " a dim exist in more than one joint");
+                    continue;
+                }
+            }
+
+            index++;
+        }
+    }
+
+    protected int getMaxAgrGroupSize() {
+        String size = KylinConfig.getInstanceFromEnv().getProperty(KEY_MAX_AGR_GROUP_SIZE, String.valueOf(DEFAULT_MAX_AGR_GROUP_SIZE));
+        return Integer.parseInt(size);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupSizeRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupSizeRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupSizeRule.java
deleted file mode 100644
index e37b9a9..0000000
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupSizeRule.java
+++ /dev/null
@@ -1,66 +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.model.validation.rule;
-
-import org.apache.kylin.common.KylinConfig;
-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;
-
-/**
- * Rule to validate: 1. The aggregationGroup size must be less than 20
- * 
- * @author jianliu
- * 
- */
-public class AggregationGroupSizeRule 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) {
-        innerValidateMaxSize(cube, context);
-    }
-
-    /**
-     * @param cube
-     * @param context
-     */
-    private void innerValidateMaxSize(CubeDesc cube, ValidateContext context) {
-        int maxSize = getMaxAgrGroupSize();
-        String[][] groups = cube.getRowkey().getAggregationGroups();
-        for (int i = 0; i < groups.length; i++) {
-            String[] group = groups[i];
-            if (group.length >= maxSize) {
-                context.addResult(ResultLevel.ERROR, "Length of the number " + i + " aggregation group's length should be less than " + maxSize);
-            }
-        }
-    }
-
-    protected int getMaxAgrGroupSize() {
-        String size = KylinConfig.getInstanceFromEnv().getProperty(KEY_MAX_AGR_GROUP_SIZE, String.valueOf(DEFAULT_MAX_AGR_GROUP_SIZE));
-        return Integer.parseInt(size);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/IKylinValidationConstants.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/IKylinValidationConstants.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/IKylinValidationConstants.java
index 52e5b24..80d4676 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/IKylinValidationConstants.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/IKylinValidationConstants.java
@@ -20,13 +20,9 @@ package org.apache.kylin.cube.model.validation.rule;
 
 import org.apache.kylin.metadata.MetadataConstants;
 
-/**
- * @author jianliu
- * 
- */
 public interface IKylinValidationConstants extends MetadataConstants {
 
-    public static final int DEFAULT_MAX_AGR_GROUP_SIZE = 20;
+    public static final int DEFAULT_MAX_AGR_GROUP_SIZE = 12;
     public static final String KEY_MAX_AGR_GROUP_SIZE = "rule_max.arggregation.group.size";
     public static final String KEY_IGNORE_UNKNOWN_FUNC = "rule_ignore_unknown_func";
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/MandatoryColumnRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/MandatoryColumnRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/MandatoryColumnRule.java
deleted file mode 100644
index 4585900..0000000
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/MandatoryColumnRule.java
+++ /dev/null
@@ -1,75 +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.model.validation.rule;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.commons.lang.ArrayUtils;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.RowKeyColDesc;
-import org.apache.kylin.cube.model.validation.IValidatorRule;
-import org.apache.kylin.cube.model.validation.ResultLevel;
-import org.apache.kylin.cube.model.validation.ValidateContext;
-
-/**
- * Validate that mandatory column must NOT appear in aggregation group.
- * 
- * @author jianliu
- * 
- */
-public class MandatoryColumnRule 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) {
-        Set<String> mands = new HashSet<String>();
-        RowKeyColDesc[] cols = cube.getRowkey().getRowKeyColumns();
-        if (cols == null || cols.length == 0) {
-            return;
-        }
-        for (int i = 0; i < cols.length; i++) {
-            RowKeyColDesc rowKeyColDesc = cols[i];
-            if (rowKeyColDesc.isMandatory()) {
-                mands.add(rowKeyColDesc.getColumn());
-            }
-        }
-        if (mands.isEmpty()) {
-            return;
-        }
-        String[][] groups = cube.getRowkey().getAggregationGroups();
-        for (int i = 0; i < groups.length; i++) {
-            String[] group = groups[i];
-            for (int j = 0; j < group.length; j++) {
-                String col = group[j];
-                if (mands.contains(col)) {
-                    context.addResult(ResultLevel.ERROR, "mandatory column " + col + " must not be in aggregation group [" + ArrayUtils.toString(group) + "]");
-                }
-            }
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/RowKeyAttrRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/RowKeyAttrRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/RowKeyAttrRule.java
index e90f6cd..932754c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/RowKeyAttrRule.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/RowKeyAttrRule.java
@@ -18,7 +18,6 @@
 
 package org.apache.kylin.cube.model.validation.rule;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.RowKeyColDesc;
 import org.apache.kylin.cube.model.RowKeyDesc;
@@ -28,19 +27,9 @@ import org.apache.kylin.cube.model.validation.ValidateContext;
 
 /**
  * Validate that only one of "length" and "dictionary" appears on rowkey_column
- * 
- * @author jianliu
- * 
  */
 public class RowKeyAttrRule 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) {
         RowKeyDesc row = cube.getRowkey();
@@ -50,25 +39,21 @@ public class RowKeyAttrRule implements IValidatorRule<CubeDesc> {
         }
 
         RowKeyColDesc[] rcd = row.getRowKeyColumns();
-        if (rcd == null) {
+        if (rcd == null || rcd.length == 0) {
             context.addResult(ResultLevel.ERROR, "Rowkey columns do not exist");
             return;
         }
-        if (rcd.length == 0) {
-            context.addResult(ResultLevel.ERROR, "Rowkey columns is empty");
-            return;
-        }
 
         for (int i = 0; i < rcd.length; i++) {
             RowKeyColDesc rd = rcd[i];
-            if (rd.getLength() != 0 && (!StringUtils.isEmpty(rd.getDictionary()) && !rd.getDictionary().equals("false"))) {
-                context.addResult(ResultLevel.ERROR, "Rowkey column " + rd.getColumn() + " must not have both 'length' and 'dictionary' attribute");
+            if (rd.getColumn() == null || rd.getColumn().length() == 0) {
+                context.addResult(ResultLevel.ERROR, "Rowkey column empty");
+                continue;
             }
-            if (rd.getLength() == 0 && (StringUtils.isEmpty(rd.getDictionary()) || rd.getDictionary().equals("false"))) {
-                context.addResult(ResultLevel.ERROR, "Rowkey column " + rd.getColumn() + " must not have both 'length' and 'dictionary' empty");
+            if (!rd.getEncoding().equalsIgnoreCase("dict") && !rd.getEncoding().toLowerCase().startsWith("fixed_length")) {
+                context.addResult(ResultLevel.ERROR, "Rowkey column " + rd.getColumn() + " encoding not dict nor fixed_length");
             }
         }
 
     }
-
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/upgrade/V2/CubeDescUpgraderV2.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/upgrade/V2/CubeDescUpgraderV2.java b/core-cube/src/main/java/org/apache/kylin/cube/upgrade/V2/CubeDescUpgraderV2.java
new file mode 100644
index 0000000..6366881
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/upgrade/V2/CubeDescUpgraderV2.java
@@ -0,0 +1,287 @@
+/*
+ * 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.upgrade.V2;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.JsonSerializer;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.persistence.Serializer;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.cube.model.AggregationGroup;
+import org.apache.kylin.cube.model.SelectRule;
+import org.apache.kylin.cube.model.v2.CubeDesc;
+import org.apache.kylin.cube.model.v2.DimensionDesc;
+import org.apache.kylin.cube.model.v2.HBaseMappingDesc;
+import org.apache.kylin.cube.model.v2.RowKeyColDesc;
+import org.apache.kylin.cube.model.v2.RowKeyDesc;
+
+import com.google.common.collect.Lists;
+
+public class CubeDescUpgraderV2 {
+
+    @SuppressWarnings("unused")
+    private static final Log logger = LogFactory.getLog(CubeDescUpgraderV2.class);
+    private static final Serializer<CubeDesc> oldCubeDescSerializer = new JsonSerializer<CubeDesc>(CubeDesc.class);
+
+    private String resourcePath;
+
+    private List<String[]> oldHierarchies = Lists.newArrayList();
+    private List<String> oldMandatories = Lists.newArrayList();
+    private String[][] oldAggGroup = null;
+    private Set<String> allRowKeyCols = newIgnoreCaseSet(null);
+
+    public CubeDescUpgraderV2(String resourcePath) {
+        this.resourcePath = resourcePath;
+    }
+
+    public org.apache.kylin.cube.model.CubeDesc upgrade() throws IOException {
+        CubeDesc oldModel = loadOldCubeDesc(resourcePath);
+
+        org.apache.kylin.cube.model.CubeDesc newModel = new org.apache.kylin.cube.model.CubeDesc();
+        copyUnChangedProperties(oldModel, newModel);
+        upgradeDimension(oldModel, newModel);
+        upgradeRowKey(oldModel, newModel);
+        upgradeHBaseMapping(oldModel, newModel);
+        upgradeAggregationGroup(newModel);//must do at last
+
+        return newModel;
+    }
+
+    private CubeDesc loadOldCubeDesc(String path) throws IOException {
+        ResourceStore store = getStore();
+
+        CubeDesc ndesc = store.getResource(path, CubeDesc.class, oldCubeDescSerializer);
+
+        if (StringUtils.isBlank(ndesc.getName())) {
+            throw new IllegalStateException("CubeDesc name must not be blank");
+        }
+
+        return ndesc;
+    }
+
+    private Set<String> newIgnoreCaseSet(Set<String> input) {
+        Set<String> ret = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+        if (input != null)
+            ret.addAll(input);
+        return ret;
+    }
+
+    private String[] toArray(Set<String> input) {
+        return input.toArray(new String[input.size()]);
+    }
+
+    private boolean rowKeyColExistsInMultipleAggGroup() {
+        if (oldAggGroup == null)
+            return false;
+
+        int total = 0;
+        Set<String> overall = newIgnoreCaseSet(null);
+        for (String[] group : oldAggGroup) {
+            Set<String> temp = newIgnoreCaseSet(null);
+            for (String entry : group) {
+
+                overall.add(entry);
+                temp.add(entry);
+            }
+            total += temp.size();
+        }
+        return overall.size() != total;
+    }
+
+    private void upgradeAggregationGroup(org.apache.kylin.cube.model.CubeDesc newModel) {
+
+        List<AggregationGroup> aggs = Lists.newArrayList();
+        if (oldAggGroup == null || oldAggGroup.length == 0) {
+            oldAggGroup = new String[1][];
+            oldAggGroup[0] = toArray(allRowKeyCols);
+        }
+
+        if (rowKeyColExistsInMultipleAggGroup()) {
+            throw new IllegalArgumentException("rowKeyColExistsInMultipleAggGroup!");
+        }
+
+        Set<String> visited = newIgnoreCaseSet(null);
+
+        for (String[] group : oldAggGroup) {
+            AggregationGroup agg = new AggregationGroup();
+
+            Set<String> remaining = newIgnoreCaseSet(allRowKeyCols);
+            remaining.removeAll(visited);
+
+            Set<String> joint = newIgnoreCaseSet(remaining);
+            joint.removeAll(oldMandatories);
+
+            Set<String> groupAsSet = newIgnoreCaseSet(null);
+            for (String entry : group) {
+                groupAsSet.add(entry);
+            }
+            visited.addAll(groupAsSet);
+            joint.removeAll(groupAsSet);
+
+            List<String> mandatories = Lists.newArrayList();
+            List<String[]> hierarchies = Lists.newArrayList();
+
+            for (String s : oldMandatories) {
+                mandatories.add(s);
+            }
+
+            for (String[] h : oldHierarchies) {
+                if (groupAsSet.containsAll(Arrays.asList(h))) {
+                    hierarchies.add(h);
+                }
+            }
+
+            agg.setIncludes(toArray(remaining));
+
+            SelectRule selectRule = new SelectRule();
+            selectRule.hierarchy_dims = hierarchies.toArray(new String[hierarchies.size()][]);
+            if (joint.size() != 0) {
+                selectRule.joint_dims = new String[1][];
+                selectRule.joint_dims[0] = joint.toArray(new String[joint.size()]);
+            } else {
+                selectRule.joint_dims = new String[0][];
+            }
+            selectRule.mandatory_dims = mandatories.toArray(new String[mandatories.size()]);
+            agg.setSelectRule(selectRule);
+
+            aggs.add(agg);
+
+        }
+        newModel.setAggregationGroups(aggs);
+    }
+
+    private void upgradeDimension(CubeDesc oldModel, org.apache.kylin.cube.model.CubeDesc newModel) {
+        List<DimensionDesc> oldDimensions = oldModel.getDimensions();
+        if (oldDimensions == null) {
+            throw new IllegalArgumentException("dimensions is null");
+        }
+        List<org.apache.kylin.cube.model.DimensionDesc> newDimensions = Lists.newArrayList();
+
+        for (DimensionDesc oldDim : oldDimensions) {
+            if (oldDim.isDerived()) {
+                org.apache.kylin.cube.model.DimensionDesc newDim = new org.apache.kylin.cube.model.DimensionDesc();
+
+                newDim.setName(oldDim.getName());
+                newDim.setTable(oldDim.getTable());
+                newDim.setColumn("{FK}");
+                newDim.setDerived(oldDim.getDerived());
+
+                newDimensions.add(newDim);
+            } else {
+                if (oldDim.isHierarchy()) {
+                    oldHierarchies.add(oldDim.getColumn());
+                }
+
+                for (String columnStr : oldDim.getColumn()) {
+                    org.apache.kylin.cube.model.DimensionDesc newDim = new org.apache.kylin.cube.model.DimensionDesc();
+
+                    newDim.setName(oldDim.getName());
+                    newDim.setTable(oldDim.getTable());
+                    newDim.setColumn(columnStr);
+                    newDim.setDerived(null);
+
+                    newDimensions.add(newDim);
+                }
+            }
+        }
+
+        newModel.setDimensions(newDimensions);
+    }
+
+    private void upgradeRowKey(CubeDesc oldModel, org.apache.kylin.cube.model.CubeDesc newModel) {
+        RowKeyDesc oldRowKey = oldModel.getRowkey();
+        if (oldModel == null) {
+            throw new IllegalArgumentException("RowKeyDesc is null");
+        }
+
+        if (oldRowKey.getRowKeyColumns() == null) {
+            throw new IllegalArgumentException("RowKeyDesc.getRowKeyColumns is null");
+        }
+
+        org.apache.kylin.cube.model.RowKeyDesc newRowKey = new org.apache.kylin.cube.model.RowKeyDesc();
+        org.apache.kylin.cube.model.RowKeyColDesc[] cols = new org.apache.kylin.cube.model.RowKeyColDesc[oldRowKey.getRowKeyColumns().length];
+        int index = 0;
+        for (RowKeyColDesc oldRowKeyCol : oldRowKey.getRowKeyColumns()) {
+            org.apache.kylin.cube.model.RowKeyColDesc newRowKeyCol = new org.apache.kylin.cube.model.RowKeyColDesc();
+
+            allRowKeyCols.add(oldRowKeyCol.getColumn());
+            if (oldRowKeyCol.isMandatory()) {
+                oldMandatories.add(oldRowKeyCol.getColumn());
+            }
+
+            newRowKeyCol.setColumn(oldRowKeyCol.getColumn());
+            if (oldRowKeyCol.getDictionary() != null && "true".equalsIgnoreCase(oldRowKeyCol.getDictionary())) {
+                newRowKeyCol.setEncoding("dict");
+            } else if (oldRowKeyCol.getLength() > 0) {
+                newRowKeyCol.setEncoding("fixed_length:" + oldRowKeyCol.getLength());
+            } else {
+                throw new IllegalArgumentException("Unknow encoding: Dictionary " + oldRowKeyCol.getDictionary() + ", length: " + oldRowKeyCol.getLength());
+            }
+            cols[index++] = newRowKeyCol;
+        }
+        oldAggGroup = oldRowKey.getAggregationGroups();
+
+        newRowKey.setRowkeyColumns(cols);
+        newModel.setRowkey(newRowKey);
+    }
+
+    private void upgradeHBaseMapping(CubeDesc oldModel, org.apache.kylin.cube.model.CubeDesc newModel) {
+        HBaseMappingDesc hbaseMappingDesc = oldModel.getHBaseMapping();
+        try {
+
+            ByteArrayOutputStream os = new ByteArrayOutputStream();
+            JsonUtil.writeValueIndent(os, hbaseMappingDesc);
+            byte[] blob = os.toByteArray();
+            ByteArrayInputStream is = new ByteArrayInputStream(blob);
+            org.apache.kylin.cube.model.HBaseMappingDesc newHBaseMappingDesc = JsonUtil.readValue(is, org.apache.kylin.cube.model.HBaseMappingDesc.class);
+            newModel.setHbaseMapping(newHBaseMappingDesc);
+
+        } catch (IOException e) {
+            throw new RuntimeException("error when copying HBaseMappingDesc");
+        }
+    }
+
+    private void copyUnChangedProperties(CubeDesc oldModel, org.apache.kylin.cube.model.CubeDesc newModel) {
+        newModel.setUuid(oldModel.getUuid());
+        newModel.setName(oldModel.getName());
+        newModel.setDescription(oldModel.getDescription());
+        newModel.setMeasures(oldModel.getMeasures());
+        newModel.setNullStrings(oldModel.getNullStrings());
+        newModel.setModelName(oldModel.getModelName());
+        newModel.setNotifyList(oldModel.getNotifyList());
+        newModel.setLastModified(oldModel.getLastModified());
+        newModel.setStorageType(oldModel.getStorageType());
+        newModel.setEngineType(oldModel.getEngineType());
+    }
+
+    protected static ResourceStore getStore() {
+        return ResourceStore.getStore(KylinConfig.getInstanceFromEnv());
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/upgrade/V2/CubeMetadataUpgradeV2.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/upgrade/V2/CubeMetadataUpgradeV2.java b/core-cube/src/main/java/org/apache/kylin/cube/upgrade/V2/CubeMetadataUpgradeV2.java
new file mode 100644
index 0000000..4d3fce9
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/upgrade/V2/CubeMetadataUpgradeV2.java
@@ -0,0 +1,177 @@
+/*
+ * 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.upgrade.V2;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.cube.CubeDescManager;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.metadata.MetadataConstants;
+import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.project.ProjectManager;
+
+import com.google.common.collect.Lists;
+
+/**
+ * back in 1.x branch there was a CubeMetadataUpgrade which is actually CubeMetadataUpgradeV1,
+ * that upgrades metadata store from v1(prior kylin 0.7) to v2.
+ * the major difference is that we split cube desc to cube desc + model desc
+ * 
+ * this CubeMetadataUpgradeV2 upgrades metadata store from v2(prior kylin 2.1) to v3
+ * the major different is a brand new definition of partial cubes to allow users to select 
+ * cuboids more flexibly 
+ */
+public class CubeMetadataUpgradeV2 {
+
+    private KylinConfig config = null;
+    private ResourceStore store;
+
+    private List<String> updatedResources = Lists.newArrayList();
+    private List<String> errorMsgs = Lists.newArrayList();
+
+    private static final Log logger = LogFactory.getLog(CubeMetadataUpgradeV2.class);
+
+    public CubeMetadataUpgradeV2(String newMetadataUrl) {
+        KylinConfig.destoryInstance();
+        System.setProperty(KylinConfig.KYLIN_CONF, newMetadataUrl);
+        KylinConfig.getInstanceFromEnv().setMetadataUrl(newMetadataUrl);
+
+        config = KylinConfig.getInstanceFromEnv();
+        store = getStore();
+    }
+
+    public void upgrade() {
+
+        upgradeCubeDesc();
+        verify();
+    }
+
+    public void verify() {
+        MetadataManager.clearCache();
+        MetadataManager.getInstance(config);
+        CubeDescManager.clearCache();
+        CubeDescManager.getInstance(config);
+        CubeManager.clearCache();
+        CubeManager.getInstance(config);
+        ProjectManager.clearCache();
+        ProjectManager.getInstance(config);
+        //cleanup();
+    }
+
+    private List<String> listResourceStore(String pathRoot) {
+        List<String> paths = null;
+        try {
+            paths = store.collectResourceRecursively(pathRoot, MetadataConstants.FILE_SURFIX);
+        } catch (IOException e1) {
+            e1.printStackTrace();
+            errorMsgs.add("Get IOException when scan resource store at: " + ResourceStore.CUBE_DESC_RESOURCE_ROOT);
+        }
+
+        return paths;
+    }
+
+    private void upgradeCubeDesc() {
+        logger.info("Reloading Cube Metadata from folder " + store.getReadableResourcePath(ResourceStore.CUBE_DESC_RESOURCE_ROOT));
+
+        List<String> paths = listResourceStore(ResourceStore.CUBE_DESC_RESOURCE_ROOT);
+        for (String path : paths) {
+
+            try {
+                CubeDescUpgraderV2 upgrade = new CubeDescUpgraderV2(path);
+                CubeDesc ndesc = upgrade.upgrade();
+                ndesc.setSignature(ndesc.calculateSignature());
+
+                getStore().putResource(ndesc.getResourcePath(), ndesc, CubeDescManager.CUBE_DESC_SERIALIZER);
+                updatedResources.add(ndesc.getResourcePath());
+            } catch (IOException e) {
+                e.printStackTrace();
+                errorMsgs.add("Upgrade CubeDesc at '" + path + "' failed: " + e.getLocalizedMessage());
+            }
+        }
+
+    }
+
+    private ResourceStore getStore() {
+        return ResourceStore.getStore(config);
+    }
+
+    public static void main(String[] args) {
+
+        if (!(args != null && (args.length == 1 || args.length == 2))) {
+            System.out.println("Usage: java CubeMetadataUpgrade <metadata_export_folder> <verify>; e.g, /export/kylin/meta ");
+            return;
+        }
+
+        String exportFolder = args[0];
+        boolean verify = false;
+        if (args.length == 2 && "verify".equals(args[1])) {
+            System.out.println("Only verify the metadata in folder " + exportFolder);
+            verify = true;
+        }
+
+        CubeMetadataUpgradeV2 instance = null;
+        if (verify) {
+            instance = new CubeMetadataUpgradeV2(exportFolder);
+            instance.verify();
+        } else {
+            File oldMetaFolder = new File(exportFolder);
+            if (!oldMetaFolder.exists()) {
+                System.out.println("Provided folder doesn't exist: '" + exportFolder + "'");
+                return;
+            }
+
+            if (!oldMetaFolder.isDirectory()) {
+                System.out.println("Provided folder is not a directory: '" + exportFolder + "'");
+                return;
+            }
+
+            String newMetadataUrl = oldMetaFolder.getAbsolutePath() + "_v3";//upgrades metadata store to v3 format
+            try {
+                FileUtils.deleteDirectory(new File(newMetadataUrl));
+                FileUtils.copyDirectory(oldMetaFolder, new File(newMetadataUrl));
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+
+            instance = new CubeMetadataUpgradeV2(newMetadataUrl);
+            instance.upgrade();
+            logger.info("=================================================================");
+            logger.info("Run CubeMetadataUpgrade completed;");
+
+        }
+
+        logger.info("=================================================================");
+        if (instance.errorMsgs.size() > 0) {
+            logger.info("Here are the error/warning messages, you may need check:");
+            for (String s : instance.errorMsgs) {
+                logger.warn(s);
+            }
+        } else {
+            logger.info("No error or warning messages; The migration is success.");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java b/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
index 0cfd020..ada31b6 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
@@ -34,7 +34,6 @@
 package org.apache.kylin.cube.util;
 
 import java.io.IOException;
-import java.util.BitSet;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
@@ -42,14 +41,6 @@ import java.util.Map;
 
 import javax.annotation.Nullable;
 
-import com.google.common.base.Function;
-import com.google.common.collect.Collections2;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hasher;
-import com.google.common.hash.Hashing;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.hll.HyperLogLogPlusCounter;
 import org.apache.kylin.common.util.ByteArray;
@@ -59,12 +50,25 @@ import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.cuboid.CuboidScheduler;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
-import org.apache.kylin.dict.*;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.dict.DictionaryGenerator;
+import org.apache.kylin.dict.DictionaryInfo;
+import org.apache.kylin.dict.DictionaryManager;
+import org.apache.kylin.dict.IterableDictionaryValueEnumerator;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.source.ReadableTable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+
 /**
  */
 public class CubingUtils {
@@ -82,8 +86,7 @@ public class CubingUtils {
             @Nullable
             @Override
             public Integer[] apply(@Nullable Long cuboidId) {
-                BitSet bitSet = BitSet.valueOf(new long[] { cuboidId });
-                Integer[] result = new Integer[bitSet.cardinality()];
+                Integer[] result = new Integer[Long.bitCount(cuboidId)];
 
                 long mask = Long.highestOneBit(baseCuboidId);
                 int position = 0;
@@ -100,8 +103,7 @@ public class CubingUtils {
         final Map<Long, HyperLogLogPlusCounter> result = Maps.newHashMapWithExpectedSize(allCuboidIds.size());
         for (Long cuboidId : allCuboidIds) {
             result.put(cuboidId, new HyperLogLogPlusCounter(14));
-            BitSet bitSet = BitSet.valueOf(new long[] { cuboidId });
-            Integer[] cuboidBitSet = new Integer[bitSet.cardinality()];
+            Integer[] cuboidBitSet = new Integer[Long.bitCount(cuboidId)];
 
             long mask = Long.highestOneBit(baseCuboidId);
             int position = 0;

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/test/java/org/apache/kylin/cube/AggregationGroupRuleTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/AggregationGroupRuleTest.java b/core-cube/src/test/java/org/apache/kylin/cube/AggregationGroupRuleTest.java
new file mode 100644
index 0000000..2b75d48
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/cube/AggregationGroupRuleTest.java
@@ -0,0 +1,127 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.validation.IValidatorRule;
+import org.apache.kylin.cube.model.validation.ValidateContext;
+import org.apache.kylin.cube.model.validation.rule.AggregationGroupRule;
+import org.junit.Test;
+
+public class AggregationGroupRuleTest {
+
+    @Test
+    public void testGoodDesc() throws IOException {
+        AggregationGroupRule rule = getAggregationGroupRule();
+
+        for (File f : new File("../examples/test_case_data/localmeta/cube_desc/").listFiles()) {
+            CubeDesc desc = JsonUtil.readValue(new FileInputStream(f), CubeDesc.class);
+            ValidateContext vContext = new ValidateContext();
+            rule.validate(desc, vContext);
+            vContext.print(System.out);
+            assertTrue(vContext.getResults().length == 0);
+        }
+    }
+
+    @Test
+    public void testGoodBecomeBadDesc() throws IOException {
+        AggregationGroupRule rule = new AggregationGroupRule() {
+            @Override
+            protected int getMaxAgrGroupSize() {
+                return 4;
+            }
+        };
+
+        for (File f : new File("../examples/test_case_data/localmeta/cube_desc/").listFiles()) {
+            System.out.println(f.getName());
+            CubeDesc desc = JsonUtil.readValue(new FileInputStream(f), CubeDesc.class);
+            ValidateContext vContext = new ValidateContext();
+            rule.validate(desc, vContext);
+            vContext.print(System.out);
+            assertEquals(1, vContext.getResults().length);
+            assertEquals("Aggregation group 0 has too many dimensions", (vContext.getResults()[0].getMessage()));
+        }
+    }
+
+    @Test
+    public void testGoodDesc2() throws IOException {
+
+        ValidateContext vContext = new ValidateContext();
+        CubeDesc desc = JsonUtil.readValue(new FileInputStream("../examples/test_case_data/localmeta/cube_desc/test_kylin_cube_with_slr_desc.json"), CubeDesc.class);
+        desc.getAggregationGroups().get(0).getSelectRule().joint_dims = new String[][] {//
+        new String[] { "lstg_format_name", "lstg_site_id", "slr_segment_cd", "CATEG_LVL2_NAME" } };
+
+        IValidatorRule<CubeDesc> rule = getAggregationGroupRule();
+        rule.validate(desc, vContext);
+        vContext.print(System.out);
+        assertEquals(0, vContext.getResults().length);
+    }
+
+    @Test
+    public void testBadDesc1() throws IOException {
+
+        ValidateContext vContext = new ValidateContext();
+        CubeDesc desc = JsonUtil.readValue(new FileInputStream("../examples/test_case_data/localmeta/cube_desc/test_kylin_cube_with_slr_desc.json"), CubeDesc.class);
+        String[] temp = Arrays.asList(desc.getAggregationGroups().get(0).getIncludes()).subList(0, 3).toArray(new String[3]);
+
+        desc.getAggregationGroups().get(0).setIncludes(temp);
+        IValidatorRule<CubeDesc> rule = getAggregationGroupRule();
+        rule.validate(desc, vContext);
+        vContext.print(System.out);
+        //        System.out.println(vContext.getResults().length);
+        //        System.out.println(vContext.getResults()[0].getMessage());
+        assertEquals(1, vContext.getResults().length);
+        assertEquals("Aggregation group 0 Include dims not containing all the used dims", (vContext.getResults()[0].getMessage()));
+    }
+
+    @Test
+    public void testBadDesc2() throws IOException {
+
+        ValidateContext vContext = new ValidateContext();
+        CubeDesc desc = JsonUtil.readValue(new FileInputStream("../examples/test_case_data/localmeta/cube_desc/test_kylin_cube_with_slr_desc.json"), CubeDesc.class);
+        desc.getAggregationGroups().get(0).getSelectRule().joint_dims = new String[][] {//
+        new String[] { "lstg_format_name", "lstg_site_id", "slr_segment_cd", "META_CATEG_NAME", "CATEG_LVL2_NAME" } };
+
+        IValidatorRule<CubeDesc> rule = getAggregationGroupRule();
+        rule.validate(desc, vContext);
+        vContext.print(System.out);
+        assertEquals(1, vContext.getResults().length);
+        assertEquals("Aggregation group 0 joint columns overlap with more than 1 dim in same hierarchy", (vContext.getResults()[0].getMessage()));
+    }
+
+    public AggregationGroupRule getAggregationGroupRule() {
+        AggregationGroupRule rule = new AggregationGroupRule() {
+            @Override
+            protected int getMaxAgrGroupSize() {
+                return 20;
+            }
+        };
+
+        return rule;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/test/java/org/apache/kylin/cube/AggregationGroupSizeRuleTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/AggregationGroupSizeRuleTest.java b/core-cube/src/test/java/org/apache/kylin/cube/AggregationGroupSizeRuleTest.java
deleted file mode 100644
index e7914c7..0000000
--- a/core-cube/src/test/java/org/apache/kylin/cube/AggregationGroupSizeRuleTest.java
+++ /dev/null
@@ -1,101 +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;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.kylin.common.util.JsonUtil;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.validation.IValidatorRule;
-import org.apache.kylin.cube.model.validation.ValidateContext;
-import org.apache.kylin.cube.model.validation.rule.AggregationGroupSizeRule;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * @author jianliu
- * 
- */
-public class AggregationGroupSizeRuleTest {
-
-    private CubeDesc cube;
-    private ValidateContext vContext = new ValidateContext();
-
-    /**
-     * @throws java.lang.Exception
-     */
-    @Before
-    public void setUp() throws Exception {
-        CubeDesc desc2 = JsonUtil.readValue(getClass().getClassLoader().getResourceAsStream("data/TEST2_desc.json"), CubeDesc.class);
-        this.cube = desc2;
-
-    }
-
-    @Test
-    public void testOneMandatoryColumn() {
-        IValidatorRule<CubeDesc> rule = new AggregationGroupSizeRule() {
-            /*
-             * (non-Javadoc)
-             * 
-             * @see
-             * org.apache.kylin.metadata.validation.rule.AggregationGroupSizeRule
-             * #getMaxAgrGroupSize()
-             */
-            @Override
-            protected int getMaxAgrGroupSize() {
-                return 3;
-            }
-        };
-        rule.validate(cube, vContext);
-        vContext.print(System.out);
-        assertEquals("Failed to validate aggragation group error", vContext.getResults().length, 2);
-        assertTrue("Failed to validate aggragation group error", vContext.getResults()[0].getMessage().startsWith("Length of the number"));
-        assertTrue("Failed to validate aggragation group error", vContext.getResults()[1].getMessage().startsWith("Length of the number"));
-        // assertTrue("Failed to validate aggragation group error",
-        // vContext.getResults()[2].getMessage()
-        // .startsWith("Hierachy column"));
-    }
-
-    @Test
-    public void testAggColumnSize() {
-        AggregationGroupSizeRule rule = new AggregationGroupSizeRule() {
-            /*
-             * (non-Javadoc)
-             * 
-             * @see
-             * org.apache.kylin.metadata.validation.rule.AggregationGroupSizeRule
-             * #getMaxAgrGroupSize()
-             */
-            @Override
-            protected int getMaxAgrGroupSize() {
-                return 20;
-            }
-        };
-        rule.validate(cube, vContext);
-        vContext.print(System.out);
-        assertEquals("Failed to validate aggragation group error", vContext.getResults().length, 0);
-        // assertTrue("Failed to validate aggragation group error",
-        // vContext.getResults()[0].getMessage()
-        // .startsWith("Aggregation group"));
-        // assertTrue("Failed to validate aggragation group error",
-        // vContext.getResults()[0].getMessage()
-        // .startsWith("Hierachy column"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/test/java/org/apache/kylin/cube/CubeManagerTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/CubeManagerTest.java b/core-cube/src/test/java/org/apache/kylin/cube/CubeManagerTest.java
index fcfa67d..15c6deb 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/CubeManagerTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/CubeManagerTest.java
@@ -66,7 +66,7 @@ public class CubeManagerTest extends LocalFileMetadataTestCase {
 
         String signature = desc.calculateSignature();
         desc.getModel().getPartitionDesc().setPartitionDateColumn("test_column");
-        assertTrue(!signature.equals(desc.calculateSignature()));
+        assertTrue(signature.equals(desc.calculateSignature()));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/test/java/org/apache/kylin/cube/DictionaryManagerTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/DictionaryManagerTest.java b/core-cube/src/test/java/org/apache/kylin/cube/DictionaryManagerTest.java
index d7feb56..0de5c22 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/DictionaryManagerTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/DictionaryManagerTest.java
@@ -56,10 +56,10 @@ public class DictionaryManagerTest extends LocalFileMetadataTestCase {
         CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_without_slr_desc");
         TblColRef col = cubeDesc.findColumnRef("DEFAULT.TEST_CATEGORY_GROUPINGS", "META_CATEG_NAME");
 
-        DictionaryInfo info1 = dictMgr.buildDictionary(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null);
+        DictionaryInfo info1 = dictMgr.buildDictionary(cubeDesc.getModel(), cubeDesc.getRowkey().isUseDictionary(col), col, null);
         System.out.println(JsonUtil.writeValueAsIndentString(info1));
 
-        DictionaryInfo info2 = dictMgr.buildDictionary(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null);
+        DictionaryInfo info2 = dictMgr.buildDictionary(cubeDesc.getModel(), cubeDesc.getRowkey().isUseDictionary(col), col, null);
         System.out.println(JsonUtil.writeValueAsIndentString(info2));
 
         assertTrue(info1.getUuid() == info2.getUuid());

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/test/java/org/apache/kylin/cube/MandatoryColumnRuleTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/MandatoryColumnRuleTest.java b/core-cube/src/test/java/org/apache/kylin/cube/MandatoryColumnRuleTest.java
deleted file mode 100644
index 29b7183..0000000
--- a/core-cube/src/test/java/org/apache/kylin/cube/MandatoryColumnRuleTest.java
+++ /dev/null
@@ -1,57 +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;
-
-import static org.junit.Assert.assertTrue;
-
-import org.apache.kylin.common.util.JsonUtil;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.validation.IValidatorRule;
-import org.apache.kylin.cube.model.validation.ValidateContext;
-import org.apache.kylin.cube.model.validation.rule.MandatoryColumnRule;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * @author jianliu
- * 
- */
-public class MandatoryColumnRuleTest {
-
-    private CubeDesc cube;
-    private ValidateContext vContext = new ValidateContext();
-
-    /**
-     * @throws java.lang.Exception
-     */
-    @Before
-    public void setUp() throws Exception {
-        CubeDesc desc2 = JsonUtil.readValue(getClass().getClassLoader().getResourceAsStream("data/TEST1_desc.json"), CubeDesc.class);
-        this.cube = desc2;
-
-    }
-
-    @Test
-    public void testOneMandatoryColumn() {
-        IValidatorRule<CubeDesc> rule = new MandatoryColumnRule();
-        rule.validate(cube, vContext);
-        assertTrue("Failed to validate mandatory error", vContext.getResults().length == 1);
-        assertTrue("Failed to validate mandatory error", vContext.getResults()[0].getMessage().startsWith("mandatory column"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/test/java/org/apache/kylin/cube/RowKeyAttrRuleTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/RowKeyAttrRuleTest.java b/core-cube/src/test/java/org/apache/kylin/cube/RowKeyAttrRuleTest.java
index 000fa1d..be7b624 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/RowKeyAttrRuleTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/RowKeyAttrRuleTest.java
@@ -20,39 +20,42 @@ package org.apache.kylin.cube;
 
 import static org.junit.Assert.assertTrue;
 
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
 import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.validation.IValidatorRule;
 import org.apache.kylin.cube.model.validation.ValidateContext;
 import org.apache.kylin.cube.model.validation.rule.RowKeyAttrRule;
-import org.junit.Before;
 import org.junit.Test;
 
-/**
- * @author jianliu
- * 
- */
 public class RowKeyAttrRuleTest {
 
-    private CubeDesc cube;
-    private ValidateContext vContext = new ValidateContext();
-
-    /**
-     * @throws java.lang.Exception
-     */
-    @Before
-    public void setUp() throws Exception {
-        CubeDesc desc2 = JsonUtil.readValue(getClass().getClassLoader().getResourceAsStream("data/TEST3_desc.json"), CubeDesc.class);
-        this.cube = desc2;
-
+    @Test
+    public void testGoodDesc() throws IOException {
+        for (File f : new File("../examples/test_case_data/localmeta/cube_desc/").listFiles()) {
+            CubeDesc desc = JsonUtil.readValue(new FileInputStream(f), CubeDesc.class);
+            ValidateContext vContext = new ValidateContext();
+            IValidatorRule<CubeDesc> rule = new RowKeyAttrRule();
+            rule.validate(desc, vContext);
+            vContext.print(System.out);
+            assertTrue(vContext.getResults().length == 0);
+        }
     }
 
     @Test
-    public void testOneMandatoryColumn() {
+    public void testBadDesc() throws IOException {
+        ValidateContext vContext = new ValidateContext();
+        CubeDesc desc = JsonUtil.readValue(new FileInputStream("../examples/test_case_data/localmeta/cube_desc/test_kylin_cube_with_slr_desc.json"), CubeDesc.class);
+        desc.getRowkey().getRowKeyColumns()[1].setEncoding("non-supported-encoding");
+        desc.getRowkey().getRowKeyColumns()[2].setColumn("");
         IValidatorRule<CubeDesc> rule = new RowKeyAttrRule();
-        rule.validate(cube, vContext);
+        rule.validate(desc, vContext);
         vContext.print(System.out);
-        assertTrue("Failed to validate rowkey", vContext.getResults().length == 1);
-        assertTrue("Failed to validate mandatory error", vContext.getResults()[0].getMessage().startsWith("Rowkey column"));
+        assertTrue(vContext.getResults().length == 2);
+        assertTrue("Rowkey column cal_dt encoding not dict nor fixed_length".equals(vContext.getResults()[0].getMessage()));
+        assertTrue("Rowkey column empty".equalsIgnoreCase(vContext.getResults()[1].getMessage()));
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidSchedulerTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidSchedulerTest.java b/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidSchedulerTest.java
index 83e8d94..0711d6e 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidSchedulerTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidSchedulerTest.java
@@ -18,14 +18,12 @@
 
 package org.apache.kylin.cube.cuboid;
 
-import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.cube.CubeDescManager;
@@ -79,37 +77,16 @@ public class CuboidSchedulerTest extends LocalFileMetadataTestCase {
         return getCubeDescManager().getCubeDesc("test_kylin_cube_with_slr_desc");
     }
 
+    private CubeDesc getTestKylinCubeWithSellerLeft() {
+        return getCubeDescManager().getCubeDesc("test_kylin_cube_with_slr_left_join_desc");
+    }
+
     private CubeDesc getTestKylinCubeWithoutSellerLeftJoin() {
         return getCubeDescManager().getCubeDesc("test_kylin_cube_without_slr_left_join_desc");
     }
 
-    @Test
-    public void testFindSmallerSibling1() {
-        CubeDesc cube = getTestKylinCubeWithoutSeller();
-        CuboidScheduler scheduler = new CuboidScheduler(cube);
-
-        Collection<Long> siblings;
-
-        siblings = scheduler.findSmallerSibling(255);
-        assertEquals("[]", siblings.toString());
-
-        siblings = scheduler.findSmallerSibling(133);
-        assertEquals("[131]", siblings.toString());
-
-        siblings = scheduler.findSmallerSibling(127);
-        assertEquals("[]", siblings.toString());
-
-        siblings = scheduler.findSmallerSibling(134);
-        assertEquals("[131, 133]", sortToString(siblings));
-
-        siblings = scheduler.findSmallerSibling(130);
-        assertEquals("[129]", siblings.toString());
-
-        siblings = scheduler.findSmallerSibling(5);
-        assertEquals("[]", siblings.toString());
-
-        siblings = scheduler.findSmallerSibling(135);
-        assertEquals("[]", siblings.toString());
+    private CubeDesc getStreamingCubeDesc() {
+        return getCubeDescManager().getCubeDesc("test_streaming_table_cube_desc");
     }
 
     private void testSpanningAndGetParent(CuboidScheduler scheduler, CubeDesc cube, long[] cuboidIds) {
@@ -124,32 +101,6 @@ public class CuboidSchedulerTest extends LocalFileMetadataTestCase {
     }
 
     @Test
-    public void testFindSmallerSibling2() {
-        CubeDesc cube = getTestKylinCubeWithSeller();
-        CuboidScheduler scheduler = new CuboidScheduler(cube);
-
-        Collection<Long> siblings;
-
-        siblings = scheduler.findSmallerSibling(511);
-        assertEquals("[]", siblings.toString());
-
-        siblings = scheduler.findSmallerSibling(toLong("110111111"));
-        assertEquals("[383]", siblings.toString());
-
-        siblings = scheduler.findSmallerSibling(toLong("101110111"));
-        assertEquals("[319]", siblings.toString());
-
-        siblings = scheduler.findSmallerSibling(toLong("111111000"));
-        assertEquals("[]", siblings.toString());
-
-        siblings = scheduler.findSmallerSibling(toLong("111111000"));
-        assertEquals("[]", siblings.toString());
-
-        siblings = scheduler.findSmallerSibling(toLong("110000000"));
-        assertEquals("[288, 320]", sortToString(siblings));
-    }
-
-    @Test
     public void testGetSpanningCuboid2() {
         CubeDesc cube = getTestKylinCubeWithSeller();
         CuboidScheduler scheduler = new CuboidScheduler(cube);
@@ -183,7 +134,12 @@ public class CuboidSchedulerTest extends LocalFileMetadataTestCase {
         CuboidScheduler scheduler = new CuboidScheduler(cube);
 
         long quiz = toLong("01100111");
-        testSpanningAndGetParent(scheduler, cube, new long[] { quiz });
+        try {
+            testSpanningAndGetParent(scheduler, cube, new long[] { quiz });
+            fail();
+        } catch (IllegalStateException ex) {
+            //expected
+        }
 
         // generate 7d
         System.out.println("Spanning for 7D Cuboids");
@@ -210,16 +166,6 @@ public class CuboidSchedulerTest extends LocalFileMetadataTestCase {
     }
 
     @Test
-    public void testGetSpanningCuboid() {
-        CubeDesc cube = getTestKylinCubeWithoutSeller();
-        CuboidScheduler scheduler = new CuboidScheduler(cube);
-
-        Collection<Long> spnanningCuboids = scheduler.getSpanningCuboid(248);
-
-        assertEquals("[]", spnanningCuboids.toString());
-    }
-
-    @Test
     public void testGetCardinality() {
         CubeDesc cube = getTestKylinCubeWithSeller();
         CuboidScheduler scheduler = new CuboidScheduler(cube);
@@ -233,43 +179,98 @@ public class CuboidSchedulerTest extends LocalFileMetadataTestCase {
 
     @Test
     public void testCuboidGeneration1() {
+
         CubeDesc cube = getTestKylinCubeWithoutSeller();
-        CuboidCLI.simulateCuboidGeneration(cube);
+        CuboidCLI.simulateCuboidGeneration(cube, true);
     }
 
     @Test
     public void testCuboidGeneration2() {
         CubeDesc cube = getTestKylinCubeWithSeller();
-        CuboidCLI.simulateCuboidGeneration(cube);
+        CuboidCLI.simulateCuboidGeneration(cube, true);
     }
 
     @Test
     public void testCuboidGeneration3() {
         CubeDesc cube = getTestKylinCubeWithoutSellerLeftJoin();
-        CuboidCLI.simulateCuboidGeneration(cube);
+        CuboidCLI.simulateCuboidGeneration(cube, true);
+    }
+
+    @Test
+    public void testCuboidGeneration4() {
+        CubeDesc cube = getTestKylinCubeWithSellerLeft();
+        CuboidCLI.simulateCuboidGeneration(cube, true);
+    }
+
+    @Test
+    public void testCuboidGeneration5() {
+        CubeDesc cube = getStreamingCubeDesc();
+        CuboidCLI.simulateCuboidGeneration(cube, true);
     }
 
     @Test
     public void testCuboidCounts1() {
         CubeDesc cube = getTestKylinCubeWithoutSeller();
+        CuboidScheduler cuboidScheduler = new CuboidScheduler(cube);
         int[] counts = CuboidCLI.calculateAllLevelCount(cube);
         printCount(counts);
-        assertArrayEquals(new int[] { 1, 4, 6, 6, 4, 4, 2, 0 }, counts);
+        int sum = 0;
+        for (Integer x : counts) {
+            sum += x;
+        }
+        assertEquals(cuboidScheduler.getCuboidCount(), sum);
     }
 
     @Test
     public void testCuboidCounts2() {
+        CubeDesc cube = getTestKylinCubeWithoutSellerLeftJoin();
+        CuboidScheduler cuboidScheduler = new CuboidScheduler(cube);
+        int[] counts = CuboidCLI.calculateAllLevelCount(cube);
+        printCount(counts);
+        int sum = 0;
+        for (Integer x : counts) {
+            sum += x;
+        }
+        assertEquals(cuboidScheduler.getCuboidCount(), sum);
+    }
+
+    @Test
+    public void testCuboidCounts3() {
         CubeDesc cube = getTestKylinCubeWithSeller();
-        CuboidCLI.calculateAllLevelCount(cube);
+        CuboidScheduler cuboidScheduler = new CuboidScheduler(cube);
+        int[] counts = CuboidCLI.calculateAllLevelCount(cube);
+        printCount(counts);
+        int sum = 0;
+        for (Integer x : counts) {
+            sum += x;
+        }
+        assertEquals(cuboidScheduler.getCuboidCount(), sum);
+    }
+
+    @Test
+    public void testCuboidCounts4() {
+        CubeDesc cube = getTestKylinCubeWithSellerLeft();
+        CuboidScheduler cuboidScheduler = new CuboidScheduler(cube);
         int[] counts = CuboidCLI.calculateAllLevelCount(cube);
         printCount(counts);
-        assertArrayEquals(new int[] { 1, 4, 7, 8, 7, 4 }, counts);
+        int sum = 0;
+        for (Integer x : counts) {
+            sum += x;
+        }
+        assertEquals(cuboidScheduler.getCuboidCount(), sum);
     }
 
-    private String sortToString(Collection<Long> longs) {
-        ArrayList<Long> copy = new ArrayList<Long>(longs);
-        Collections.sort(copy);
-        return copy.toString();
+    @Test
+    public void testCuboidCounts5() {
+        CubeDesc cube = getStreamingCubeDesc();
+        CuboidScheduler cuboidScheduler = new CuboidScheduler(cube);
+        int[] counts = CuboidCLI.calculateAllLevelCount(cube);
+        printCount(counts);
+        int sum = 0;
+        for (Integer x : counts) {
+            sum += x;
+        }
+        assertEquals(cuboidScheduler.getCuboidCount(), sum);
     }
 
     public CubeDescManager getCubeDescManager() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidTest.java b/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidTest.java
index 89e235f..eecf368 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/cuboid/CuboidTest.java
@@ -19,7 +19,6 @@
 package org.apache.kylin.cube.cuboid;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
 
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.cube.CubeDescManager;
@@ -116,12 +115,7 @@ public class CuboidTest extends LocalFileMetadataTestCase {
     @Test
     public void testIsValid2() {
         CubeDesc cube = getTestKylinCubeWithoutSeller();
-        try {
-            assertEquals(false, Cuboid.isValid(cube, toLong("111111111")));
-            fail();
-        } catch (IllegalArgumentException ex) {
-            // expected
-        }
+        assertEquals(false, Cuboid.isValid(cube, toLong("111111111")));
 
         // base
         assertEquals(false, Cuboid.isValid(cube, 0));
@@ -148,7 +142,7 @@ public class CuboidTest extends LocalFileMetadataTestCase {
         Cuboid cuboid;
 
         cuboid = Cuboid.findById(cube, 0);
-        assertEquals(toLong("100100000"), cuboid.getId());
+        assertEquals(toLong("101000000"), cuboid.getId());
 
         cuboid = Cuboid.findById(cube, 1);
         assertEquals(toLong("100000111"), cuboid.getId());

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/test/resources/data/TEST1_desc.json
----------------------------------------------------------------------
diff --git a/core-cube/src/test/resources/data/TEST1_desc.json b/core-cube/src/test/resources/data/TEST1_desc.json
deleted file mode 100644
index 082a3cf..0000000
--- a/core-cube/src/test/resources/data/TEST1_desc.json
+++ /dev/null
@@ -1,183 +0,0 @@
-{
-  "uuid" : "9c53506d-d7b9-4ad4-b3b1-53ea42673c4b",
-  "last_modified" : 1401429176099,
-  "name" : "TEST1_desc",
-  "model_name": "TEST1_model_desc",
-  "fact_table" : "TEST_KYLIN_FACT",
-  "dimensions" : [ {
-    "id" : 1,
-    "name" : "CAL_DT",
-    "join" : {
-      "type" : "inner",
-      "primary_key" : [ "CAL_DT" ],
-      "foreign_key" : [ "CAL_DT" ]
-    },
-    "hierarchy" : null,
-    "table" : "TEST_CAL_DT",
-    "column" : ["TEST_CAL_DT.CAL_DT"],
-    "derived" : [ "WEEK_BEG_DT" ]
-  }, {
-    "id" : 2,
-    "name" : "CATEGORY",
-    "join" : {
-      "type" : "inner",
-      "primary_key" : [ "LEAF_CATEG_ID", "SITE_ID" ],
-      "foreign_key" : [ "LEAF_CATEG_ID", "LSTG_SITE_ID" ]
-    },
-    "hierarchy" : true,
-    "table" : "TEST_CATEGORY_GROUPINGS",
-    "column" : ["TEST_CATEGORY_GROUPINGS.META_CATEG_NAME", "TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME", "TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME"],
-    "datatype" : null,
-    "derived" : null
-  }, {
-    "id" : 3,
-    "name" : "LSTG_FORMAT_NAME",
-    "join" : null,
-    "hierarchy" : null,
-    "table" : "TEST_KYLIN_FACT",
-    "column" : ["TEST_KYLIN_FACT.LSTG_FORMAT_NAME"],
-    "datatype" : "string",
-    "derived" : null
-  }, {
-    "id" : 4,
-    "name" : "SITE_ID",
-    "join" : {
-      "type" : "inner",
-      "primary_key" : [ "SITE_ID" ],
-      "foreign_key" : [ "LSTG_SITE_ID" ]
-    },
-    "hierarchy" : null,
-    "table" : "TEST_SITES",
-    "column" : ["TEST_SITES.SITE_ID"],
-    "datatype" : "string",
-    "derived" : [ "SITE_NAME", "CRE_USER" ]
-  }, {
-    "id" : 5,
-    "name" : "SELLER_TYPE_CD",
-    "join" : {
-      "type" : "inner",
-      "primary_key" : [ "SELLER_TYPE_CD" ],
-      "foreign_key" : [ "SLR_SEGMENT_CD" ]
-    },
-    "hierarchy" : null,
-    "table" : "TEST_SELLER_TYPE_DIM",
-    "column" : ["TEST_SELLER_TYPE_DIM.SELLER_TYPE_CD"],
-    "datatype" : "string",
-    "derived" : [ "SELLER_TYPE_DESC" ]
-  } ],
-  "measures" : [ {
-    "id" : 1,
-    "name" : "GMV_SUM",
-    "function" : {
-      "expression" : "SUM",
-      "parameter" : {
-        "type" : "column",
-        "value" : "PRICE"
-      },
-      "returntype" : "decimal"
-    }
-  }, {
-    "id" : 2,
-    "name" : "GMV_MIN",
-    "function" : {
-      "expression" : "MIN",
-      "parameter" : {
-        "type" : "column",
-        "value" : "PRICE"
-      },
-      "returntype" : "decimal"
-    }
-  }, {
-    "id" : 3,
-    "name" : "GMV_MAX",
-    "function" : {
-      "expression" : "MAX",
-      "parameter" : {
-        "type" : "column",
-        "value" : "PRICE"
-      },
-      "returntype" : "decimal"
-    }
-  }, {
-    "id" : 4,
-    "name" : "TRANS_CNT",
-    "function" : {
-      "expression" : "COUNT",
-      "parameter" : {
-        "type" : "constant",
-        "value" : "1"
-      },
-      "returntype" : "long"
-    }
-  }, {
-    "id" : 5,
-    "name" : "SELLER_CNT",
-    "function" : {
-      "expression" : "COUNT_DISTINCT",
-      "parameter" : {
-        "type" : "column",
-        "value" : "SELLER_ID"
-      },
-      "returntype" : "hllc10"
-    }
-  }, {
-    "id" : 6,
-    "name" : "SELLER_FORMAT_CNT",
-    "function" : {
-      "expression" : "COUNT_DISTINCT",
-      "parameter" : {
-        "type" : "column",
-        "value" : "LSTG_FORMAT_NAME,SELLER_ID"
-      },
-      "returntype" : "hllc10"
-    }
-  } ],
-  "rowkey" : {
-    "rowkey_columns" : [ {
-      "column" : "CAL_DT",
-      "length" : 0,
-      "dictionary" : "date(yyyy-mm-dd)",
-      "mandatory" : true
-    }, {
-      "column" : "META_CATEG_NAME",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    }, {
-      "column" : "CATEG_LVL2_NAME",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    }, {
-      "column" : "CATEG_LVL3_NAME",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    }, {
-      "column" : "LSTG_FORMAT_NAME",
-      "length" : 12,
-      "dictionary" : null,
-      "mandatory" : false
-    }, {
-      "column" : "SITE_ID",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    }, {
-      "column" : "SELLER_TYPE_CD",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    } ],
-    "aggregation_groups" : [ [ "META_CATEG_NAME", "CATEG_LVL3_NAME", "CATEG_LVL2_NAME", "CAL_DT" ], [ "LSTG_FORMAT_NAME", "SITE_ID", "SELLER_TYPE_CD" ] ]
-  },
-  "hbase_mapping" : {
-    "column_family" : [ {
-      "name" : "F1",
-      "columns" : [ {
-        "qualifier" : "M",
-        "measure_refs" : [ "GMV_SUM", "GMV_MIN", "GMV_MAX", "TRANS_CNT", "SELLER_CNT", "SELLER_FORMAT_CNT" ]
-      } ]
-    } ]
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/test/resources/data/TEST2_desc.json
----------------------------------------------------------------------
diff --git a/core-cube/src/test/resources/data/TEST2_desc.json b/core-cube/src/test/resources/data/TEST2_desc.json
deleted file mode 100644
index 90bd806..0000000
--- a/core-cube/src/test/resources/data/TEST2_desc.json
+++ /dev/null
@@ -1,168 +0,0 @@
-{
-  "uuid" : "9c53506d-d7b9-4ad4-b3b1-53ea42673c4b",
-  "last_modified" : 1401429176099,
-  "model_name": "TEST2_model_desc",
-  "name" : "TEST2_desc",
-  "dimensions" : [ {
-    "name" : "CAL_DT",
-    "hierarchy" : null,
-    "column" : ["TEST_CAL_DT.CAL_DT"],
-    "derived" : [ "WEEK_BEG_DT" ]
-  }, {
-    "id" : 2,
-    "name" : "CATEGORY",
-    "join" : {
-      "type" : "inner",
-      "primary_key" : [ "LEAF_CATEG_ID", "SITE_ID" ],
-      "foreign_key" : [ "LEAF_CATEG_ID", "LSTG_SITE_ID" ]
-    },
-    "hierarchy" : true,
-    "column" : ["TEST_CATEGORY_GROUPINGS.META_CATEG_NAME", "TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME", "TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME"],
-    "derived" : null
-  }, {
-    "id" : 3,
-    "name" : "LSTG_FORMAT_NAME",
-    "hierarchy" : null,
-    "table" : "TEST_KYLIN_FACT",
-    "column" : ["TEST_KYLIN_FACT.LSTG_FORMAT_NAME"],
-    "derived" : null
-  }, {
-    "id" : 4,
-    "name" : "SITE_ID",
-    "join" : {
-      "type" : "inner",
-      "primary_key" : [ "SITE_ID" ],
-      "foreign_key" : [ "LSTG_SITE_ID" ]
-    },
-    "hierarchy" : null,
-    "column" : ["TEST_SITES.SITE_ID"],
-    "derived" : [ "SITE_NAME", "CRE_USER" ]
-  }, {
-    "id" : 5,
-    "name" : "SELLER_TYPE_CD",
-    "join" : {
-      "type" : "inner",
-      "primary_key" : [ "SELLER_TYPE_CD" ],
-      "foreign_key" : [ "SLR_SEGMENT_CD" ]
-    },
-    "hierarchy" : null,
-    "table" : "TEST_SELLER_TYPE_DIM",
-    "column" : ["TEST_SELLER_TYPE_DIM.SELLER_TYPE_CD"],
-    "derived" : [ "SELLER_TYPE_DESC" ]
-  } ],
-  "measures" : [ {
-    "id" : 1,
-    "name" : "GMV_SUM",
-    "function" : {
-      "expression" : "SUM",
-      "parameter" : {
-        "type" : "column",
-        "value" : "PRICE"
-      },
-      "returntype" : "decimal"
-    }
-  }, {
-    "id" : 2,
-    "name" : "GMV_MIN",
-    "function" : {
-      "expression" : "MIN",
-      "parameter" : {
-        "type" : "column",
-        "value" : "PRICE"
-      },
-      "returntype" : "decimal"
-    }
-  }, {
-    "id" : 3,
-    "name" : "GMV_MAX",
-    "function" : {
-      "expression" : "MAX",
-      "parameter" : {
-        "type" : "column",
-        "value" : "PRICE"
-      },
-      "returntype" : "decimal"
-    }
-  }, {
-    "id" : 4,
-    "name" : "TRANS_CNT",
-    "function" : {
-      "expression" : "COUNT",
-      "parameter" : {
-        "type" : "constant",
-        "value" : "1"
-      },
-      "returntype" : "long"
-    }
-  }, {
-    "id" : 5,
-    "name" : "SELLER_CNT",
-    "function" : {
-      "expression" : "COUNT_DISTINCT",
-      "parameter" : {
-        "type" : "column",
-        "value" : "SELLER_ID"
-      },
-      "returntype" : "hllc10"
-    }
-  }, {
-    "id" : 6,
-    "name" : "SELLER_FORMAT_CNT",
-    "function" : {
-      "expression" : "COUNT_DISTINCT",
-      "parameter" : {
-        "type" : "column",
-        "value" : "LSTG_FORMAT_NAME,SELLER_ID"
-      },
-      "returntype" : "hllc10"
-    }
-  } ],
-  "rowkey" : {
-    "rowkey_columns" : [ {
-      "column" : "CAL_DT",
-      "length" : 0,
-      "dictionary" : "date(yyyy-mm-dd)",
-      "mandatory" : true
-    }, {
-      "column" : "META_CATEG_NAME",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    }, {
-      "column" : "CATEG_LVL2_NAME",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    }, {
-      "column" : "CATEG_LVL3_NAME",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    }, {
-      "column" : "LSTG_FORMAT_NAME",
-      "length" : 12,
-      "dictionary" : null,
-      "mandatory" : false
-    }, {
-      "column" : "SITE_ID",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    }, {
-      "column" : "SELLER_TYPE_CD",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    } ],
-    "aggregation_groups" : [ [ "META_CATEG_NAME", "CATEG_LVL3_NAME", "CAL_DT" ], [ "LSTG_FORMAT_NAME", "SITE_ID", "SELLER_TYPE_CD", "CATEG_LVL2_NAME" ] ]
-  },
-  "hbase_mapping" : {
-    "column_family" : [ {
-      "name" : "F1",
-      "columns" : [ {
-        "qualifier" : "M",
-        "measure_refs" : [ "GMV_SUM", "GMV_MIN", "GMV_MAX", "TRANS_CNT", "SELLER_CNT", "SELLER_FORMAT_CNT" ]
-      } ]
-    } ]
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/test/resources/data/TEST3_desc.json
----------------------------------------------------------------------
diff --git a/core-cube/src/test/resources/data/TEST3_desc.json b/core-cube/src/test/resources/data/TEST3_desc.json
deleted file mode 100644
index 4b0836b..0000000
--- a/core-cube/src/test/resources/data/TEST3_desc.json
+++ /dev/null
@@ -1,182 +0,0 @@
-{
-  "uuid" : "9c53506d-d7b9-4ad4-b3b1-53ea42673c4b",
-  "last_modified" : 1401429176099,
-  "name" : "TEST1_desc",
-  "model_name": "TEST1_model_desc",
-  "dimensions" : [ {
-    "id" : 1,
-    "name" : "CAL_DT",
-    "join" : {
-      "type" : "inner",
-      "primary_key" : [ "CAL_DT" ],
-      "foreign_key" : [ "CAL_DT" ]
-    },
-    "hierarchy" : null,
-    "table" : "TEST_CAL_DT",
-    "column" : ["TEST_CAL_DT.CAL_DT"],
-    "datatype" : "date",
-    "derived" : [ "WEEK_BEG_DT" ]
-  }, {
-    "id" : 2,
-    "name" : "CATEGORY",
-    "join" : {
-      "type" : "inner",
-      "primary_key" : [ "LEAF_CATEG_ID", "SITE_ID" ],
-      "foreign_key" : [ "LEAF_CATEG_ID", "LSTG_SITE_ID" ]
-    },
-    "hierarchy" : true,
-    "table" : "TEST_CATEGORY_GROUPINGS",
-    "column" : ["TEST_CATEGORY_GROUPINGS.META_CATEG_NAME", "TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME", "TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME"],
-    "derived" : null
-  }, {
-    "id" : 3,
-    "name" : "LSTG_FORMAT_NAME",
-    "join" : null,
-    "hierarchy" : null,
-    "table" : "TEST_KYLIN_FACT",
-    "column" : ["TEST_KYLIN_FACT.LSTG_FORMAT_NAME"],
-    "datatype" : "string",
-    "derived" : null
-  }, {
-    "id" : 4,
-    "name" : "SITE_ID",
-    "join" : {
-      "type" : "inner",
-      "primary_key" : [ "SITE_ID" ],
-      "foreign_key" : [ "LSTG_SITE_ID" ]
-    },
-    "hierarchy" : null,
-    "table" : "TEST_SITES",
-    "column" : ["TEST_SITES.SITE_ID"],
-    "datatype" : "string",
-    "derived" : [ "SITE_NAME", "CRE_USER" ]
-  }, {
-    "id" : 5,
-    "name" : "SELLER_TYPE_CD",
-    "join" : {
-      "type" : "inner",
-      "primary_key" : [ "SELLER_TYPE_CD" ],
-      "foreign_key" : [ "SLR_SEGMENT_CD" ]
-    },
-    "hierarchy" : null,
-    "table" : "TEST_SELLER_TYPE_DIM",
-    "column" : ["TEST_SELLER_TYPE_DIM.SELLER_TYPE_CD"],
-    "datatype" : "string",
-    "derived" : [ "SELLER_TYPE_DESC" ]
-  } ],
-  "measures" : [ {
-    "id" : 1,
-    "name" : "GMV_SUM",
-    "function" : {
-      "expression" : "SUM",
-      "parameter" : {
-        "type" : "column",
-        "value" : "PRICE"
-      },
-      "returntype" : "decimal"
-    }
-  }, {
-    "id" : 2,
-    "name" : "GMV_MIN",
-    "function" : {
-      "expression" : "MIN",
-      "parameter" : {
-        "type" : "column",
-        "value" : "PRICE"
-      },
-      "returntype" : "decimal"
-    }
-  }, {
-    "id" : 3,
-    "name" : "GMV_MAX",
-    "function" : {
-      "expression" : "MAX",
-      "parameter" : {
-        "type" : "column",
-        "value" : "PRICE"
-      },
-      "returntype" : "decimal"
-    }
-  }, {
-    "id" : 4,
-    "name" : "TRANS_CNT",
-    "function" : {
-      "expression" : "COUNT",
-      "parameter" : {
-        "type" : "constant",
-        "value" : "1"
-      },
-      "returntype" : "long"
-    }
-  }, {
-    "id" : 5,
-    "name" : "SELLER_CNT",
-    "function" : {
-      "expression" : "COUNT_DISTINCT",
-      "parameter" : {
-        "type" : "column",
-        "value" : "SELLER_ID"
-      },
-      "returntype" : "hllc10"
-    }
-  }, {
-    "id" : 6,
-    "name" : "SELLER_FORMAT_CNT",
-    "function" : {
-      "expression" : "COUNT_DISTINCT",
-      "parameter" : {
-        "type" : "column",
-        "value" : "LSTG_FORMAT_NAME,SELLER_ID"
-      },
-      "returntype" : "hllc10"
-    }
-  } ],
-  "rowkey" : {
-    "rowkey_columns" : [ {
-      "column" : "CAL_DT",
-      "length" : 10,
-      "dictionary" : "date(yyyy-mm-dd)",
-      "mandatory" : false
-    }, {
-      "column" : "META_CATEG_NAME",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    }, {
-      "column" : "CATEG_LVL2_NAME",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    }, {
-      "column" : "CATEG_LVL3_NAME",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    }, {
-      "column" : "LSTG_FORMAT_NAME",
-      "length" : 12,
-      "dictionary" : null,
-      "mandatory" : false
-    }, {
-      "column" : "SITE_ID",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    }, {
-      "column" : "SELLER_TYPE_CD",
-      "length" : 0,
-      "dictionary" : "string",
-      "mandatory" : false
-    } ],
-    "aggregation_groups" : [ [ "META_CATEG_NAME", "CATEG_LVL3_NAME", "CATEG_LVL2_NAME", "CAL_DT" ], [ "LSTG_FORMAT_NAME", "SITE_ID", "SELLER_TYPE_CD" ] ]
-  },
-  "hbase_mapping" : {
-    "column_family" : [ {
-      "name" : "F1",
-      "columns" : [ {
-        "qualifier" : "M",
-        "measure_refs" : [ "GMV_SUM", "GMV_MIN", "GMV_MAX", "TRANS_CNT", "SELLER_CNT", "SELLER_FORMAT_CNT" ]
-      } ]
-    } ]
-  }
-}
\ No newline at end of file