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/01 12:03:35 UTC

[1/2] incubator-kylin git commit: KYLIN-242 refine AggregationGroupRuleTest

Repository: incubator-kylin
Updated Branches:
  refs/heads/KYLIN-242-new 894ac9b8a -> bb192414f


KYLIN-242 refine AggregationGroupRuleTest


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

Branch: refs/heads/KYLIN-242-new
Commit: 15d489c1ae9c625315619ead4966fb611645b91a
Parents: 894ac9b
Author: honma <ho...@ebay.com>
Authored: Tue Dec 1 18:39:40 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Tue Dec 1 18:39:40 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/cube/model/RowKeyDesc.java |   4 -
 .../cube/model/validation/ValidateContext.java  |   2 +-
 .../rule/AggregationGroupOverlapRule.java       |  85 ---------
 .../validation/rule/AggregationGroupRule.java   |  53 ++++--
 .../model/validation/rule/RowKeyAttrRule.java   |  27 +--
 .../kylin/cube/AggregationGroupRuleTest.java    | 127 +++++++------
 .../apache/kylin/cube/RowKeyAttrRuleTest.java   |  45 ++---
 .../src/test/resources/data/TEST1_desc.json     | 183 -------------------
 .../src/test/resources/data/TEST2_desc.json     | 168 -----------------
 .../src/test/resources/data/TEST3_desc.json     | 182 ------------------
 10 files changed, 141 insertions(+), 735 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15d489c1/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 87b182d..d085067 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
@@ -90,10 +90,6 @@ public class RowKeyDesc {
         buildRowKey(colNameAbbr);
     }
 
-    public RowKeyColDesc[] getRowkeyColumns() {
-        return rowkeyColumns;
-    }
-
     public void setRowkeyColumns(RowKeyColDesc[] rowkeyColumns) {
         this.rowkeyColumns = rowkeyColumns;
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15d489c1/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 7f1f617..632cbff 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
@@ -81,7 +81,7 @@ public class ValidateContext {
      */
     public void print(PrintStream out) {
         if (results.isEmpty()) {
-            out.print("The element is perfect.");
+            out.println("The element is perfect.");
         }
         Iterator<Result> it = results.iterator();
         while (it.hasNext()) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15d489c1/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
deleted file mode 100644
index fb78927..0000000
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupOverlapRule.java
+++ /dev/null
@@ -1,85 +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.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.getJoints()) {
-                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++;
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15d489c1/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
index 4e7b99e..17f5b2d 100644
--- 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
@@ -36,33 +36,39 @@ import org.apache.kylin.cube.model.validation.ValidateContext;
  */
 public class AggregationGroupRule 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);
+        inner(cube, context);
     }
 
-    /**
-     * @param cube
-     * @param context
-     */
-    private void innerValidateMaxSize(CubeDesc cube, ValidateContext 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 + " include dims not set");
+                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;
             }
 
@@ -75,8 +81,8 @@ public class AggregationGroupRule implements IValidatorRule<CubeDesc> {
 
             Set<String> mandatoryDims = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
             if (agg.getSelectRule().mandatory_dims != null) {
-                for (String include : agg.getSelectRule().mandatory_dims) {
-                    mandatoryDims.add(include);
+                for (String m : agg.getSelectRule().mandatory_dims) {
+                    mandatoryDims.add(m);
                 }
             }
 
@@ -98,6 +104,7 @@ public class AggregationGroupRule implements IValidatorRule<CubeDesc> {
 
             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);
@@ -107,17 +114,21 @@ public class AggregationGroupRule implements IValidatorRule<CubeDesc> {
             normalDims.removeAll(jointDims);
 
             int normalDimSize = normalDims.size();
-            int hierarchySize = (agg.getSelectRule().hierarchy_dims == null ? 0 : agg.getSelectRule().hierarchy_dims.length);
-            int jointSize = agg.getSelectRule().joint_dims == null ? 0 : agg.getSelectRule().joint_dims.length;
+            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;
@@ -131,6 +142,7 @@ public class AggregationGroupRule implements IValidatorRule<CubeDesc> {
 
                     if (oneJoint.size() < 2) {
                         context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " require at least 2 dims in a joint");
+                        continue;
                     }
                     jointDimNum += oneJoint.size();
 
@@ -145,17 +157,20 @@ public class AggregationGroupRule implements IValidatorRule<CubeDesc> {
                             }
                             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 1 joint");
+                    context.addResult(ResultLevel.ERROR, "Aggregation group " + index + " a dim exist in more than one joint");
+                    continue;
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15d489c1/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 7f4b6e7..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
@@ -30,13 +30,6 @@ import org.apache.kylin.cube.model.validation.ValidateContext;
  */
 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();
@@ -46,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.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.getColumn() == null || rd.getColumn().length() == 0) {
+                context.addResult(ResultLevel.ERROR, "Rowkey column empty");
+                continue;
+            }
+            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/incubator-kylin/blob/15d489c1/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
index a2395e1..2b75d48 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/AggregationGroupRuleTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/AggregationGroupRuleTest.java
@@ -21,84 +21,107 @@ 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.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 
-/**
- * @author jianliu
- * 
- */
-
-@Ignore
 public class AggregationGroupRuleTest {
 
-    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 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 testOneMandatoryColumn() {
-        IValidatorRule<CubeDesc> rule = new AggregationGroupRule() {
-            /*
-             * (non-Javadoc)
-             * 
-             * @see
-             * org.apache.kylin.metadata.validation.rule.AggregationGroupSizeRule
-             * #getMaxAgrGroupSize()
-             */
+    public void testGoodBecomeBadDesc() throws IOException {
+        AggregationGroupRule rule = new AggregationGroupRule() {
             @Override
             protected int getMaxAgrGroupSize() {
-                return 3;
+                return 4;
             }
         };
-        rule.validate(cube, vContext);
+
+        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("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"));
+        assertEquals(0, vContext.getResults().length);
     }
 
     @Test
-    public void testAggColumnSize() {
+    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() {
-            /*
-             * (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"));
+
+        return rule;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15d489c1/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 6b53df8..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,41 +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.Ignore;
 import org.junit.Test;
 
-/**
- * @author jianliu
- * 
- */
-@Ignore
 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/incubator-kylin/blob/15d489c1/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/incubator-kylin/blob/15d489c1/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/incubator-kylin/blob/15d489c1/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



[2/2] incubator-kylin git commit: temp

Posted by ma...@apache.org.
temp


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

Branch: refs/heads/KYLIN-242-new
Commit: bb192414fd034e88374174c7cb6bdd0c44190378
Parents: 15d489c
Author: honma <ho...@ebay.com>
Authored: Tue Dec 1 19:09:34 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Tue Dec 1 19:09:34 2015 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bb192414/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
index 7b12cd7..59037b7 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
@@ -101,7 +101,7 @@ public class RowKeyColDesc {
         if (this.colEncoding.type == ColEncodingType.FIXED_LEN) {
             return (Integer) this.colEncoding.param;
         } else {
-            throw new IllegalStateException("Not knowing the col's length");
+            throw new IllegalStateException("Not knowing the col's length, col: " + this.column + " encoding: " + this.encoding + " encoding type: " + this.colEncoding.type);
         }
     }