You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2016/10/22 04:42:37 UTC

[1/2] kylin git commit: KYLIN-1919 performance enhancement

Repository: kylin
Updated Branches:
  refs/heads/KYLIN-1971 5156ccd47 -> b1a965097


KYLIN-1919 performance enhancement


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

Branch: refs/heads/KYLIN-1971
Commit: 3142c74cb22ea2b80ba36de462da69197020e1b3
Parents: 5156ccd
Author: shaofengshi <sh...@apache.org>
Authored: Fri Oct 21 18:30:44 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Oct 21 18:32:52 2016 +0800

----------------------------------------------------------------------
 .../kylin/source/kafka/StreamingParser.java     |  49 +++--
 .../source/kafka/TimedJsonStreamParser.java     |  31 ++--
 .../test/java/TimedJsonStreamParserTest.java    | 178 -------------------
 .../source/kafka/TimedJsonStreamParserTest.java | 166 +++++++++++++++++
 source-kafka/src/test/resources/message.json    |  10 +-
 5 files changed, 217 insertions(+), 217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/3142c74c/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java
index 43b2ac5..75f9c4b 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java
@@ -21,7 +21,6 @@ package org.apache.kylin.source.kafka;
 import java.lang.reflect.Constructor;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import com.google.common.collect.Maps;
 import org.apache.commons.lang3.StringUtils;
@@ -31,7 +30,6 @@ import org.apache.kylin.common.util.StreamingMessage;
 import org.apache.kylin.common.util.TimeUtil;
 import org.apache.kylin.metadata.model.TblColRef;
 
-import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,15 +45,15 @@ public abstract class StreamingParser {
     public static final String EMBEDDED_PROPERTY_SEPARATOR = "separator";
 
     public static final Map<String, String> defaultProperties = Maps.newHashMap();
-    public static final Set derivedTimeColumns = Sets.newHashSet();
+    public static final Map<String, Integer> derivedTimeColumns = Maps.newHashMap();
     static {
-        derivedTimeColumns.add("minute_start");
-        derivedTimeColumns.add("hour_start");
-        derivedTimeColumns.add("day_start");
-        derivedTimeColumns.add("week_start");
-        derivedTimeColumns.add("month_start");
-        derivedTimeColumns.add("quarter_start");
-        derivedTimeColumns.add("year_start");
+        derivedTimeColumns.put("minute_start", 1);
+        derivedTimeColumns.put("hour_start", 2);
+        derivedTimeColumns.put("day_start", 3);
+        derivedTimeColumns.put("week_start", 4);
+        derivedTimeColumns.put("month_start", 5);
+        derivedTimeColumns.put("quarter_start", 6);
+        derivedTimeColumns.put("year_start", 7);
         defaultProperties.put(PROPERTY_TS_COLUMN_NAME, "timestamp");
         defaultProperties.put(PROPERTY_TS_PARSER, "org.apache.kylin.source.kafka.DefaultTimeParser");
         defaultProperties.put(PROPERTY_TS_PATTERN, DateFormat.DEFAULT_DATETIME_PATTERN_WITHOUT_MILLISECONDS);
@@ -108,32 +106,45 @@ public abstract class StreamingParser {
      * @return true if the columnName is a derived time column; otherwise false;
      */
     public static final boolean populateDerivedTimeColumns(String columnName, List<String> result, long t) {
-        if (derivedTimeColumns.contains(columnName) == false)
+
+        Integer derivedTimeColumn = derivedTimeColumns.get(columnName);
+        if (derivedTimeColumn == null) {
             return false;
+        }
 
         long normalized = 0;
-        if (columnName.equals("minute_start")) {
+        switch (derivedTimeColumn) {
+        case 1:
             normalized = TimeUtil.getMinuteStart(t);
             result.add(DateFormat.formatToTimeWithoutMilliStr(normalized));
-        } else if (columnName.equals("hour_start")) {
+            break;
+        case 2:
             normalized = TimeUtil.getHourStart(t);
             result.add(DateFormat.formatToTimeWithoutMilliStr(normalized));
-        } else if (columnName.equals("day_start")) {
-            //from day_start on, formatTs will output date format
+            break;
+        case 3:
             normalized = TimeUtil.getDayStart(t);
             result.add(DateFormat.formatToDateStr(normalized));
-        } else if (columnName.equals("week_start")) {
+            break;
+        case 4:
             normalized = TimeUtil.getWeekStart(t);
             result.add(DateFormat.formatToDateStr(normalized));
-        } else if (columnName.equals("month_start")) {
+            break;
+        case 5:
             normalized = TimeUtil.getMonthStart(t);
             result.add(DateFormat.formatToDateStr(normalized));
-        } else if (columnName.equals("quarter_start")) {
+            break;
+        case 6:
             normalized = TimeUtil.getQuarterStart(t);
             result.add(DateFormat.formatToDateStr(normalized));
-        } else if (columnName.equals("year_start")) {
+            break;
+        case 7:
             normalized = TimeUtil.getYearStart(t);
             result.add(DateFormat.formatToDateStr(normalized));
+            break;
+        default:
+            throw new IllegalStateException();
+
         }
 
         return true;

http://git-wip-us.apache.org/repos/asf/kylin/blob/3142c74c/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java
index 633a30c..e00ce16 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java
@@ -64,6 +64,9 @@ public final class TimedJsonStreamParser extends StreamingParser {
     private String tsColName = null;
     private String tsParser = null;
     private String separator = null;
+    private final Map<String, Object> root = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+    private final Map<String, Object> tempMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+    private final Map<String, String[]> nameMap = new HashMap<>();
 
     private final JavaType mapType = MapType.construct(HashMap.class, SimpleType.construct(String.class), SimpleType.construct(Object.class));
 
@@ -100,15 +103,14 @@ public final class TimedJsonStreamParser extends StreamingParser {
     public StreamingMessage parse(ByteBuffer buffer) {
         try {
             Map<String, Object> message = mapper.readValue(new ByteBufferBackedInputStream(buffer), mapType);
-            Map<String, Object> root = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+            root.clear();
             root.putAll(message);
             String tsStr = objToString(root.get(tsColName));
             long t = streamTimeParser.parseTime(tsStr);
             ArrayList<String> result = Lists.newArrayList();
 
             for (TblColRef column : allColumns) {
-                String columnName = column.getName();
-                columnName = columnName.toLowerCase();
+                final String columnName = column.getName().toLowerCase();
                 if (populateDerivedTimeColumns(columnName, result, t) == false) {
                     result.add(getValueByKey(columnName, root));
                 }
@@ -126,18 +128,24 @@ public final class TimedJsonStreamParser extends StreamingParser {
         return true;
     }
 
-    protected String getValueByKey(String key, Map<String, Object> root) throws IOException {
-        if (root.containsKey(key)) {
-            return objToString(root.get(key));
+    protected String getValueByKey(String key, Map<String, Object> rootMap) throws IOException {
+        if (rootMap.containsKey(key)) {
+            return objToString(rootMap.get(key));
         }
 
-        if (key.contains(separator)) {
-            String[] names = key.toLowerCase().split(separator);
-            Map<String, Object> tempMap = root;
+        String[] names = nameMap.get(key);
+        if (names == null && key.contains(separator)) {
+            names = key.toLowerCase().split(separator);
+            nameMap.put(key, names);
+        }
+
+        if (names != null && names.length > 0) {
+            tempMap.clear();
+            tempMap.putAll(rootMap);
             for (int i = 0; i < names.length - 1; i++) {
-                Object o = root.get(names[i]);
+                Object o = tempMap.get(names[i]);
                 if (o instanceof Map) {
-                    tempMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+                    tempMap.clear();
                     tempMap.putAll((Map<String, Object>) o);
                 } else {
                     throw new IOException("Property '" + names[i] + "' is not embedded format");
@@ -145,7 +153,6 @@ public final class TimedJsonStreamParser extends StreamingParser {
             }
             Object finalObject = tempMap.get(names[names.length - 1]);
             return objToString(finalObject);
-
         }
 
         return StringUtils.EMPTY;

http://git-wip-us.apache.org/repos/asf/kylin/blob/3142c74c/source-kafka/src/test/java/TimedJsonStreamParserTest.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/test/java/TimedJsonStreamParserTest.java b/source-kafka/src/test/java/TimedJsonStreamParserTest.java
deleted file mode 100644
index 5a52b61..0000000
--- a/source-kafka/src/test/java/TimedJsonStreamParserTest.java
+++ /dev/null
@@ -1,178 +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.
- */
-
-import com.fasterxml.jackson.databind.JavaType;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import com.fasterxml.jackson.databind.type.MapType;
-import com.fasterxml.jackson.databind.type.SimpleType;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.kylin.common.util.StreamingMessage;
-import org.apache.kylin.metadata.model.ColumnDesc;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.source.kafka.TimedJsonStreamParser;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.List;
-import java.util.HashMap;
-import java.util.ArrayList;
-
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-
-
-public class TimedJsonStreamParserTest extends LocalFileMetadataTestCase {
-
-    private static String[] userNeedColNames;
-
-    private static final String jsonFilePath = "src/test/resources/message.json";
-
-    private static ObjectMapper mapper;
-
-    private final JavaType mapType = MapType.construct(HashMap.class, SimpleType.construct(String.class),
-            SimpleType.construct(Object.class));
-
-
-    @BeforeClass
-    public static void setUp() throws Exception {
-        staticCreateTestMetadata();
-        mapper = new ObjectMapper();
-    }
-
-    @AfterClass
-    public static void after() throws Exception {
-        cleanAfterClass();
-    }
-
-
-    @Test
-    public void testNormalValue() throws Exception {
-        userNeedColNames = new String[]{"createdAt", "id", "isTruncated", "text"};
-        List<TblColRef> allCol = mockupTblColRefList();
-        TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
-        Object msg = mapper.readValue(new File(jsonFilePath), mapType);
-        ByteBuffer buffer = getJsonByteBuffer(msg);
-        StreamingMessage sMsg = parser.parse(buffer);
-        List<String> result = sMsg.getData();
-        assertEquals("Jul 20, 2016 9:59:17 AM", result.get(0));
-        assertEquals("755703618762862600", result.get(1));
-        assertEquals("false", result.get(2));
-        assertEquals("dejamos las tapas regionales de este #Miercoles https://t.co/kfe0kT2Fup", result.get(3));
-    }
-
-    @Test
-    public void testEmbeddedValue() throws Exception {
-        userNeedColNames = new String[]{"user_id", "user_description", "user_isProtected"};
-        List<TblColRef> allCol = mockupTblColRefList();
-        TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
-        Object msg = mapper.readValue(new File(jsonFilePath), mapType);
-        ByteBuffer buffer = getJsonByteBuffer(msg);
-        StreamingMessage sMsg = parser.parse(buffer);
-        List<String> result = sMsg.getData();
-        assertEquals("4853763947", result.get(0));
-        assertEquals("Noticias, an\ufffd\ufffdlisis e informaci\ufffd\ufffdn para el crecimiento de la regi\ufffd\ufffdn.", result.get(1));
-        assertEquals("false", result.get(2));
-    }
-
-    @Test
-    public void testArrayValue() throws Exception {
-        userNeedColNames = new String[]{"userMentionEntities", "mediaEntities"};
-        List<TblColRef> allCol = mockupTblColRefList();
-        TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
-        Object msg = mapper.readValue(new File(jsonFilePath), mapType);
-        HashMap<String, Object> map = (HashMap<String, Object>) msg;
-        Object array = map.get("mediaEntities");
-        ByteBuffer buffer = getJsonByteBuffer(msg);
-        StreamingMessage sMsg = parser.parse(buffer);
-        List<String> result = sMsg.getData();
-        System.out.println(result);
-
-    }
-
-    @Test
-    public void testMapValue() throws Exception {
-        userNeedColNames = new String[]{"user"};
-        List<TblColRef> allCol = mockupTblColRefList();
-        TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
-        Object msg = mapper.readValue(new File(jsonFilePath), mapType);
-        ByteBuffer buffer = getJsonByteBuffer(msg);
-        StreamingMessage sMsg = parser.parse(buffer);
-        List<String> result = sMsg.getData();
-        System.out.println("result:" + result);
-
-    }
-
-    @Test
-    public void testNullKey() throws Exception {
-        userNeedColNames = new String[]{"null", ""};
-        List<TblColRef> allCol = mockupTblColRefList();
-        TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
-        Object msg = mapper.readValue(new File(jsonFilePath), mapType);
-        ByteBuffer buffer = getJsonByteBuffer(msg);
-        StreamingMessage sMsg = parser.parse(buffer);
-        List<String> result = sMsg.getData();
-        assertEquals(StringUtils.EMPTY, result.get(0));
-        assertEquals(StringUtils.EMPTY, result.get(1));
-    }
-
-
-    private static ByteBuffer getJsonByteBuffer(Object obj) throws IOException {
-        byte[] bytes = mapper.writeValueAsBytes(obj);
-        ByteBuffer buff = ByteBuffer.wrap(bytes);
-        buff.position(0);
-        return buff;
-    }
-
-
-    private static List<TblColRef> mockupTblColRefList() {
-        TableDesc t = mockupTableDesc("table_a");
-        List<TblColRef> list = new ArrayList<>();
-        for (int i = 0; i < userNeedColNames.length; i++) {
-            ColumnDesc c = mockupColumnDesc(t, i, userNeedColNames[i], "string");
-            list.add(c.getRef());
-        }
-        return list;
-    }
-
-    private static TableDesc mockupTableDesc(String tableName) {
-        TableDesc mockup = new TableDesc();
-        mockup.setName(tableName);
-        return mockup;
-    }
-
-    private static ColumnDesc mockupColumnDesc(TableDesc table, int oneBasedColumnIndex, String name, String datatype) {
-        ColumnDesc desc = new ColumnDesc();
-        String id = "" + oneBasedColumnIndex;
-        desc.setId(id);
-        desc.setName(name);
-        desc.setDatatype(datatype);
-        desc.init(table);
-        return desc;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/3142c74c/source-kafka/src/test/java/org/apache/kylin/source/kafka/TimedJsonStreamParserTest.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/test/java/org/apache/kylin/source/kafka/TimedJsonStreamParserTest.java b/source-kafka/src/test/java/org/apache/kylin/source/kafka/TimedJsonStreamParserTest.java
new file mode 100644
index 0000000..f92a24e
--- /dev/null
+++ b/source-kafka/src/test/java/org/apache/kylin/source/kafka/TimedJsonStreamParserTest.java
@@ -0,0 +1,166 @@
+/*
+ * 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.source.kafka;
+
+import com.fasterxml.jackson.databind.JavaType;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import com.fasterxml.jackson.databind.type.MapType;
+import com.fasterxml.jackson.databind.type.SimpleType;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kylin.common.util.StreamingMessage;
+import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.HashMap;
+import java.util.ArrayList;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+
+public class TimedJsonStreamParserTest extends LocalFileMetadataTestCase {
+
+    private static String[] userNeedColNames;
+    private static final String jsonFilePath = "src/test/resources/message.json";
+    private static ObjectMapper mapper;
+    private final JavaType mapType = MapType.construct(HashMap.class, SimpleType.construct(String.class), SimpleType.construct(Object.class));
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        staticCreateTestMetadata();
+        mapper = new ObjectMapper();
+    }
+
+    @AfterClass
+    public static void after() throws Exception {
+        cleanAfterClass();
+    }
+
+    @Test
+    public void testNormalValue() throws Exception {
+        userNeedColNames = new String[] { "createdAt", "id", "isTruncated", "text" };
+        List<TblColRef> allCol = mockupTblColRefList();
+        TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
+        Object msg = mapper.readValue(new File(jsonFilePath), mapType);
+        ByteBuffer buffer = getJsonByteBuffer(msg);
+        StreamingMessage sMsg = parser.parse(buffer);
+        List<String> result = sMsg.getData();
+        assertEquals("Jul 20, 2016 9:59:17 AM", result.get(0));
+        assertEquals("755703618762862600", result.get(1));
+        assertEquals("false", result.get(2));
+        assertEquals("dejamos", result.get(3));
+    }
+
+    @Test
+    public void testEmbeddedValue() throws Exception {
+        userNeedColNames = new String[] { "user_id", "user_description", "user_isProtected" };
+        List<TblColRef> allCol = mockupTblColRefList();
+        TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
+        Object msg = mapper.readValue(new File(jsonFilePath), mapType);
+        ByteBuffer buffer = getJsonByteBuffer(msg);
+        StreamingMessage sMsg = parser.parse(buffer);
+        List<String> result = sMsg.getData();
+        assertEquals("4853763947", result.get(0));
+        assertEquals("Noticias", result.get(1));
+        assertEquals("false", result.get(2));
+    }
+
+    @Test
+    public void testArrayValue() throws Exception {
+        userNeedColNames = new String[] { "userMentionEntities", "mediaEntities" };
+        List<TblColRef> allCol = mockupTblColRefList();
+        TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
+        Object msg = mapper.readValue(new File(jsonFilePath), mapType);
+        HashMap<String, Object> map = (HashMap<String, Object>) msg;
+        Object array = map.get("mediaEntities");
+        ByteBuffer buffer = getJsonByteBuffer(msg);
+        StreamingMessage sMsg = parser.parse(buffer);
+        List<String> result = sMsg.getData();
+        System.out.println(result);
+
+    }
+
+    @Test
+    public void testMapValue() throws Exception {
+        userNeedColNames = new String[] { "user" };
+        List<TblColRef> allCol = mockupTblColRefList();
+        TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
+        Object msg = mapper.readValue(new File(jsonFilePath), mapType);
+        ByteBuffer buffer = getJsonByteBuffer(msg);
+        StreamingMessage sMsg = parser.parse(buffer);
+        List<String> result = sMsg.getData();
+
+    }
+
+    @Test
+    public void testNullKey() throws Exception {
+        userNeedColNames = new String[] { "null", "" };
+        List<TblColRef> allCol = mockupTblColRefList();
+        TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
+        Object msg = mapper.readValue(new File(jsonFilePath), mapType);
+        ByteBuffer buffer = getJsonByteBuffer(msg);
+        StreamingMessage sMsg = parser.parse(buffer);
+        List<String> result = sMsg.getData();
+        assertEquals(StringUtils.EMPTY, result.get(0));
+        assertEquals(StringUtils.EMPTY, result.get(1));
+    }
+
+    private static ByteBuffer getJsonByteBuffer(Object obj) throws IOException {
+        byte[] bytes = mapper.writeValueAsBytes(obj);
+        ByteBuffer buff = ByteBuffer.wrap(bytes);
+        buff.position(0);
+        return buff;
+    }
+
+    private static List<TblColRef> mockupTblColRefList() {
+        TableDesc t = mockupTableDesc("table_a");
+        List<TblColRef> list = new ArrayList<>();
+        for (int i = 0; i < userNeedColNames.length; i++) {
+            ColumnDesc c = mockupColumnDesc(t, i, userNeedColNames[i], "string");
+            list.add(c.getRef());
+        }
+        return list;
+    }
+
+    private static TableDesc mockupTableDesc(String tableName) {
+        TableDesc mockup = new TableDesc();
+        mockup.setName(tableName);
+        return mockup;
+    }
+
+    private static ColumnDesc mockupColumnDesc(TableDesc table, int oneBasedColumnIndex, String name, String datatype) {
+        ColumnDesc desc = new ColumnDesc();
+        String id = "" + oneBasedColumnIndex;
+        desc.setId(id);
+        desc.setName(name);
+        desc.setDatatype(datatype);
+        desc.init(table);
+        return desc;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/3142c74c/source-kafka/src/test/resources/message.json
----------------------------------------------------------------------
diff --git a/source-kafka/src/test/resources/message.json b/source-kafka/src/test/resources/message.json
index dfafd45..55f35d3 100644
--- a/source-kafka/src/test/resources/message.json
+++ b/source-kafka/src/test/resources/message.json
@@ -1,8 +1,7 @@
 {
   "createdAt": "Jul 20, 2016 9:59:17 AM",
   "id": 755703618762862600,
-  "text": "dejamos las tapas regionales de este #Miercoles https://t.co/kfe0kT2Fup",
-  "source": "<a href=\"http://twitter.com\" rel=\"nofollow\">Twitter Web Client</a>",
+  "text": "dejamos",
   "isTruncated": false,
   "inReplyToStatusId": -1,
   "inReplyToUserId": -1,
@@ -15,8 +14,6 @@
   "mediaEntities": [
     {
       "id": 755703584084328400,
-      "url": "https://t.co/kfe0kT2Fup",
-      "displayURL": "pic.twitter.com/kfe0kT2Fup",
       "sizes": {
         "0": {
           "width": 150,
@@ -38,11 +35,8 @@
   "currentUserRetweetId": -1,
   "user": {
     "id": 4853763947,
-    "name": "El Metropolitano",
-    "screenName": "ElTWdelMetro",
-    "description": "Noticias, an\ufffd\ufffdlisis e informaci\ufffd\ufffdn para el crecimiento de la regi\ufffd\ufffdn.",
+    "description": "Noticias",
     "isDefaultProfileImage": false,
-    "url": "http://elmetropolitano.com.ar/",
     "isProtected": false
   }
 }
\ No newline at end of file


[2/2] kylin git commit: Cube meta upgrade

Posted by li...@apache.org.
Cube meta upgrade


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

Branch: refs/heads/KYLIN-1971
Commit: b1a965097d739b783759100804cf7d9af7f665a0
Parents: 3142c74
Author: Yang Li <li...@apache.org>
Authored: Sat Oct 22 12:42:19 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sat Oct 22 12:42:19 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/cube/CubeDescManager.java  | 14 ++-
 .../java/org/apache/kylin/cube/CubeManager.java | 18 ----
 .../kylin/cube/model/AggregationGroup.java      | 48 ++++++----
 .../org/apache/kylin/cube/model/CubeDesc.java   | 97 +++++---------------
 .../apache/kylin/cube/model/DimensionDesc.java  | 42 +++------
 .../apache/kylin/cube/model/RowKeyColDesc.java  | 10 +-
 .../org/apache/kylin/cube/model/RowKeyDesc.java | 17 ++--
 .../kylin/cube/model/v1_4_0/CubeDesc.java       |  4 +-
 .../org/apache/kylin/cube/CubeDescTest.java     | 27 +++---
 .../validation/rule/DictionaryRuleTest.java     |  7 +-
 .../model/validation/rule/FunctionRuleTest.java | 21 ++---
 .../kylin/metadata/model/FunctionDesc.java      | 21 +----
 .../hbase/util/ExtendCubeToHybridCLI.java       |  2 +-
 13 files changed, 108 insertions(+), 220 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
index d6364fe..50312bf 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
@@ -32,7 +32,6 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.validation.CubeMetadataValidator;
 import org.apache.kylin.cube.model.validation.ValidateContext;
 import org.apache.kylin.metadata.MetadataConstants;
-import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.cachesync.Broadcaster;
 import org.apache.kylin.metadata.cachesync.Broadcaster.Event;
 import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache;
@@ -176,8 +175,9 @@ public class CubeDescManager {
             throw new IllegalArgumentException("No cube desc found at " + path);
 
         try {
-            ndesc.init(config, getMetadataManager().getAllTablesMap());
+            ndesc.init(config);
         } catch (Exception e) {
+            logger.warn("Broken cube desc " + path, e);
             ndesc.addError(e.getMessage());
         }
 
@@ -202,8 +202,9 @@ public class CubeDescManager {
             throw new IllegalArgumentException("CubeDesc '" + cubeDesc.getName() + "' already exists");
 
         try {
-            cubeDesc.init(config, getMetadataManager().getAllTablesMap());
+            cubeDesc.init(config);
         } catch (Exception e) {
+            logger.warn("Broken cube desc " + cubeDesc, e);
             cubeDesc.addError(e.getMessage());
         }
         // Check base validation
@@ -283,8 +284,9 @@ public class CubeDescManager {
         }
 
         try {
-            desc.init(config, getMetadataManager().getAllTablesMap());
+            desc.init(config);
         } catch (Exception e) {
+            logger.warn("Broken cube desc " + desc, e);
             desc.addError(e.getMessage());
             return desc;
         }
@@ -310,10 +312,6 @@ public class CubeDescManager {
         return ndesc;
     }
 
-    private MetadataManager getMetadataManager() {
-        return MetadataManager.getInstance(config);
-    }
-
     private ResourceStore getStore() {
         return ResourceStore.getStore(this.config);
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
index c558c6d..a53849e 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -647,24 +647,6 @@ public class CubeManager implements IRealizationProvider {
         }
     }
 
-    private long calculateStartOffsetForAppendSegment(CubeInstance cube) {
-        List<CubeSegment> existing = cube.getSegments();
-        if (existing.isEmpty()) {
-            return 0;
-        } else {
-            return existing.get(existing.size() - 1).getSourceOffsetEnd();
-        }
-    }
-
-    private long calculateStartDateForAppendSegment(CubeInstance cube) {
-        List<CubeSegment> existing = cube.getSegments();
-        if (existing.isEmpty()) {
-            return cube.getDescriptor().getPartitionDateStart();
-        } else {
-            return existing.get(existing.size() - 1).getDateRangeEnd();
-        }
-    }
-
     private void checkBuildingSegment(CubeInstance cube) {
         int maxBuldingSeg = cube.getConfig().getMaxBuildingSegments();
         if (cube.getBuildingSegments().size() >= maxBuldingSeg) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java b/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
index 601ee0a..6e76ac3 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
@@ -20,8 +20,8 @@ package org.apache.kylin.cube.model;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
+import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.metadata.model.TblColRef;
 
@@ -59,36 +59,52 @@ public class AggregationGroup {
     public void init(CubeDesc cubeDesc, RowKeyDesc rowKeyDesc) {
         this.cubeDesc = cubeDesc;
         this.isMandatoryOnlyValid = cubeDesc.getConfig().getCubeAggrGroupIsMandatoryOnlyValid();
-        Map<String, TblColRef> colNameAbbr = cubeDesc.buildColumnNameAbbreviation();
 
         if (this.includes == null || this.includes.length == 0 || this.selectRule == null) {
             throw new IllegalStateException("AggregationGroup incomplete");
         }
 
-        buildPartialCubeFullMask(colNameAbbr, rowKeyDesc);
-        buildMandatoryColumnMask(colNameAbbr, rowKeyDesc);
-        buildHierarchyMasks(colNameAbbr, rowKeyDesc);
-        buildJointColumnMask(colNameAbbr, rowKeyDesc);
+        columnNamesToUpperCase();
+        
+        buildPartialCubeFullMask(rowKeyDesc);
+        buildMandatoryColumnMask(rowKeyDesc);
+        buildHierarchyMasks(rowKeyDesc);
+        buildJointColumnMask(rowKeyDesc);
         buildJointDimsMask();
         buildNormalDimsMask();
         buildHierarchyDimsMask();
 
     }
 
-    private void buildPartialCubeFullMask(Map<String, TblColRef> colNameAbbr, RowKeyDesc rowKeyDesc) {
+    private void columnNamesToUpperCase() {
+        StringUtil.toUpperCaseArray(includes, includes);
+        StringUtil.toUpperCaseArray(selectRule.mandatory_dims, selectRule.mandatory_dims);
+        if (selectRule.hierarchy_dims != null) {
+            for (String[] cols : selectRule.hierarchy_dims) {
+                StringUtil.toUpperCaseArray(cols, cols);
+            }
+        }
+        if (selectRule.joint_dims != null) {
+            for (String[] cols : selectRule.joint_dims) {
+                StringUtil.toUpperCaseArray(cols, cols);
+            }
+        }
+    }
+
+    private void buildPartialCubeFullMask(RowKeyDesc rowKeyDesc) {
         Preconditions.checkState(this.includes != null);
         Preconditions.checkState(this.includes.length != 0);
 
         partialCubeFullMask = 0L;
         for (String dim : this.includes) {
-            TblColRef hColumn = colNameAbbr.get(dim);
+            TblColRef hColumn = cubeDesc.getModel().findColumn(dim);
             Integer index = rowKeyDesc.getColumnBitIndex(hColumn);
             long bit = 1L << index;
             partialCubeFullMask |= bit;
         }
     }
 
-    private void buildJointColumnMask(Map<String, TblColRef> colNameAbbr, RowKeyDesc rowKeyDesc) {
+    private void buildJointColumnMask(RowKeyDesc rowKeyDesc) {
         joints = Lists.newArrayList();
 
         if (this.selectRule.joint_dims == null || this.selectRule.joint_dims.length == 0) {
@@ -102,7 +118,7 @@ public class AggregationGroup {
 
             long joint = 0L;
             for (int i = 0; i < joint_dims.length; i++) {
-                TblColRef hColumn = colNameAbbr.get(joint_dims[i]);
+                TblColRef hColumn = cubeDesc.getModel().findColumn(joint_dims[i]);
                 Integer index = rowKeyDesc.getColumnBitIndex(hColumn);
                 long bit = 1L << index;
                 joint |= bit;
@@ -113,7 +129,7 @@ public class AggregationGroup {
         }
     }
 
-    private void buildMandatoryColumnMask(Map<String, TblColRef> colNameAbbr, RowKeyDesc rowKeyDesc) {
+    private void buildMandatoryColumnMask(RowKeyDesc rowKeyDesc) {
         mandatoryColumnMask = 0L;
 
         String[] mandatory_dims = this.selectRule.mandatory_dims;
@@ -122,14 +138,14 @@ public class AggregationGroup {
         }
 
         for (String dim : mandatory_dims) {
-            TblColRef hColumn = colNameAbbr.get(dim);
+            TblColRef hColumn = cubeDesc.getModel().findColumn(dim);
             Integer index = rowKeyDesc.getColumnBitIndex(hColumn);
             mandatoryColumnMask |= 1 << index;
         }
 
     }
 
-    private void buildHierarchyMasks(Map<String, TblColRef> colNameAbbr, RowKeyDesc rowKeyDesc) {
+    private void buildHierarchyMasks(RowKeyDesc rowKeyDesc) {
         this.hierarchyMasks = new ArrayList<HierarchyMask>();
 
         if (this.selectRule.hierarchy_dims == null || this.selectRule.hierarchy_dims.length == 0) {
@@ -145,14 +161,10 @@ public class AggregationGroup {
             ArrayList<Long> allMaskList = new ArrayList<Long>();
             ArrayList<Long> dimList = new ArrayList<Long>();
             for (int i = 0; i < hierarchy_dims.length; i++) {
-                TblColRef hColumn = colNameAbbr.get(hierarchy_dims[i]);
+                TblColRef hColumn = cubeDesc.getModel().findColumn(hierarchy_dims[i]);
                 Integer index = rowKeyDesc.getColumnBitIndex(hColumn);
                 long bit = 1L << index;
 
-                //                if ((tailMask & bit) > 0)
-                //                    continue; // ignore levels in tail, they don't participate
-                //                // aggregation group combination anyway
-
                 mask.fullMask |= bit;
                 allMaskList.add(mask.fullMask);
                 dimList.add(bit);

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index 7db460e..64360d5 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -49,13 +49,11 @@ import org.apache.kylin.common.KylinVersion;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
 import org.apache.kylin.common.util.Array;
-import org.apache.kylin.common.util.CaseInsensitiveStringMap;
 import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.measure.extendedcolumn.ExtendedColumnMeasureType;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.IEngineAware;
@@ -63,6 +61,7 @@ import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.JoinDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -161,7 +160,6 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
     @JsonProperty("override_kylin_properties")
     private LinkedHashMap<String, String> overrideKylinProps = new LinkedHashMap<String, String>();
 
-    private Map<String, Map<String, TblColRef>> columnMap = new HashMap<String, Map<String, TblColRef>>();
     private LinkedHashSet<TblColRef> allColumns = new LinkedHashSet<TblColRef>();
     private LinkedHashSet<TblColRef> dimensionColumns = new LinkedHashSet<TblColRef>();
 
@@ -233,11 +231,7 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
     }
 
     public TblColRef findColumnRef(String table, String column) {
-        Map<String, TblColRef> cols = columnMap.get(table);
-        if (cols == null)
-            return null;
-        else
-            return cols.get(column);
+        return model.findColumn(table, column);
     }
 
     public DimensionDesc findDimensionByTable(String lookupTableName) {
@@ -522,15 +516,7 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
         }
     }
 
-    public Map<String, TblColRef> buildColumnNameAbbreviation() {
-        Map<String, TblColRef> r = new CaseInsensitiveStringMap<TblColRef>();
-        for (TblColRef col : listDimensionColumnsExcludingDerived(true)) {
-            r.put(col.getName(), col);
-        }
-        return r;
-    }
-
-    public void init(KylinConfig config, Map<String, TableDesc> tables) {
+    public void init(KylinConfig config) {
         this.errors.clear();
         this.config = KylinConfigExt.createInstance(config, overrideKylinProps);
 
@@ -540,11 +526,8 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
         this.model = MetadataManager.getInstance(config).getDataModelDesc(modelName);
         checkNotNull(this.model, "DateModelDesc(%s) not found", modelName);
 
-        // check if aggregation group is valid
-        validate();
-
         for (DimensionDesc dim : dimensions) {
-            dim.init(this, tables);
+            dim.init(this);
         }
 
         initDimensionColumns();
@@ -554,6 +537,7 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
         for (AggregationGroup agg : this.aggregationGroups) {
             agg.init(this, rowkey);
         }
+        validateAggregationGroups(); // check if aggregation group is valid
 
         if (hbaseMapping != null) {
             hbaseMapping.init(this);
@@ -563,14 +547,12 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
 
         // check all dimension columns are presented on rowkey
         List<TblColRef> dimCols = listDimensionColumnsExcludingDerived(true);
-        checkState(rowkey.getRowKeyColumns().length == dimCols.size(),
-                "RowKey columns count (%d) doesn't match dimensions columns count (%d)",
-                rowkey.getRowKeyColumns().length, dimCols.size());
+        checkState(rowkey.getRowKeyColumns().length == dimCols.size(), "RowKey columns count (%d) doesn't match dimensions columns count (%d)", rowkey.getRowKeyColumns().length, dimCols.size());
 
         initDictionaryDesc();
     }
 
-    public void validate() {
+    public void validateAggregationGroups() {
         int index = 0;
 
         for (AggregationGroup agg : getAggregationGroups()) {
@@ -706,27 +688,17 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
 
             // init dimension columns
             ArrayList<TblColRef> dimCols = Lists.newArrayList();
-            String colStrs = dim.getColumn();
+            String colStr = dim.getColumn();
 
-            if ((colStrs == null && dim.isDerived()) || ("{FK}".equalsIgnoreCase(colStrs))) {
+            if ((colStr == null && dim.isDerived()) || ("{FK}".equalsIgnoreCase(colStr))) {
                 // when column is omitted, special case
-
                 for (TblColRef col : join.getForeignKeyColumns()) {
                     dimCols.add(initDimensionColRef(col));
                 }
             } else {
                 // normal case
-
-                if (StringUtils.isEmpty(colStrs))
-                    throw new IllegalStateException("Dimension column must not be blank " + dim);
-
-                dimCols.add(initDimensionColRef(dim, colStrs));
-
-                //                // fill back column ref in hierarchy
-                //                if (dim.isHierarchy()) {
-                //                    for (int i = 0; i < dimCols.size(); i++)
-                //                        dim.getHierarchy()[i].setColumnRef(dimCols.get(i));
-                //                }
+                checkState(!StringUtils.isEmpty(colStr), "Dimension column must not be blank: %s", dim);
+                dimCols.add(initDimensionColRef(dim, colStr));
             }
 
             TblColRef[] dimColArray = dimCols.toArray(new TblColRef[dimCols.size()]);
@@ -759,15 +731,6 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
                         initDerivedMap(new TblColRef[] { dimColArray[find] }, DeriveType.PK_FK, dim, new TblColRef[] { derivedCol }, null);
                     }
                 }
-                /** disable this code as we don't need fk be derived from pk
-                 for (int i = 0; i < pk.length; i++) {
-                 int find = ArrayUtils.indexOf(hostCols, pk[i]);
-                 if (find >= 0) {
-                 TblColRef derivedCol = initDimensionColRef(fk[i]);
-                 initDerivedMap(hostCols[find], DeriveType.PK_FK, dim, derivedCol);
-                 }
-                 }
-                 */
             }
         }
     }
@@ -822,39 +785,25 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
     }
 
     private TblColRef initDimensionColRef(DimensionDesc dim, String colName) {
-        TableDesc table = dim.getTableDesc();
-        ColumnDesc col = table.findColumnByName(colName);
-        if (col == null)
-            throw new IllegalArgumentException("No column '" + colName + "' found in table " + table);
-
-        TblColRef ref = col.getRef();
+        TableRef table = dim.getTableRef();
+        TblColRef col = table.getColumn(colName);
+        checkArgument(col != null, "No column '%s' found in table %s", colName, table);
 
         // always use FK instead PK, FK could be shared by more than one lookup tables
         JoinDesc join = dim.getJoin();
         if (join != null) {
-            int idx = ArrayUtils.indexOf(join.getPrimaryKeyColumns(), ref);
+            int idx = ArrayUtils.indexOf(join.getPrimaryKeyColumns(), col);
             if (idx >= 0) {
-                ref = join.getForeignKeyColumns()[idx];
+                col = join.getForeignKeyColumns()[idx];
             }
         }
-        return initDimensionColRef(ref);
+        return initDimensionColRef(col);
     }
 
-    private TblColRef initDimensionColRef(TblColRef ref) {
-        TblColRef existing = findColumnRef(ref.getTable(), ref.getName());
-        if (existing != null) {
-            return existing;
-        }
-
-        allColumns.add(ref);
-        dimensionColumns.add(ref);
-
-        Map<String, TblColRef> cols = columnMap.get(ref.getTable());
-        if (cols == null) {
-            columnMap.put(ref.getTable(), cols = new HashMap<String, TblColRef>());
-        }
-        cols.put(ref.getName(), ref);
-        return ref;
+    private TblColRef initDimensionColRef(TblColRef col) {
+        allColumns.add(col);
+        dimensionColumns.add(col);
+        return col;
     }
 
     private void initMeasureColumns() {
@@ -862,8 +811,6 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
             return;
         }
 
-        TableDesc factTable = getFactTableDesc();
-        List<TableDesc> lookupTables = getLookupTableDescs();
         for (MeasureDesc m : measures) {
             m.setName(m.getName().toUpperCase());
 
@@ -872,7 +819,7 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
             }
 
             FunctionDesc func = m.getFunction();
-            func.init(factTable, lookupTables);
+            func.init(model);
             allColumns.addAll(func.getParameter().getColRefs());
 
             if (ExtendedColumnMeasureType.FUNC_RAW.equalsIgnoreCase(m.getFunction().getExpression())) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java
index 0214ff0..ae90a18 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java
@@ -19,13 +19,13 @@
 package org.apache.kylin.cube.model;
 
 import java.util.Arrays;
-import java.util.Map;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.JoinDesc;
 import org.apache.kylin.metadata.model.LookupDesc;
-import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
@@ -47,52 +47,32 @@ public class DimensionDesc {
     @JsonProperty("derived")
     private String[] derived;
 
-    private TableDesc tableDesc;
+    private TableRef tableRef;
     private JoinDesc join;
 
     // computed
     private TblColRef[] columnRefs;
 
-    public void init(CubeDesc cubeDesc, Map<String, TableDesc> tables) {
+    public void init(CubeDesc cubeDesc) {
         if (name != null)
             name = name.toUpperCase();
 
         if (table != null)
             table = table.toUpperCase();
 
-        tableDesc = tables.get(this.getTable());
-        if (tableDesc == null)
+        DataModelDesc model = cubeDesc.getModel();
+        tableRef = model.findTable(this.getTable());
+        if (tableRef == null)
             throw new IllegalStateException("Can't find table " + table + " for dimension " + name);
 
         join = null;
-        for (LookupDesc lookup : cubeDesc.getModel().getLookups()) {
-            if (lookup.getTable().equalsIgnoreCase(this.getTable())) {
+        for (LookupDesc lookup : model.getLookups()) {
+            if (lookup.getTableRef().equals(this.tableRef)) {
                 join = lookup.getJoin();
                 break;
             }
         }
 
-        //        if (isHierarchy && this.column.length > 0) {
-        //            List<HierarchyDesc> hierarchyList = new ArrayList<HierarchyDesc>(3);
-        //            for (int i = 0, n = this.column.length; i < n; i++) {
-        //                String aColumn = this.column[i];
-        //                HierarchyDesc aHierarchy = new HierarchyDesc();
-        //                aHierarchy.setLevel(String.valueOf(i + 1));
-        //                aHierarchy.setColumn(aColumn);
-        //                hierarchyList.add(aHierarchy);
-        //            }
-        //
-        //            this.hierarchy = hierarchyList.toArray(new HierarchyDesc[hierarchyList.size()]);
-        //        }
-        //
-        //        if (hierarchy != null && hierarchy.length == 0)
-        //            hierarchy = null;
-
-        //        if (hierarchy != null) {
-        //            for (HierarchyDesc h : hierarchy)
-        //                h.setColumn(h.getColumn().toUpperCase());
-        //        }
-
         if (derived != null && derived.length == 0) {
             derived = null;
         }
@@ -153,8 +133,8 @@ public class DimensionDesc {
         this.derived = derived;
     }
 
-    public TableDesc getTableDesc() {
-        return this.tableDesc;
+    public TableRef getTableRef() {
+        return this.tableRef;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/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 12c4dfc..9e2cb48 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
@@ -18,8 +18,6 @@
 
 package org.apache.kylin.cube.model;
 
-import java.util.Map;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.dimension.DateDimEnc;
 import org.apache.kylin.dimension.DictionaryDimEnc;
@@ -59,13 +57,11 @@ public class RowKeyColDesc {
     private int bitIndex;
     private TblColRef colRef;
 
-    public void init(int index, Map<String, TblColRef> colNameAbbr, CubeDesc cubeDesc) {
+    public void init(int index, CubeDesc cubeDesc) {
         column = column.toUpperCase();
         bitIndex = index;
-        colRef = colNameAbbr.get(column);
-        if (colRef == null) {
-            throw new IllegalArgumentException("Cannot find rowkey column " + column + " in cube " + cubeDesc);
-        }
+        colRef = cubeDesc.getModel().findColumn(column);
+        Preconditions.checkArgument(colRef != null, "Cannot find rowkey column %s in cube %s", column,  cubeDesc);
 
         Preconditions.checkState(StringUtils.isNotEmpty(this.encoding));
         Object[] encodingConf = DimensionEncoding.parseEncodingConf(this.encoding);

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/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 dfd82de..f1a403d 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
@@ -25,7 +25,6 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.lang.ArrayUtils;
-import org.apache.kylin.dimension.DictionaryDimEnc;
 import org.apache.kylin.metadata.model.TblColRef;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
@@ -76,16 +75,16 @@ public class RowKeyDesc {
     }
 
     public void init(CubeDesc cubeDesc) {
-
         setCubeDesc(cubeDesc);
-        Map<String, TblColRef> colNameAbbr = cubeDesc.buildColumnNameAbbreviation();
-
-        buildRowKey(colNameAbbr);
+        buildRowKey();
+        initColumnsNeedIndex();
+    }
 
+    private void initColumnsNeedIndex() {
         int[] tmp = new int[100];
         int x = 0;
         for (int i = 0, n = rowkeyColumns.length; i < n; i++) {
-            if ("true".equalsIgnoreCase(rowkeyColumns[i].getIndex()) && DictionaryDimEnc.ENCODING_NAME.equalsIgnoreCase(rowkeyColumns[i].getEncoding())) {
+            if ("true".equalsIgnoreCase(rowkeyColumns[i].getIndex()) && rowkeyColumns[i].isUsingDictionary()) {
                 tmp[x] = i;
                 x++;
             }
@@ -103,13 +102,13 @@ public class RowKeyDesc {
         return Objects.toStringHelper(this).add("RowKeyColumns", Arrays.toString(rowkeyColumns)).toString();
     }
 
-    private void buildRowKey(Map<String, TblColRef> colNameAbbr) {
-        columnMap = new HashMap<TblColRef, RowKeyColDesc>();
+    private void buildRowKey() {
+        columnMap = new HashMap<>();
         shardByColumns = new HashSet<>();
 
         for (int i = 0; i < rowkeyColumns.length; i++) {
             RowKeyColDesc rowKeyColDesc = rowkeyColumns[i];
-            rowKeyColDesc.init(rowkeyColumns.length - i - 1, colNameAbbr, cubeDesc);
+            rowKeyColDesc.init(rowkeyColumns.length - i - 1, cubeDesc);
             columnMap.put(rowKeyColDesc.getColRef(), rowKeyColDesc);
 
             if (rowKeyColDesc.isShardBy()) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/CubeDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/CubeDesc.java
index 760b3e7..24c9ceb 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/CubeDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/CubeDesc.java
@@ -647,8 +647,6 @@ public class CubeDesc extends RootPersistentEntity {
             return;
         }
 
-        TableDesc factTable = getFactTableDesc();
-        List<TableDesc> lookups = getLookupTableDescs();
         for (MeasureDesc m : measures) {
             m.setName(m.getName().toUpperCase());
 
@@ -657,7 +655,7 @@ public class CubeDesc extends RootPersistentEntity {
             }
 
             FunctionDesc func = m.getFunction();
-            func.init(factTable, lookups);
+            func.init(model);
             allColumns.addAll(func.getParameter().getColRefs());
 
             //            // verify holistic count distinct as a dependent measure

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/test/java/org/apache/kylin/cube/CubeDescTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/CubeDescTest.java b/core-cube/src/test/java/org/apache/kylin/cube/CubeDescTest.java
index 98cff43..db80025 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/CubeDescTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/CubeDescTest.java
@@ -26,7 +26,6 @@ import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.SelectRule;
-import org.apache.kylin.metadata.MetadataManager;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -57,7 +56,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
     @Test
     public void testGoodInit() throws Exception {
         CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
-        cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+        cubeDesc.init(getTestConfig());
     }
 
     @Test
@@ -69,7 +68,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
         String[] temp = null;
         cubeDesc.getAggregationGroups().get(0).setIncludes(temp);
 
-        cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+        cubeDesc.init(getTestConfig());
     }
 
     @Test
@@ -81,7 +80,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
         SelectRule temp = null;
         cubeDesc.getAggregationGroups().get(0).setSelectRule(temp);
 
-        cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+        cubeDesc.init(getTestConfig());
     }
 
     @Test
@@ -93,7 +92,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
         String[] temp = Arrays.asList(cubeDesc.getAggregationGroups().get(0).getIncludes()).subList(0, 3).toArray(new String[3]);
         cubeDesc.getAggregationGroups().get(0).setIncludes(temp);
 
-        cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+        cubeDesc.init(getTestConfig());
     }
 
     @Test
@@ -104,7 +103,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
         CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
         try {
             System.setProperty("kylin.cube.aggrgroup.max.combination", "8");
-            cubeDesc.validate();
+            cubeDesc.validateAggregationGroups();
         } finally {
             System.clearProperty("kylin.cube.aggrgroup.max.combination");
         }
@@ -115,7 +114,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
         CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
         cubeDesc.getAggregationGroups().get(0).getSelectRule().mandatory_dims = new String[] { "seller_id", "META_CATEG_NAME" };
 
-        cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+        cubeDesc.init(getTestConfig());
     }
 
     @Test
@@ -123,7 +122,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
         CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
         cubeDesc.getAggregationGroups().get(0).getSelectRule().mandatory_dims = new String[] { "seller_id", "lstg_format_name" };
 
-        cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+        cubeDesc.init(getTestConfig());
     }
 
     @Test
@@ -134,7 +133,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
         CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
         cubeDesc.getAggregationGroups().get(0).getSelectRule().joint_dims = new String[][] { new String[] { "lstg_format_name" } };
 
-        cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+        cubeDesc.init(getTestConfig());
     }
 
     @Test
@@ -145,7 +144,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
         CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
         cubeDesc.getAggregationGroups().get(0).getSelectRule().joint_dims = new String[][] { new String[] { "META_CATEG_NAME", "CATEG_LVL2_NAME" } };
 
-        cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+        cubeDesc.init(getTestConfig());
     }
 
     @Test
@@ -157,7 +156,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
         cubeDesc.getAggregationGroups().get(0).getSelectRule().hierarchy_dims = new String[][] { new String[] { "META_CATEG_NAME", "CATEG_LVL2_NAME", "CATEG_LVL3_NAME" }, new String[] { "lstg_format_name", "lstg_site_id" } };
         cubeDesc.getAggregationGroups().get(0).getSelectRule().joint_dims = new String[][] { new String[] { "META_CATEG_NAME", "lstg_format_name" } };
 
-        cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+        cubeDesc.init(getTestConfig());
     }
 
     @Test
@@ -168,7 +167,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
         CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
         cubeDesc.getAggregationGroups().get(0).getSelectRule().joint_dims = new String[][] { new String[] { "lstg_format_name", "lstg_site_id", "slr_segment_cd" }, new String[] { "lstg_format_name", "lstg_site_id", "leaf_categ_id" } };
 
-        cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+        cubeDesc.init(getTestConfig());
     }
 
     @Test
@@ -179,7 +178,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
         CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
         cubeDesc.getAggregationGroups().get(0).getSelectRule().hierarchy_dims = new String[][] { new String[] { "META_CATEG_NAME" } };
 
-        cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+        cubeDesc.init(getTestConfig());
     }
 
     @Test
@@ -190,7 +189,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
         CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
         cubeDesc.getAggregationGroups().get(0).getSelectRule().hierarchy_dims = new String[][] { new String[] { "META_CATEG_NAME", "CATEG_LVL2_NAME", "CATEG_LVL3_NAME" }, new String[] { "META_CATEG_NAME", "CATEG_LVL2_NAME" } };
 
-        cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+        cubeDesc.init(getTestConfig());
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
index fdbbb2c..5492ad8 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
@@ -32,7 +32,6 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.DictionaryDesc;
 import org.apache.kylin.cube.model.validation.ValidateContext;
 import org.apache.kylin.dict.GlobalDictionaryBuilder;
-import org.apache.kylin.metadata.MetadataManager;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -42,13 +41,11 @@ import org.junit.Test;
  */
 public class DictionaryRuleTest extends LocalFileMetadataTestCase {
     private static KylinConfig config;
-    private static MetadataManager metadataManager;
 
     @Before
     public void setUp() throws Exception {
         this.createTestMetadata();
         config = KylinConfig.getInstanceFromEnv();
-        metadataManager = MetadataManager.getInstance(config);
     }
 
     @After
@@ -62,7 +59,7 @@ public class DictionaryRuleTest extends LocalFileMetadataTestCase {
 
         for (File f : new File(LocalFileMetadataTestCase.LOCALMETA_TEST_DATA + "/cube_desc/").listFiles()) {
             CubeDesc desc = JsonUtil.readValue(new FileInputStream(f), CubeDesc.class);
-            desc.init(config, metadataManager.getAllTablesMap());
+            desc.init(config);
             ValidateContext vContext = new ValidateContext();
             rule.validate(desc, vContext);
             vContext.print(System.out);
@@ -99,7 +96,7 @@ public class DictionaryRuleTest extends LocalFileMetadataTestCase {
             desc.getDictionaries().add(dictDesc);
         }
 
-        desc.init(config, metadataManager.getAllTablesMap());
+        desc.init(config);
         ValidateContext vContext = new ValidateContext();
         rule.validate(desc, vContext);
         vContext.print(System.out);

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/FunctionRuleTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/FunctionRuleTest.java b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/FunctionRuleTest.java
index e041080..a3456c4 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/FunctionRuleTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/FunctionRuleTest.java
@@ -18,33 +18,30 @@
 
 package org.apache.kylin.cube.model.validation.rule;
 
+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 org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.validation.ValidateContext;
-import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 public class FunctionRuleTest extends LocalFileMetadataTestCase {
     private static KylinConfig config;
-    private static MetadataManager metadataManager;
 
     @Before
     public void setUp() throws Exception {
         this.createTestMetadata();
         config = KylinConfig.getInstanceFromEnv();
-        metadataManager = MetadataManager.getInstance(config);
     }
 
     @After
@@ -58,7 +55,7 @@ public class FunctionRuleTest extends LocalFileMetadataTestCase {
 
         File f = new File(LocalFileMetadataTestCase.LOCALMETA_TEST_DATA + "/cube_desc/ssb.json");
         CubeDesc desc = JsonUtil.readValue(new FileInputStream(f), CubeDesc.class);
-        desc.init(config, metadataManager.getAllTablesMap());
+        desc.init(config);
         ValidateContext vContext = new ValidateContext();
         rule.validate(desc, vContext);
         vContext.print(System.out);
@@ -75,7 +72,7 @@ public class FunctionRuleTest extends LocalFileMetadataTestCase {
         MeasureDesc measureDescDuplicated = desc.getMeasures().get(1);
         desc.getMeasures().add(measureDescDuplicated);
 
-        desc.init(config, metadataManager.getAllTablesMap());
+        desc.init(config);
         ValidateContext vContext = new ValidateContext();
         rule.validate(desc, vContext);
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
index 36fff5c..a2c5756 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.kylin.measure.MeasureType;
@@ -75,7 +74,7 @@ public class FunctionDesc {
     private MeasureType<?> measureType;
     private boolean isDimensionAsMetric = false;
 
-    public void init(TableDesc factTable, List<TableDesc> lookupTables) {
+    public void init(DataModelDesc model) {
         expression = expression.toUpperCase();
         returnDataType = DataType.getType(returnType);
 
@@ -86,8 +85,7 @@ public class FunctionDesc {
         ArrayList<TblColRef> colRefs = Lists.newArrayList();
         for (ParameterDesc p = parameter; p != null; p = p.getNextParameter()) {
             if (p.isColumnType()) {
-                ColumnDesc sourceColumn = findColumn(factTable, lookupTables, p.getValue());
-                TblColRef colRef = new TblColRef(sourceColumn);
+                TblColRef colRef = model.findColumn(p.getValue());
                 colRefs.add(colRef);
             }
         }
@@ -95,21 +93,6 @@ public class FunctionDesc {
         parameter.setColRefs(colRefs);
     }
 
-    private ColumnDesc findColumn(TableDesc factTable, List<TableDesc> lookups, String columnName) {
-        ColumnDesc ret = factTable.findColumnByName(columnName);
-        if (ret != null) {
-            return ret;
-        }
-
-        for (TableDesc lookup : lookups) {
-            ret = lookup.findColumnByName(columnName);
-            if (ret != null) {
-                return ret;
-            }
-        }
-        throw new IllegalStateException("Column is not found in any table from the model: " + columnName);
-    }
-
     private void reInitMeasureType() {
         if (isDimensionAsMetric && isCountDistinct()) {
             // create DimCountDis

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
index a5a85fa..b883df2 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
@@ -170,7 +170,7 @@ public class ExtendCubeToHybridCLI {
         CubeDesc newCubeDesc = CubeDesc.getCopyOf(cubeDesc);
         newCubeDesc.setName(newCubeDescName);
         newCubeDesc.updateRandomUuid();
-        newCubeDesc.init(kylinConfig, metadataManager.getAllTablesMap());
+        newCubeDesc.init(kylinConfig);
         newCubeDesc.setPartitionDateEnd(partitionDate);
         newCubeDesc.calculateSignature();
         cubeDescManager.createCubeDesc(newCubeDesc);