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/11/20 12:44:43 UTC

[1/2] kylin git commit: KYLIN-1875 Major metadata refactor, still backward compatible [Forced Update!]

Repository: kylin
Updated Branches:
  refs/heads/KYLIN-1875 f3953ab30 -> cfa3d0d43 (forced update)


http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java b/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
index ac70716..21b60f1 100644
--- a/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
+++ b/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
@@ -200,10 +200,10 @@ public class OLAPTable extends AbstractQueryableTable implements TranslatableTab
         //2. All integer measures in non-cube realizations
         MetadataManager metadataManager = MetadataManager.getInstance(olapSchema.getConfig());
         for (IRealization realization : mgr.listAllRealizations(olapSchema.getProjectName())) {
-            if (realization.getType() == RealizationType.INVERTED_INDEX && sourceTable.getIdentity().equalsIgnoreCase(realization.getFactTable())) {
-                DataModelDesc dataModelDesc = realization.getDataModelDesc();
+            if (realization.getType() == RealizationType.INVERTED_INDEX && realization.getModel().isFactTable(sourceTable.getIdentity())) {
+                DataModelDesc dataModelDesc = realization.getModel();
                 for (String metricColumn : dataModelDesc.getMetrics()) {
-                    ColumnDesc columnDesc = metadataManager.getColumnDesc(dataModelDesc.getFactTable() + "." + metricColumn);
+                    ColumnDesc columnDesc = metadataManager.getColumnDesc(dataModelDesc.getRootFactTable() + "." + metricColumn);
                     if (columnDesc.getType().isIntegerFamily() && !columnDesc.getType().isBigInt())
                         updateColumns.add(columnDesc);
                 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java b/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
index a3ce5c5..19c9c6e 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
@@ -38,7 +38,7 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.cachesync.Broadcaster;
 import org.apache.kylin.metadata.model.DataModelDesc;
-import org.apache.kylin.metadata.model.LookupDesc;
+import org.apache.kylin.metadata.model.JoinTableDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.project.ProjectManager;
@@ -289,6 +289,7 @@ public class CacheServiceTest extends LocalFileMetadataTestCase {
         return tableDesc;
     }
 
+    @SuppressWarnings("deprecation")
     @Test
     public void testMetaCRUD() throws Exception {
         final MetadataManager metadataManager = MetadataManager.getInstance(configA);
@@ -320,14 +321,13 @@ public class CacheServiceTest extends LocalFileMetadataTestCase {
         waitForCounterAndClear(2);
         assertEquals(dataModelDesc.getName(), metadataManagerB.getDataModelDesc(dataModelName).getName());
 
-        final LookupDesc[] lookups = dataModelDesc.getLookups();
+        final JoinTableDesc[] lookups = dataModelDesc.getJoinTables();
         assertTrue(lookups.length > 0);
-        dataModelDesc.setLookups(lookups);
         metadataManager.updateDataModelDesc(dataModelDesc);
         //only one for data model update
         assertEquals(1, broadcaster.getCounterAndClear());
         waitForCounterAndClear(1);
-        assertEquals(dataModelDesc.getLookups().length, metadataManagerB.getDataModelDesc(dataModelName).getLookups().length);
+        assertEquals(dataModelDesc.getJoinTables().length, metadataManagerB.getDataModelDesc(dataModelName).getJoinTables().length);
 
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
index 9e9dc25..3b66287 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
@@ -50,7 +50,7 @@ import org.apache.kylin.job.execution.ExecuteResult;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
 import org.apache.kylin.metadata.model.ISegment;
-import org.apache.kylin.metadata.model.LookupDesc;
+import org.apache.kylin.metadata.model.JoinTableDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -164,7 +164,7 @@ public class HiveMRInput implements IMRInput {
             MetadataManager metadataManager = MetadataManager.getInstance(kylinConfig);
             final Set<TableDesc> lookupViewsTables = Sets.newHashSet();
 
-            for (LookupDesc lookupDesc : flatDesc.getDataModel().getLookups()) {
+            for (JoinTableDesc lookupDesc : flatDesc.getDataModel().getJoinTables()) {
                 TableDesc tableDesc = metadataManager.getTableDesc(lookupDesc.getTable());
                 if (TableDesc.TABLE_TYPE_VIRTUAL_VIEW.equalsIgnoreCase(tableDesc.getTableType())) {
                     lookupViewsTables.add(tableDesc);

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java b/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java
index 87a8870..3172251 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java
@@ -29,7 +29,6 @@ import javax.annotation.Nullable;
 
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.ColumnDesc;
@@ -104,15 +103,7 @@ public class SchemaChecker {
                 if (cube == null || cube.allowBrokenDescriptor()) {
                     return false;
                 }
-                CubeDesc desc = cube.getDescriptor();
-
-                Set<String> usedTables = Sets.newHashSet();
-                usedTables.add(desc.getFactTableDesc().getIdentity());
-                for (TableDesc lookup : desc.getLookupTableDescs()) {
-                    usedTables.add(lookup.getIdentity());
-                }
-
-                return usedTables.contains(fullTableName);
+                return cube.getModel().containsTable(fullTableName);
             }
         });
 
@@ -210,13 +201,11 @@ public class SchemaChecker {
 
         List<String> issues = Lists.newArrayList();
         for (CubeInstance cube : findCubeByTable(fullTableName)) {
-            TableDesc factTable = cube.getFactTableDesc();
-            List<TableDesc> lookupTables = cube.getDescriptor().getLookupTableDescs();
-            String modelName = cube.getDataModelDesc().getName();
+            String modelName = cube.getModel().getName();
 
-            // if user reloads a fact table used by cube, then all used columns
-            // must match current schema
-            if (factTable.getIdentity().equals(fullTableName)) {
+            // if user reloads a fact table used by cube, then all used columns must match current schema
+            if (cube.getModel().isFactTable(fullTableName)) {
+                TableDesc factTable = cube.getModel().findFirstTable(fullTableName).getTableDesc();
                 List<String> violateColumns = checkAllColumnsInCube(cube, factTable, currentFieldsMap);
                 if (!violateColumns.isEmpty()) {
                     issues.add(format("Column %s used in cube[%s] and model[%s], but changed in hive", violateColumns, cube.getName(), modelName));
@@ -225,8 +214,9 @@ public class SchemaChecker {
 
             // if user reloads a lookup table used by cube, only append column(s) are allowed, all existing columns
             // must be the same (except compatible type changes)
-            for (TableDesc lookupTable : lookupTables) {
-                if (lookupTable.getIdentity().equals(fullTableName) && !checkAllColumnsInTableDesc(lookupTable, currentFields)) {
+            if (cube.getModel().isLookupTable(fullTableName)) {
+                TableDesc lookupTable = cube.getModel().findFirstTable(fullTableName).getTableDesc();
+                if (!checkAllColumnsInTableDesc(lookupTable, currentFields)) {
                     issues.add(format("Table '%s' is used as Lookup Table in cube[%s] and model[%s], but changed in hive", lookupTable.getIdentity(), cube.getName(), modelName));
                 }
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
index 8695276..dde3584 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
@@ -56,11 +56,11 @@ import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.apache.kylin.source.kafka.hadoop.KafkaFlatTableJob;
 import org.apache.kylin.source.kafka.job.MergeOffsetStep;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class KafkaMRInput implements IMRInput {
 
@@ -178,13 +178,11 @@ public class KafkaMRInput implements IMRInput {
         @Override
         public IMRTableInputFormat getFlatTableInputFormat() {
             KafkaConfigManager kafkaConfigManager = KafkaConfigManager.getInstance(KylinConfig.getInstanceFromEnv());
-            KafkaConfig kafkaConfig = kafkaConfigManager.getKafkaConfig(seg.getRealization().getFactTable());
+            KafkaConfig kafkaConfig = kafkaConfigManager.getKafkaConfig(seg.getCubeInstance().getRootFactTable());
             List<TblColRef> columns = new CubeJoinedFlatTableDesc(seg).getAllColumns();
 
             return new KafkaTableInputFormat(seg, columns, kafkaConfig, conf);
-
         }
-
     }
 
     class KafkaMRBatchMergeInputSide implements IMRBatchMergeInputSide {

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaSource.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaSource.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaSource.java
index b0c8e7f..e469f77 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaSource.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaSource.java
@@ -89,7 +89,7 @@ public class KafkaSource implements ISource {
             }
         }
 
-        final KafkaConfig kafakaConfig = KafkaConfigManager.getInstance(KylinConfig.getInstanceFromEnv()).getKafkaConfig(cube.getFactTable());
+        final KafkaConfig kafakaConfig = KafkaConfigManager.getInstance(KylinConfig.getInstanceFromEnv()).getKafkaConfig(cube.getRootFactTable());
         final String brokers = KafkaClient.getKafkaBrokers(kafakaConfig);
         final String topic = kafakaConfig.getTopic();
         try (final KafkaConsumer consumer = KafkaClient.getKafkaConsumer(brokers, cube.getName(), null)) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableJob.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableJob.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableJob.java
index 5fe6e00..3033bfd 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableJob.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableJob.java
@@ -90,7 +90,7 @@ public class KafkaFlatTableJob extends AbstractHadoopJob {
             setJobClasspath(job, cube.getConfig());
 
             KafkaConfigManager kafkaConfigManager = KafkaConfigManager.getInstance(KylinConfig.getInstanceFromEnv());
-            KafkaConfig kafkaConfig = kafkaConfigManager.getKafkaConfig(cube.getFactTable());
+            KafkaConfig kafkaConfig = kafkaConfigManager.getKafkaConfig(cube.getRootFactTable());
             String brokers = KafkaClient.getKafkaBrokers(kafkaConfig);
             String topic = kafkaConfig.getTopic();
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaClient.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaClient.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaClient.java
index 446c076..2a7b0e8 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaClient.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaClient.java
@@ -118,7 +118,7 @@ public class KafkaClient {
     }
 
     public static Map<Integer, Long> getCurrentOffsets(final CubeInstance cubeInstance) {
-        final KafkaConfig kafakaConfig = KafkaConfigManager.getInstance(KylinConfig.getInstanceFromEnv()).getKafkaConfig(cubeInstance.getFactTable());
+        final KafkaConfig kafakaConfig = KafkaConfigManager.getInstance(KylinConfig.getInstanceFromEnv()).getKafkaConfig(cubeInstance.getRootFactTable());
 
         final String brokers = KafkaClient.getKafkaBrokers(kafakaConfig);
         final String topic = kafakaConfig.getTopic();
@@ -136,7 +136,7 @@ public class KafkaClient {
 
 
     public static Map<Integer, Long> getEarliestOffsets(final CubeInstance cubeInstance) {
-        final KafkaConfig kafakaConfig = KafkaConfigManager.getInstance(KylinConfig.getInstanceFromEnv()).getKafkaConfig(cubeInstance.getFactTable());
+        final KafkaConfig kafakaConfig = KafkaConfigManager.getInstance(KylinConfig.getInstanceFromEnv()).getKafkaConfig(cubeInstance.getRootFactTable());
 
         final String brokers = KafkaClient.getKafkaBrokers(kafakaConfig);
         final String topic = kafakaConfig.getTopic();

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
index f4dfd2b..43b65cb 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
@@ -75,7 +75,7 @@ public class HBaseStorage implements IStorage {
     }
 
     private static TblColRef getPartitionCol(IRealization realization) {
-        String modelName = realization.getDataModelDesc().getName();
+        String modelName = realization.getModel().getName();
         DataModelDesc dataModelDesc = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getDataModelDesc(modelName);
         PartitionDesc partitionDesc = dataModelDesc.getPartitionDesc();
         Preconditions.checkArgument(partitionDesc != null, "PartitionDesc for " + realization + " is null!");

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
index dcf1690..2e682b1 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
@@ -60,6 +60,7 @@ import org.apache.kylin.metadata.cachesync.Broadcaster;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
@@ -250,8 +251,8 @@ public class CubeMigrationCLI {
         metaResource.add(cubeDesc.getResourcePath());
         metaResource.add(DataModelDesc.concatResourcePath(cubeDesc.getModelName()));
 
-        for (String table : cubeDesc.getModel().getAllTables()) {
-            metaResource.add(TableDesc.concatResourcePath(table.toUpperCase()));
+        for (TableRef table : cubeDesc.getModel().getAllTables()) {
+            metaResource.add(TableDesc.concatResourcePath(table.getTableIdentity()));
         }
 
         for (CubeSegment segment : cube.getSegments()) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/tool/src/main/java/org/apache/kylin/tool/CubeMetaExtractor.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/CubeMetaExtractor.java b/tool/src/main/java/org/apache/kylin/tool/CubeMetaExtractor.java
index 9cb135a..0067e24 100644
--- a/tool/src/main/java/org/apache/kylin/tool/CubeMetaExtractor.java
+++ b/tool/src/main/java/org/apache/kylin/tool/CubeMetaExtractor.java
@@ -35,8 +35,6 @@ import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.metadata.streaming.StreamingConfig;
-import org.apache.kylin.metadata.streaming.StreamingManager;
 import org.apache.kylin.job.dao.ExecutableDao;
 import org.apache.kylin.job.dao.ExecutablePO;
 import org.apache.kylin.job.exception.PersistentException;
@@ -45,12 +43,15 @@ import org.apache.kylin.metadata.badquery.BadQueryHistoryManager;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.project.ProjectManager;
 import org.apache.kylin.metadata.project.RealizationEntry;
 import org.apache.kylin.metadata.realization.IRealization;
 import org.apache.kylin.metadata.realization.RealizationRegistry;
 import org.apache.kylin.metadata.realization.RealizationType;
+import org.apache.kylin.metadata.streaming.StreamingConfig;
+import org.apache.kylin.metadata.streaming.StreamingManager;
 import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.apache.kylin.storage.hybrid.HybridInstance;
 import org.apache.kylin.storage.hybrid.HybridManager;
@@ -231,7 +232,7 @@ public class CubeMetaExtractor extends AbstractInfoExtractor {
     private void dealWithStreaming(CubeInstance cube) {
         streamingManager = StreamingManager.getInstance(kylinConfig);
         for (StreamingConfig streamingConfig : streamingManager.listAllStreaming()) {
-            if (streamingConfig.getName() != null && streamingConfig.getName().equalsIgnoreCase(cube.getFactTable())) {
+            if (streamingConfig.getName() != null && streamingConfig.getName().equalsIgnoreCase(cube.getRootFactTable())) {
                 addRequired(StreamingConfig.concatResourcePath(streamingConfig.getName()));
                 addRequired(KafkaConfig.concatResourcePath(streamingConfig.getName()));
             }
@@ -254,7 +255,8 @@ public class CubeMetaExtractor extends AbstractInfoExtractor {
 
             dealWithStreaming(cube);
 
-            for (String tableName : modelDesc.getAllTables()) {
+            for (TableRef table : modelDesc.getAllTables()) {
+                String tableName = table.getTableIdentity();
                 addRequired(TableDesc.concatResourcePath(tableName));
                 addOptional(TableDesc.concatExdResourcePath(tableName));
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/tool/src/main/java/org/apache/kylin/tool/CubeMetaIngester.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/CubeMetaIngester.java b/tool/src/main/java/org/apache/kylin/tool/CubeMetaIngester.java
index 1ce88d3..40cbf32 100644
--- a/tool/src/main/java/org/apache/kylin/tool/CubeMetaIngester.java
+++ b/tool/src/main/java/org/apache/kylin/tool/CubeMetaIngester.java
@@ -152,7 +152,7 @@ public class CubeMetaIngester extends AbstractApplication {
 
         for (CubeInstance cube : srcCubeManager.listAllCubes()) {
             logger.info("add " + cube + " to " + targetProjectName);
-            projectManager.updateModelToProject(cube.getDataModelDesc().getName(), targetProjectName);
+            projectManager.updateModelToProject(cube.getModel().getName(), targetProjectName);
             projectManager.moveRealizationToProject(RealizationType.CUBE, cube.getName(), targetProjectName, null);
         }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java b/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
index 46f8d75..40306c9 100644
--- a/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
+++ b/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
@@ -60,6 +60,7 @@ import org.apache.kylin.metadata.cachesync.Broadcaster;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
@@ -248,8 +249,8 @@ public class CubeMigrationCLI {
         metaResource.add(cubeDesc.getResourcePath());
         metaResource.add(DataModelDesc.concatResourcePath(cubeDesc.getModelName()));
 
-        for (String table : cubeDesc.getModel().getAllTables()) {
-            metaResource.add(TableDesc.concatResourcePath(table.toUpperCase()));
+        for (TableRef tableRef : cubeDesc.getModel().getAllTables()) {
+            metaResource.add(TableDesc.concatResourcePath(tableRef.getTableIdentity()));
         }
 
         for (CubeSegment segment : cube.getSegments()) {


[2/2] kylin git commit: KYLIN-1875 Major metadata refactor, still backward compatible

Posted by li...@apache.org.
KYLIN-1875 Major metadata refactor, still backward compatible


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

Branch: refs/heads/KYLIN-1875
Commit: cfa3d0d431c1e396356b5c9d79092a0203664769
Parents: 1d4f57a
Author: Yang Li <li...@apache.org>
Authored: Sun Nov 20 18:07:27 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sun Nov 20 20:43:50 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/job/DeployUtil.java   |   6 +-
 .../kylin/job/dataGen/FactTableGenerator.java   |   6 +-
 .../kylin/cube/CubeCapabilityChecker.java       |  10 +-
 .../org/apache/kylin/cube/CubeInstance.java     |  24 +-
 .../java/org/apache/kylin/cube/JoinChecker.java |  66 ++--
 .../kylin/cube/cli/DictionaryGeneratorCLI.java  |  10 +-
 .../org/apache/kylin/cube/model/CubeDesc.java   |  17 +-
 .../apache/kylin/cube/model/DimensionDesc.java  |   8 +-
 .../kylin/cube/model/v1_4_0/CubeDesc.java       |   4 +-
 .../kylin/cube/model/v1_4_0/DimensionDesc.java  |   8 +-
 .../model/validation/rule/FunctionRule.java     |   2 +-
 .../validation/rule/StreamingCubeRule.java      |  11 +-
 .../org/apache/kylin/cube/CubeSegmentsTest.java |   4 +-
 .../apache/kylin/dict/DictionaryManager.java    |  34 +-
 .../kylin/dict/DictionaryManagerTest.java       |  67 ++++
 .../org/apache/kylin/job/JoinedFlatTable.java   |  14 +-
 .../kylin/metadata/model/DataModelDesc.java     | 325 ++++++++++---------
 .../apache/kylin/metadata/model/JoinDesc.java   |  11 +
 .../kylin/metadata/model/JoinTableDesc.java     |  77 +++++
 .../apache/kylin/metadata/model/LookupDesc.java |  67 ----
 .../kylin/metadata/model/PartitionDesc.java     |  17 +-
 .../kylin/metadata/project/ProjectL2Cache.java  |  13 +-
 .../kylin/metadata/project/ProjectManager.java  |   4 -
 .../metadata/realization/IRealization.java      |   4 +-
 .../kylin/storage/hybrid/HybridInstance.java    |   9 +-
 .../engine/mr/common/AbstractHadoopJob.java     |  19 +-
 .../engine/mr/steps/MergeCuboidMapper.java      |  17 +-
 .../engine/mr/steps/MergeDictionaryStep.java    |   5 +-
 .../kylin/provision/BuildCubeWithStream.java    |   2 +-
 .../kylin/storage/hbase/ITStorageTest.java      |   2 +-
 .../org/apache/kylin/query/relnode/OLAPRel.java |  10 +-
 .../kylin/query/relnode/OLAPTableScan.java      |   2 +-
 .../kylin/query/routing/ModelChooser.java       |  10 +-
 .../apache/kylin/query/schema/OLAPTable.java    |   6 +-
 .../kylin/rest/service/CacheServiceTest.java    |   8 +-
 .../apache/kylin/source/hive/HiveMRInput.java   |   4 +-
 .../apache/kylin/source/hive/SchemaChecker.java |  26 +-
 .../apache/kylin/source/kafka/KafkaMRInput.java |   8 +-
 .../apache/kylin/source/kafka/KafkaSource.java  |   2 +-
 .../source/kafka/hadoop/KafkaFlatTableJob.java  |   2 +-
 .../kylin/source/kafka/util/KafkaClient.java    |   4 +-
 .../kylin/storage/hbase/HBaseStorage.java       |   2 +-
 .../storage/hbase/util/CubeMigrationCLI.java    |   5 +-
 .../apache/kylin/tool/CubeMetaExtractor.java    |  10 +-
 .../org/apache/kylin/tool/CubeMetaIngester.java |   2 +-
 .../org/apache/kylin/tool/CubeMigrationCLI.java |   5 +-
 46 files changed, 532 insertions(+), 437 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
index 54feb24..79a3b3b 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -150,13 +150,13 @@ public class DeployUtil {
 
     public static void prepareTestDataForStreamingCube(long startTime, long endTime, int numberOfRecords, String cubeName, StreamDataLoader streamDataLoader) throws IOException {
         CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName);
-        List<String> data = StreamingTableDataGenerator.generate(numberOfRecords, startTime, endTime, cubeInstance.getFactTable());
+        List<String> data = StreamingTableDataGenerator.generate(numberOfRecords, startTime, endTime, cubeInstance.getRootFactTable());
         //load into kafka
         streamDataLoader.loadIntoKafka(data);
         logger.info("Write {} messages into {}", data.size(), streamDataLoader.toString());
 
         //csv data for H2 use
-        TableRef factTable = cubeInstance.getDataModelDesc().getFactTableRef();
+        TableRef factTable = cubeInstance.getModel().getRootFactTable();
         List<TblColRef> tableColumns = Lists.newArrayList(factTable.getColumns());
         TimedJsonStreamParser timedJsonStreamParser = new TimedJsonStreamParser(tableColumns, null);
         StringBuilder sb = new StringBuilder();
@@ -165,7 +165,7 @@ public class DeployUtil {
             sb.append(StringUtils.join(rowColumns, ","));
             sb.append(System.getProperty("line.separator"));
         }
-        appendFactTableData(sb.toString(), cubeInstance.getFactTable());
+        appendFactTableData(sb.toString(), cubeInstance.getRootFactTable());
     }
 
     public static void overrideFactTableData(String factTableContent, String factTableName) throws IOException {

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java b/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
index 84a642d..8068fd1 100644
--- a/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
+++ b/assembly/src/test/java/org/apache/kylin/job/dataGen/FactTableGenerator.java
@@ -104,7 +104,7 @@ public class FactTableGenerator {
         KylinConfig config = KylinConfig.getInstanceFromEnv();
         cube = CubeManager.getInstance(config).getCube(cubeName);
         desc = cube.getDescriptor();
-        factTableName = desc.getFactTable();
+        factTableName = cube.getRootFactTable();
         store = ResourceStore.getStore(config);
     }
 
@@ -559,7 +559,7 @@ public class FactTableGenerator {
 
         long currentRowTime = -1;
 
-        for (TblColRef col : cube.getDataModelDesc().getFactTableRef().getColumns()) {
+        for (TblColRef col : cube.getModel().getRootFactTable().getColumns()) {
 
             String colName = col.getName();
 
@@ -579,7 +579,7 @@ public class FactTableGenerator {
                 defaultColumns.add(colName);
             }
 
-            if (col.equals(cube.getDataModelDesc().getPartitionDesc().getPartitionDateColumnRef())) {
+            if (col.equals(cube.getModel().getPartitionDesc().getPartitionDateColumnRef())) {
                 currentRowTime = format.parse(columnValues.get(columnValues.size() - 1)).getTime();
             }
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index a3c89e5..38faed9 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -66,13 +66,11 @@ public class CubeCapabilityChecker {
         Collection<FunctionDesc> unmatchedAggregations = unmatchedAggregations(aggrFunctions, cube);
 
         // try custom measure types
-        // in RAW query, unmatchedDimensions and unmatchedAggregations will null, so can't chose RAW cube well!
-        //        if (!unmatchedDimensions.isEmpty() || !unmatchedAggregations.isEmpty()) {
         tryCustomMeasureTypes(unmatchedDimensions, unmatchedAggregations, digest, cube, result);
-        //        }
 
         //more tricks
-        if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+        String rootFactTable = cube.getRootFactTable();
+        if (rootFactTable.equals(digest.factTable)) {
             //for query-on-facttable
             //1. dimension as measure
 
@@ -83,7 +81,7 @@ public class CubeCapabilityChecker {
             //for non query-on-facttable 
             if (cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
 
-                Set<TblColRef> dimCols = Sets.newHashSet(cube.getDataModelDesc().findFirstTable(digest.factTable).getColumns());
+                Set<TblColRef> dimCols = Sets.newHashSet(cube.getModel().findFirstTable(digest.factTable).getColumns());
 
                 //1. all aggregations on lookup table can be done. For distinct count, mark them all DimensionAsMeasures
                 // so that the measure has a chance to be upgraded to DimCountDistinctMeasureType in org.apache.kylin.metadata.model.FunctionDesc#reInitMeasureType
@@ -122,7 +120,7 @@ public class CubeCapabilityChecker {
             return result;
         }
 
-        if (digest.isRawQuery && cube.getFactTable().equals(digest.factTable)) {
+        if (digest.isRawQuery && rootFactTable.equals(digest.factTable)) {
             result.influences.add(new CapabilityInfluence() {
                 @Override
                 public double suggestCostMultiplier() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
index cb98991..061ab23 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
@@ -31,10 +31,9 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.IBuildable;
-import org.apache.kylin.metadata.model.LookupDesc;
+import org.apache.kylin.metadata.model.JoinTableDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.CapabilityResult;
 import org.apache.kylin.metadata.realization.CapabilityResult.CapabilityInfluence;
@@ -138,7 +137,7 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
     }
 
     @Override
-    public DataModelDesc getDataModelDesc() {
+    public DataModelDesc getModel() {
         CubeDesc cubeDesc = this.getDescriptor();
         if (cubeDesc != null) {
             return cubeDesc.getModel();
@@ -214,6 +213,10 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
         this.config = config;
     }
 
+    public String getRootFactTable() {
+        return getModel().getRootFactTable().getTableIdentity();
+    }
+
     @Override
     public String getName() {
         return name;
@@ -225,15 +228,6 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
     }
 
     @Override
-    public String getFactTable() {
-        return getDescriptor().getFactTable();
-    }
-
-    public TableDesc getFactTableDesc() {
-        return getDescriptor().getFactTableDesc();
-    }
-
-    @Override
     public List<MeasureDesc> getMeasures() {
         return getDescriptor().getMeasures();
     }
@@ -374,9 +368,9 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
         //the number of dimensions is not as accurate as number of row key cols
         calculatedCost += getRowKeyColumnCount() * COST_WEIGHT_DIMENSION + getMeasures().size() * COST_WEIGHT_MEASURE;
 
-        for (LookupDesc lookupDesc : this.getDescriptor().getModel().getLookups()) {
+        for (JoinTableDesc joinTable : this.getModel().getJoinTables()) {
             // more tables, more cost
-            if (lookupDesc.getJoin().isInnerJoin()) {
+            if (joinTable.getJoin().isInnerJoin()) {
                 // inner join cost is bigger than left join, as it will filter some records
                 calculatedCost += COST_WEIGHT_INNER_JOIN;
             }
@@ -457,7 +451,7 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
 
     @Override
     public int getSourceType() {
-        return getFactTableDesc().getSourceType();
+        return getModel().getRootFactTable().getTableDesc().getSourceType();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-cube/src/main/java/org/apache/kylin/cube/JoinChecker.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/JoinChecker.java b/core-cube/src/main/java/org/apache/kylin/cube/JoinChecker.java
index edd5be9..9068449 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/JoinChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/JoinChecker.java
@@ -19,50 +19,42 @@
 package org.apache.kylin.cube;
 
 import java.util.Collection;
-import java.util.List;
 
 import org.apache.kylin.metadata.model.JoinDesc;
-import org.apache.kylin.metadata.model.LookupDesc;
 import org.apache.kylin.metadata.realization.IRealization;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
 
 public class JoinChecker {
     
-    private static final Logger logger = LoggerFactory.getLogger(CubeCapabilityChecker.class);
-
+    // given ModelChooser has done the model join matching already, this method seems useless
     public static boolean isJoinMatch(Collection<JoinDesc> joins, IRealization realization) {
-
-        List<JoinDesc> realizationsJoins = Lists.newArrayList();
-        for (LookupDesc lookupDesc : realization.getDataModelDesc().getLookups()) {
-            realizationsJoins.add(lookupDesc.getJoin());
-        }
-
-        for (JoinDesc j : joins) {
-            // optiq engine can't decide which one is fk or pk
-            String pTable = j.getPrimaryKeyColumns()[0].getTable();
-            String factTable = realization.getFactTable();
-            if (factTable.equals(pTable)) {
-                j.swapPKFK();
-            }
-
-            // check primary key, all PK column should refer to same tale, the Fact Table of cube.
-            // Using first column's table name to check.
-            String fTable = j.getForeignKeyColumns()[0].getTable();
-            if (!factTable.equals(fTable)) {
-                logger.info("Fact Table" + factTable + " not matched in join: " + j + " on cube " + realization.getName());
-                return false;
-            }
-
-            // The hashcode() function of JoinDesc has been overwritten,
-            // which takes into consideration: pk,fk,jointype
-            if (!realizationsJoins.contains(j)) {
-                logger.info("Query joins don't macth on cube " + realization.getName());
-                return false;
-            }
-        }
+//        List<JoinDesc> realizationsJoins = Lists.newArrayList();
+//        for (JoinTableDesc joinTable : realization.getModel().getJoinTables()) {
+//            realizationsJoins.add(joinTable.getJoin());
+//        }
+//
+//        for (JoinDesc j : joins) {
+//            // optiq engine can't decide which one is fk or pk
+//            String pTable = j.getPrimaryKeyColumns()[0].getTable();
+//            String factTable = realization.getModel().getRootFactTable().getTableIdentity();
+//            if (factTable.equals(pTable)) {
+//                j.swapPKFK();
+//            }
+//
+//            // check primary key, all PK column should refer to same tale, the Fact Table of cube.
+//            // Using first column's table name to check.
+//            String fTable = j.getForeignKeyColumns()[0].getTable();
+//            if (!factTable.equals(fTable)) {
+//                logger.info("Fact Table" + factTable + " not matched in join: " + j + " on cube " + realization.getName());
+//                return false;
+//            }
+//
+//            // The hashcode() function of JoinDesc has been overwritten,
+//            // which takes into consideration: pk,fk,jointype
+//            if (!realizationsJoins.contains(j)) {
+//                logger.info("Query joins don't macth on cube " + realization.getName());
+//                return false;
+//            }
+//        }
         return true;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java b/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java
index fcd68ba..89e2e9b 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java
@@ -27,6 +27,7 @@ import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.model.DimensionDesc;
 import org.apache.kylin.dict.DistinctColumnValuesProvider;
+import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -56,13 +57,10 @@ public class DictionaryGeneratorCLI {
         // snapshot
         Set<String> toSnapshot = Sets.newHashSet();
         for (DimensionDesc dim : cubeSeg.getCubeDesc().getDimensions()) {
-            if (dim.getTableRef() == null)
-                continue;
-            
-            String lookupTable = dim.getTableRef().getTableIdentity();
-            toSnapshot.add(lookupTable);
+            TableRef table = dim.getTableRef();
+            if (cubeSeg.getModel().isLookupTable(table))
+                toSnapshot.add(table.getTableIdentity());
         }
-        toSnapshot.remove(cubeSeg.getCubeDesc().getFactTable());
         
         for (String tableIdentity : toSnapshot) {
             logger.info("Building snapshot of " + tableIdentity);

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/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 ce6389f..b62c1ec 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
@@ -63,7 +63,6 @@ import org.apache.kylin.metadata.model.IEngineAware;
 import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.JoinDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -341,18 +340,6 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
         this.description = description;
     }
 
-    public String getFactTable() {
-        return model.getFactTable();
-    }
-
-    public TableDesc getFactTableDesc() {
-        return model.getFactTableRef().getTableDesc();
-    }
-
-    public List<TableDesc> getLookupTableDescs() {
-        return model.getLookupTableDescs();
-    }
-
     public String[] getNullStrings() {
         return nullStrings;
     }
@@ -441,7 +428,7 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
         if (!name.equals(cubeDesc.name))
             return false;
 
-        if (!getFactTable().equals(cubeDesc.getFactTable()))
+        if (!modelName.equals(cubeDesc.modelName))
             return false;
 
         return true;
@@ -461,7 +448,7 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
     public int hashCode() {
         int result = 0;
         result = 31 * result + name.hashCode();
-        result = 31 * result + getFactTable().hashCode();
+        result = 31 * result + model.getRootFactTable().hashCode();
         return result;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/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 ae90a18..1106103 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
@@ -24,7 +24,7 @@ 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.JoinTableDesc;
 import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
 
@@ -66,9 +66,9 @@ public class DimensionDesc {
             throw new IllegalStateException("Can't find table " + table + " for dimension " + name);
 
         join = null;
-        for (LookupDesc lookup : model.getLookups()) {
-            if (lookup.getTableRef().equals(this.tableRef)) {
-                join = lookup.getJoin();
+        for (JoinTableDesc joinTable : model.getJoinTables()) {
+            if (joinTable.getTableRef().equals(this.tableRef)) {
+                join = joinTable.getJoin();
                 break;
             }
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/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 fb2998a..bd73bc8 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
@@ -324,11 +324,11 @@ public class CubeDesc extends RootPersistentEntity {
     }
 
     public String getFactTable() {
-        return model.getFactTable();
+        return model.getRootFactTable().getTableIdentity();
     }
 
     public TableDesc getFactTableDesc() {
-        return model.getFactTableRef().getTableDesc();
+        return model.getRootFactTable().getTableDesc();
     }
 
     public List<TableDesc> getLookupTableDescs() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/DimensionDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/DimensionDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/DimensionDesc.java
index f97f939..3903b2b 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/DimensionDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/DimensionDesc.java
@@ -25,7 +25,7 @@ import java.util.Map;
 
 import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.metadata.model.JoinDesc;
-import org.apache.kylin.metadata.model.LookupDesc;
+import org.apache.kylin.metadata.model.JoinTableDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 
@@ -73,9 +73,9 @@ public class DimensionDesc {
             throw new IllegalStateException("Can't find table " + table + " for dimension " + name);
 
         join = null;
-        for (LookupDesc lookup : cubeDesc.getModel().getLookups()) {
-            if (lookup.getTable().equalsIgnoreCase(this.getTable())) {
-                join = lookup.getJoin();
+        for (JoinTableDesc joinTable : cubeDesc.getModel().getJoinTables()) {
+            if (joinTable.getTable().equalsIgnoreCase(this.getTable())) {
+                join = joinTable.getJoin();
                 break;
             }
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java
index bcc9010..ee93f72 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/FunctionRule.java
@@ -153,7 +153,7 @@ public class FunctionRule implements IValidatorRule<CubeDesc> {
      * @param value
      */
     private void validateColumnParameter(ValidateContext context, CubeDesc cube, String value) {
-        String factTable = cube.getFactTable();
+        String factTable = cube.getModel().getRootFactTable().getTableIdentity();
         if (StringUtils.isEmpty(factTable)) {
             context.addResult(ResultLevel.ERROR, "Fact table can not be null.");
             return;

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/StreamingCubeRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/StreamingCubeRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/StreamingCubeRule.java
index 1d6b7cc..db331d5 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/StreamingCubeRule.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/StreamingCubeRule.java
@@ -23,6 +23,7 @@ import org.apache.kylin.cube.model.DimensionDesc;
 import org.apache.kylin.cube.model.validation.IValidatorRule;
 import org.apache.kylin.cube.model.validation.ResultLevel;
 import org.apache.kylin.cube.model.validation.ValidateContext;
+import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.ISourceAware;
 
 import org.apache.kylin.metadata.model.TblColRef;
@@ -41,21 +42,23 @@ public class StreamingCubeRule implements IValidatorRule<CubeDesc> {
      */
     @Override
     public void validate(CubeDesc cube, ValidateContext context) {
-        if (cube.getFactTableDesc().getSourceType() != ISourceAware.ID_STREAMING) {
+        DataModelDesc model = cube.getModel();
+        
+        if (model.getRootFactTable().getTableDesc().getSourceType() != ISourceAware.ID_STREAMING) {
             return;
         }
 
-        if (cube.getLookupTableDescs() != null && cube.getLookupTableDescs().size() > 0) {
+        if (model.getLookupTables().size() > 0) {
             context.addResult(ResultLevel.ERROR, "Streaming Cube doesn't support star-schema so far; only one fact table is allowed.");
             return;
         }
 
-        if (cube.getModel().getPartitionDesc() == null || cube.getModel().getPartitionDesc().getPartitionDateColumn() == null) {
+        if (model.getPartitionDesc() == null || model.getPartitionDesc().getPartitionDateColumn() == null) {
             context.addResult(ResultLevel.ERROR, "Must define a partition column.");
             return;
         }
 
-        final TblColRef partitionCol = cube.getModel().getPartitionDesc().getPartitionDateColumnRef();
+        final TblColRef partitionCol = model.getPartitionDesc().getPartitionDateColumnRef();
         boolean found = false;
         for (DimensionDesc dimensionDesc : cube.getDimensions()) {
             for (TblColRef dimCol : dimensionDesc.getColumnRefs()) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-cube/src/test/java/org/apache/kylin/cube/CubeSegmentsTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/CubeSegmentsTest.java b/core-cube/src/test/java/org/apache/kylin/cube/CubeSegmentsTest.java
index a5bd821..de91dd2 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/CubeSegmentsTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/CubeSegmentsTest.java
@@ -48,7 +48,7 @@ public class CubeSegmentsTest extends LocalFileMetadataTestCase {
         CubeInstance cube = mgr.getCube("test_kylin_cube_without_slr_empty");
 
         // override partition desc
-        cube.getDataModelDesc().setPartitionDesc(new PartitionDesc());
+        cube.getModel().setPartitionDesc(new PartitionDesc());
 
         // first append, creates a new & single segment
         CubeSegment seg = mgr.appendSegment(cube);
@@ -73,7 +73,7 @@ public class CubeSegmentsTest extends LocalFileMetadataTestCase {
         CubeInstance cube = mgr.getCube("test_kylin_cube_without_slr_ready");
 
         // override partition desc
-        cube.getDataModelDesc().setPartitionDesc(new PartitionDesc());
+        cube.getModel().setPartitionDesc(new PartitionDesc());
 
         // assert one ready segment
         assertEquals(1, cube.getSegments().size());

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
index 2dd5085..db165ba 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.util.ClassUtil;
@@ -36,7 +37,9 @@ import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.JoinDesc;
 import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.source.ReadableTable;
 import org.apache.kylin.source.ReadableTable.TableSignature;
@@ -333,17 +336,38 @@ public class DictionaryManager {
     /**
      * Decide a dictionary's source data, leverage PK-FK relationship.
      */
-    public TblColRef decideSourceData(DataModelDesc model, TblColRef col) throws IOException {
+    public TblColRef decideSourceData(DataModelDesc model, TblColRef col) {
         // Note FK on fact table is supported by scan the related PK on lookup table
         // FK on fact table and join type is inner, use PK from lookup instead
-        if (model.isFactTable(col.getTable())) {
-            TblColRef pkCol = model.findPKByFK(col, "inner");
-            if (pkCol != null)
-                col = pkCol; // scan the counterparty PK on lookup table instead
+        if (model.isFactTable(col.getTable()) == false)
+            return col;
+        
+        // find a lookup table that the col joins as FK
+        for (TableRef lookup : model.getLookupTables()) {
+            JoinDesc lookupJoin = model.getPKSideJoinMap().get(lookup);
+            int find = ArrayUtils.indexOf(lookupJoin.getForeignKeyColumns(), col);
+            if (find < 0)
+                continue;
+        
+            // make sure the joins are all inner up to the root
+            if (isAllInnerJoinsToRoot(model, lookupJoin))
+                return lookupJoin.getPrimaryKeyColumns()[find];
         }
+        
         return col;
     }
 
+    private boolean isAllInnerJoinsToRoot(DataModelDesc model, JoinDesc join) {
+        while (join != null) {
+            if (join.isInnerJoin() == false)
+                return false;
+            
+            TableRef table = join.getForeignKeyColumns()[0].getTableRef();
+            join = model.getPKSideJoinMap().get(table);
+        }
+        return true;
+    }
+
     private String checkDupByInfo(DictionaryInfo dictInfo) throws IOException {
         final ResourceStore store = MetadataManager.getInstance(config).getStore();
         final List<DictionaryInfo> allResources = store.getAllResources(dictInfo.getResourceDir(), DictionaryInfo.class, DictionaryInfoSerializer.INFO_SERIALIZER);

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-dictionary/src/test/java/org/apache/kylin/dict/DictionaryManagerTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/DictionaryManagerTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/DictionaryManagerTest.java
new file mode 100644
index 0000000..930d010
--- /dev/null
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/DictionaryManagerTest.java
@@ -0,0 +1,67 @@
+/*
+ * 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.dict;
+
+import static org.junit.Assert.*;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class DictionaryManagerTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setup() throws Exception {
+        createTestMetadata();
+    }
+
+    @After
+    public void tearDown() {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void testDecideSourceData() {
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+
+        DictionaryManager dictMgr = DictionaryManager.getInstance(config);
+        MetadataManager metaMgr = MetadataManager.getInstance(config);
+
+        {
+            DataModelDesc innerModel = metaMgr.getDataModelDesc("test_kylin_inner_join_model_desc");
+            TblColRef factDate = innerModel.findColumn("TEST_KYLIN_FACT.CAL_DT");
+            TblColRef lookupDate = innerModel.findColumn("TEST_CAL_DT.CAL_DT");
+            TblColRef formatName = innerModel.findColumn("lstg_format_name");
+            assertEquals(lookupDate, dictMgr.decideSourceData(innerModel, factDate));
+            assertEquals(lookupDate, dictMgr.decideSourceData(innerModel, lookupDate));
+            assertEquals(formatName, dictMgr.decideSourceData(innerModel, formatName));
+        }
+        
+        {
+            DataModelDesc outerModel = metaMgr.getDataModelDesc("test_kylin_left_join_model_desc");
+            TblColRef factDate = outerModel.findColumn("TEST_KYLIN_FACT.CAL_DT");
+            assertEquals(factDate, dictMgr.decideSourceData(outerModel, factDate));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java b/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
index b26f50d..9fa0961 100644
--- a/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
+++ b/core-job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
@@ -29,7 +29,7 @@ import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
 import org.apache.kylin.metadata.model.JoinDesc;
-import org.apache.kylin.metadata.model.LookupDesc;
+import org.apache.kylin.metadata.model.JoinTableDesc;
 import org.apache.kylin.metadata.model.PartitionDesc;
 import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -122,9 +122,9 @@ public class JoinedFlatTable {
 
     public static String generateCountDataStatement(IJoinedFlatTableDesc flatDesc, final String outputDir) {
         final StringBuilder sql = new StringBuilder();
-        final TableRef factTbl = flatDesc.getDataModel().getFactTableRef();
+        final TableRef rootTbl = flatDesc.getDataModel().getRootFactTable();
         sql.append("dfs -mkdir -p " + outputDir + ";\n");
-        sql.append("INSERT OVERWRITE DIRECTORY '" + outputDir + "' SELECT count(*) FROM " + factTbl.getTableIdentity() + " " + factTbl.getAlias() + "\n");
+        sql.append("INSERT OVERWRITE DIRECTORY '" + outputDir + "' SELECT count(*) FROM " + rootTbl.getTableIdentity() + " " + rootTbl.getAlias() + "\n");
         appendWhereStatement(flatDesc, sql);
         return sql.toString();
     }
@@ -132,11 +132,11 @@ public class JoinedFlatTable {
     private static void appendJoinStatement(IJoinedFlatTableDesc flatDesc, StringBuilder sql) {
         Set<TableRef> dimTableCache = new HashSet<>();
 
-        DataModelDesc dataModelDesc = flatDesc.getDataModel();
-        TableRef factTable = dataModelDesc.getFactTableRef();
-        sql.append("FROM " + factTable.getTableIdentity() + " as " + factTable.getAlias() + " \n");
+        DataModelDesc model = flatDesc.getDataModel();
+        TableRef rootTable = model.getRootFactTable();
+        sql.append("FROM " + rootTable.getTableIdentity() + " as " + rootTable.getAlias() + " \n");
 
-        for (LookupDesc lookupDesc : dataModelDesc.getLookups()) {
+        for (JoinTableDesc lookupDesc : model.getJoinTables()) {
             JoinDesc join = lookupDesc.getJoin();
             if (join != null && join.getType().equals("") == false) {
                 String joinType = join.getType().toUpperCase();

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
index 139fcbe..1fd4e85 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
@@ -20,12 +20,10 @@ package org.apache.kylin.metadata.model;
 
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
-import org.apache.commons.lang.ArrayUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
@@ -36,6 +34,7 @@ import org.slf4j.LoggerFactory;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -45,10 +44,15 @@ import com.google.common.collect.Sets;
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class DataModelDesc extends RootPersistentEntity {
     private static final Logger logger = LoggerFactory.getLogger(DataModelDesc.class);
+
+    public static enum TableKind {
+        FACT, LOOKUP
+    }
+
     public static enum RealizationCapacity {
         SMALL, MEDIUM, LARGE
     }
-    
+
     private KylinConfig config;
 
     @JsonProperty("name")
@@ -61,10 +65,15 @@ public class DataModelDesc extends RootPersistentEntity {
     private String description;
 
     @JsonProperty("fact_table")
-    private String factTable;
+    private String rootFactTable;
+
+    @JsonProperty("join_tables")
+    @JsonInclude(JsonInclude.Include.NON_NULL)
+    private JoinTableDesc[] joinTables;
 
     @JsonProperty("lookups")
-    private LookupDesc[] lookups;
+    @JsonInclude(JsonInclude.Include.NON_NULL)
+    private JoinTableDesc[] deprecatedLookups; // replaced by "join_tables" since KYLIN-1875
 
     @JsonProperty("dimensions")
     private List<ModelDimensionDesc> dimensions;
@@ -82,11 +91,14 @@ public class DataModelDesc extends RootPersistentEntity {
     private RealizationCapacity capacity = RealizationCapacity.MEDIUM;
 
     // computed attributes
-    private TableRef factTableRef;
-    private List<TableRef> lookupTableRefs = Lists.newArrayList();
-    private Map<String, TableRef> aliasMap = Maps.newHashMap(); // a table has exactly one alias
-    private Map<String, TableRef> tableNameMap = Maps.newHashMap(); // a table maybe referenced by multiple names
-    private Map<String, List<JoinDesc>> joinsMap = Maps.newHashMap();
+    private TableRef rootFactTableRef;
+    private Set<TableRef> factTableRefs = Sets.newLinkedHashSet();
+    private Set<TableRef> lookupTableRefs = Sets.newLinkedHashSet();
+    private Set<TableRef> allTableRefs = Sets.newLinkedHashSet();
+    private Map<String, TableRef> aliasMap = Maps.newHashMap(); // alias => TableRef, a table has exactly one alias
+    private Map<String, TableRef> tableNameMap = Maps.newHashMap(); // name => TableRef, a table maybe referenced by multiple names
+    private Map<TableRef, JoinDesc> pkSideJoinMap = Maps.newHashMap(); // table (PK side) => JoinTable
+    private Map<String, List<JoinDesc>> fkSideJoinMap = Maps.newHashMap(); // table (FK side) => JoinDesc
 
     /**
      * Error messages during resolving json metadata
@@ -101,6 +113,8 @@ public class DataModelDesc extends RootPersistentEntity {
         return name;
     }
 
+    // for test only
+    @Deprecated
     public void setName(String name) {
         this.name = name;
     }
@@ -117,67 +131,90 @@ public class DataModelDesc extends RootPersistentEntity {
         return description;
     }
 
-    public void setDescription(String description) {
-        this.description = description;
+    public TableRef getRootFactTable() {
+        return rootFactTableRef;
     }
 
-    public Collection<String> getAllTables() {
-        HashSet<String> ret = Sets.newHashSet();
-        ret.add(factTable);
-        for (LookupDesc lookupDesc : lookups)
-            ret.add(lookupDesc.getTable());
-        return ret;
+    public Set<TableRef> getAllTables() {
+        return allTableRefs;
     }
 
-    public String getFactTable() {
-        return factTable;
+    public Set<TableRef> getFactTables() {
+        return factTableRefs;
     }
 
-    public TableRef getFactTableRef() {
-        return factTableRef;
+    public Set<TableRef> getLookupTables() {
+        return lookupTableRefs;
     }
 
-    public List<TableRef> getLookupTableRefs() {
-        return lookupTableRefs;
+    public JoinTableDesc[] getJoinTables() {
+        return joinTables;
     }
-    
+
+    public Map<TableRef, JoinDesc> getPKSideJoinMap() {
+        return pkSideJoinMap;
+    }
+
+    public Map<String, List<JoinDesc>> getFKSideJoinMap() {
+        return fkSideJoinMap;
+    }
+
     @Deprecated
     public List<TableDesc> getLookupTableDescs() {
         List<TableDesc> result = Lists.newArrayList();
-        for (TableRef table : getLookupTableRefs()) {
+        for (TableRef table : getLookupTables()) {
             result.add(table.getTableDesc());
         }
         return result;
     }
 
-    public void setFactTable(String factTable) {
-        this.factTable = factTable.toUpperCase();
+    public boolean isLookupTable(TableRef t) {
+        if (t == null)
+            return false;
+        else
+            return lookupTableRefs.contains(t);
     }
 
-    public LookupDesc[] getLookups() {
-        return lookups;
+    public boolean isLookupTable(String fullTableName) {
+        for (TableRef t : lookupTableRefs) {
+            if (t.getTableIdentity().equals(fullTableName))
+                return true;
+        }
+        return false;
     }
-
-    public void setLookups(LookupDesc[] lookups) {
-        this.lookups = lookups;
+    
+    public boolean isFactTable(TableRef t) {
+        if (t == null)
+            return false;
+        else
+            return factTableRefs.contains(t);
     }
 
-    public boolean isFactTable(String factTable) {
-        return this.factTable.equalsIgnoreCase(factTable);
+    public boolean isFactTable(String fullTableName) {
+        for (TableRef t : factTableRefs) {
+            if (t.getTableIdentity().equals(fullTableName))
+                return true;
+        }
+        return false;
     }
-
+    
+    public boolean containsTable(String fullTableName) {
+        for (TableRef t : allTableRefs) {
+            if (t.getTableIdentity().equals(fullTableName))
+                return true;
+        }
+        return false;
+    }
+    
     public String getFilterCondition() {
         return filterCondition;
     }
 
-    public void setFilterCondition(String filterCondition) {
-        this.filterCondition = filterCondition;
-    }
-
     public PartitionDesc getPartitionDesc() {
         return partitionDesc;
     }
 
+    // for test only
     public void setPartitionDesc(PartitionDesc partitionDesc) {
         this.partitionDesc = partitionDesc;
     }
@@ -186,82 +223,52 @@ public class DataModelDesc extends RootPersistentEntity {
         return capacity;
     }
 
-    public void setCapacity(RealizationCapacity capacity) {
-        this.capacity = capacity;
-    }
-
-    public TblColRef findPKByFK(TblColRef fk, String joinType) {
-        assert isFactTable(fk.getTable());
-
-        TblColRef candidate = null;
-
-        for (LookupDesc dim : lookups) {
-            JoinDesc join = dim.getJoin();
-            if (join == null)
-                continue;
-
-            if (joinType != null && !joinType.equals(join.getType()))
-                continue;
-
-            int find = ArrayUtils.indexOf(join.getForeignKeyColumns(), fk);
-            if (find >= 0) {
-                candidate = join.getPrimaryKeyColumns()[find];
-                if (join.getForeignKeyColumns().length == 1) { // is single
-                    // column join?
-                    break;
-                }
-            }
-        }
-        return candidate;
-    }
-
-    public TblColRef findColumn(String table, String column) {
+    public TblColRef findColumn(String table, String column) throws IllegalArgumentException {
         TableRef tableRef = findTable(table);
-        TblColRef result = tableRef.getColumn(column);
+        TblColRef result = tableRef.getColumn(column.toUpperCase());
         if (result == null)
             throw new IllegalArgumentException("Column not found by " + table + "." + column);
         return result;
     }
-    
-    public TblColRef findColumn(String column) {
+
+    public TblColRef findColumn(String column) throws IllegalArgumentException {
         TblColRef result = null;
+        String input = column;
 
+        column = column.toUpperCase();
         int cut = column.lastIndexOf('.');
         if (cut > 0) {
             // table specified
             result = findColumn(column.substring(0, cut), column.substring(cut + 1));
         } else {
             // table not specified, try each table
-            result = factTableRef.getColumn(column);
-            if (result == null) {
-                for (TableRef tableRef : lookupTableRefs) {
-                    result = tableRef.getColumn(column);
-                    if (result != null)
-                        break;
-                }
+            for (TableRef tableRef : allTableRefs) {
+                result = tableRef.getColumn(column);
+                if (result != null)
+                    break;
             }
         }
 
         if (result == null)
-            throw new IllegalArgumentException("Column not found by " + column);
+            throw new IllegalArgumentException("Column not found by " + input);
 
         return result;
     }
 
     // find by unique name, that must uniquely identifies a table in the model
-    public TableRef findTable(String table) {
-        TableRef result = tableNameMap.get(table);
+    public TableRef findTable(String table) throws IllegalArgumentException {
+        TableRef result = tableNameMap.get(table.toUpperCase());
         if (result == null) {
             throw new IllegalArgumentException("Table not found by " + table);
         }
         return result;
     }
-    
+
     // find by table identity, that may match multiple tables in the model
-    public TableRef findFirstTable(String tableIdentity) {
-        if (factTableRef.getTableIdentity().equals(tableIdentity))
-            return factTableRef;
-        
+    public TableRef findFirstTable(String tableIdentity) throws IllegalArgumentException {
+        if (rootFactTableRef.getTableIdentity().equals(tableIdentity))
+            return rootFactTableRef;
+
         for (TableRef lookup : lookupTableRefs) {
             if (lookup.getTableIdentity().equals(tableIdentity))
                 return lookup;
@@ -271,45 +278,61 @@ public class DataModelDesc extends RootPersistentEntity {
 
     public void init(KylinConfig config, Map<String, TableDesc> tables) {
         this.config = config;
-        
-        lookupTableRefs.clear();
-        aliasMap.clear();
-        tableNameMap.clear();
-        joinsMap.clear();
-        
+
+        initJoinTablesForUpgrade();
         initTableAlias(tables);
         initJoinColumns();
         ModelDimensionDesc.capicalizeStrings(dimensions);
         initPartitionDesc();
     }
 
+    private void initJoinTablesForUpgrade() {
+        if (joinTables == null) {
+            joinTables = new JoinTableDesc[0];
+        }
+        if (deprecatedLookups != null) {
+            JoinTableDesc[] copy = Arrays.copyOf(joinTables, joinTables.length + deprecatedLookups.length);
+            System.arraycopy(deprecatedLookups, 0, copy, joinTables.length, deprecatedLookups.length);
+            joinTables = copy;
+            deprecatedLookups = null;
+        }
+    }
+
     private void initTableAlias(Map<String, TableDesc> tables) {
-        factTable = factTable.toUpperCase();
-        
-        if (tables.containsKey(factTable) == false)
-            throw new IllegalStateException("Fact table does not exist:" + factTable);
-        
-        TableDesc factDesc = tables.get(factTable);
-        factTableRef = new TableRef(this, factDesc.getName(), factDesc);
-        addAlias(factTableRef);
-        
-        for (LookupDesc lookup : lookups) {
-            lookup.setTable(lookup.getTable().toUpperCase());
-            
-            if (tables.containsKey(lookup.getTable()) == false)
-                throw new IllegalStateException("Lookup table does not exist:" + lookup.getTable());
-            
-            TableDesc tableDesc = tables.get(lookup.getTable());
-            String alias = lookup.getAlias();
+        factTableRefs.clear();
+        lookupTableRefs.clear();
+        allTableRefs.clear();
+        aliasMap.clear();
+        tableNameMap.clear();
+
+        rootFactTable = rootFactTable.toUpperCase();
+        if (tables.containsKey(rootFactTable) == false)
+            throw new IllegalStateException("Root fact table does not exist:" + rootFactTable);
+
+        TableDesc rootDesc = tables.get(rootFactTable);
+        rootFactTableRef = new TableRef(this, rootDesc.getName(), rootDesc);
+        addAlias(rootFactTableRef);
+        factTableRefs.add(rootFactTableRef);
+
+        for (JoinTableDesc join : joinTables) {
+            join.setTable(join.getTable().toUpperCase());
+
+            if (tables.containsKey(join.getTable()) == false)
+                throw new IllegalStateException("Join table does not exist:" + join.getTable());
+
+            TableDesc tableDesc = tables.get(join.getTable());
+            String alias = join.getAlias();
             if (alias == null)
                 alias = tableDesc.getName();
             TableRef ref = new TableRef(this, alias, tableDesc);
-            lookup.setTableRef(ref);
-            lookupTableRefs.add(ref);
+            join.setTableRef(ref);
             addAlias(ref);
+            (join.getKind() == TableKind.LOOKUP ? lookupTableRefs : factTableRefs).add(ref);
         }
 
         tableNameMap.putAll(aliasMap);
+        allTableRefs.addAll(factTableRefs);
+        allTableRefs.addAll(lookupTableRefs);
     }
 
     private void addAlias(TableRef ref) {
@@ -317,7 +340,7 @@ public class DataModelDesc extends RootPersistentEntity {
         if (aliasMap.containsKey(alias))
             throw new IllegalStateException("Alias '" + alias + "' ref to multiple tables: " + ref.getTableIdentity() + ", " + aliasMap.get(alias).getTableIdentity());
         aliasMap.put(alias, ref);
-        
+
         TableDesc table = ref.getTableDesc();
         addTableName(table.getName(), ref);
         addTableName(table.getIdentity(), ref);
@@ -337,11 +360,14 @@ public class DataModelDesc extends RootPersistentEntity {
     }
 
     private void initJoinColumns() {
-        for (LookupDesc lookup : this.lookups) {
-            TableRef dimTable = lookup.getTableRef();
-            JoinDesc join = lookup.getJoin();
+        pkSideJoinMap.clear();
+        fkSideJoinMap.clear();
+
+        for (JoinTableDesc joinTable : joinTables) {
+            TableRef dimTable = joinTable.getTableRef();
+            JoinDesc join = joinTable.getJoin();
             if (join == null)
-                continue;
+                throw new IllegalStateException("Missing join conditions on table " + dimTable);
 
             StringUtil.toUpperCaseArray(join.getForeignKey(), join.getForeignKey());
             StringUtil.toUpperCaseArray(join.getPrimaryKey(), join.getPrimaryKey());
@@ -362,41 +388,41 @@ public class DataModelDesc extends RootPersistentEntity {
             String[] fks = join.getForeignKey();
             TblColRef[] fkCols = new TblColRef[fks.length];
             for (int i = 0; i < fks.length; i++) {
-                TblColRef col = factTableRef.getColumn(fks[i]);
+                TblColRef col = findColumn(fks[i]);
                 if (col == null) {
-                    throw new IllegalStateException("Can't find column " + fks[i] + " in table " + this.getFactTable());
+                    throw new IllegalStateException("Can't find column " + fks[i] + " in table " + this.getRootFactTable());
                 }
                 fkCols[i] = col;
             }
             join.setForeignKeyColumns(fkCols);
-            
+
             join.sortByFK();
 
             // Validate join in dimension
+            TableRef fkTable = fkCols[0].getTableRef();
+            if (pkCols.length == 0 || fkCols.length == 0)
+                throw new IllegalStateException("Missing join columns on table " + dimTable);
             if (pkCols.length != fkCols.length) {
-                throw new IllegalStateException("Primary keys(" + lookup.getTable() + ")" + Arrays.toString(pks) + " are not consistent with Foreign keys(" + this.getFactTable() + ") " + Arrays.toString(fks));
+                throw new IllegalStateException("Primary keys(" + dimTable + ")" + Arrays.toString(pks) + " are not consistent with Foreign keys(" + fkTable + ") " + Arrays.toString(fks));
             }
             for (int i = 0; i < fkCols.length; i++) {
                 if (!fkCols[i].getDatatype().equals(pkCols[i].getDatatype())) {
-                    logger.warn("PK " + lookup.getTable() + "." + pkCols[i].getName() + "." + pkCols[i].getDatatype() + " are not consistent with FK " + this.getFactTable() + "." + fkCols[i].getName() + "." + fkCols[i].getDatatype());
+                    logger.warn("PK " + dimTable + "." + pkCols[i].getName() + "." + pkCols[i].getDatatype() + " are not consistent with FK " + fkTable + "." + fkCols[i].getName() + "." + fkCols[i].getDatatype());
                 }
             }
 
-            List<JoinDesc> list = joinsMap.get(factTableRef.getTableIdentity());
-            if (list == null)
-                joinsMap.put(factTableRef.getTableIdentity(), list = Lists.newArrayListWithCapacity(4));
+            // pk/fk side join maps
+            pkSideJoinMap.put(dimTable, join);
+            List<JoinDesc> list = fkSideJoinMap.get(fkTable.getTableIdentity());
+            if (list == null) {
+                fkSideJoinMap.put(fkTable.getTableIdentity(), list = Lists.newArrayListWithCapacity(4));
+            }
             list.add(join);
         }
     }
-    
-    public Map<String, List<JoinDesc>> getJoinsMap() {
-        return joinsMap;
-    }
 
     /**
      * Add error info and thrown exception out
-     *
-     * @param message
      */
     public void addError(String message) {
         addError(message, false);
@@ -431,7 +457,7 @@ public class DataModelDesc extends RootPersistentEntity {
 
         if (!name.equals(modelDesc.name))
             return false;
-        if (!getFactTable().equals(modelDesc.getFactTable()))
+        if (!getRootFactTable().equals(modelDesc.getRootFactTable()))
             return false;
 
         return true;
@@ -441,7 +467,7 @@ public class DataModelDesc extends RootPersistentEntity {
     public int hashCode() {
         int result = 0;
         result = 31 * result + name.hashCode();
-        result = 31 * result + getFactTable().hashCode();
+        result = 31 * result + getRootFactTable().hashCode();
         return result;
     }
 
@@ -466,25 +492,30 @@ public class DataModelDesc extends RootPersistentEntity {
         return metrics;
     }
 
+    @Deprecated
     public void setDimensions(List<ModelDimensionDesc> dimensions) {
         this.dimensions = dimensions;
     }
 
+    @Deprecated
     public void setMetrics(String[] metrics) {
         this.metrics = metrics;
     }
 
-    public static DataModelDesc getCopyOf(DataModelDesc dataModelDesc) {
-        DataModelDesc newDataModelDesc = new DataModelDesc();
-        newDataModelDesc.setName(dataModelDesc.getName());
-        newDataModelDesc.setDescription(dataModelDesc.getDescription());
-        newDataModelDesc.setDimensions(dataModelDesc.getDimensions());
-        newDataModelDesc.setFilterCondition(dataModelDesc.getFilterCondition());
-        newDataModelDesc.setFactTable(dataModelDesc.getFactTable());
-        newDataModelDesc.setLookups(dataModelDesc.getLookups());
-        newDataModelDesc.setMetrics(dataModelDesc.getMetrics());
-        newDataModelDesc.setPartitionDesc(PartitionDesc.getCopyOf(dataModelDesc.getPartitionDesc()));
-        newDataModelDesc.updateRandomUuid();
-        return newDataModelDesc;
+    public static DataModelDesc getCopyOf(DataModelDesc orig) {
+        DataModelDesc copy = new DataModelDesc();
+        copy.name = orig.name;
+        copy.owner = orig.owner;
+        copy.description = orig.description;
+        copy.rootFactTable = orig.rootFactTable;
+        copy.joinTables = orig.joinTables;
+        copy.dimensions = orig.dimensions;
+        copy.metrics = orig.metrics;
+        copy.filterCondition = orig.filterCondition;
+        copy.partitionDesc = PartitionDesc.getCopyOf(orig.getPartitionDesc());
+        copy.capacity = orig.capacity;
+        copy.updateRandomUuid();
+        return copy;
     }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinDesc.java
index 1dd8725..d3c0745 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinDesc.java
@@ -88,6 +88,7 @@ public class JoinDesc {
     }
 
     public void setPrimaryKeyColumns(TblColRef[] primaryKeyColumns) {
+        checkSameTable(primaryKeyColumns);
         this.primaryKeyColumns = primaryKeyColumns;
     }
 
@@ -96,9 +97,19 @@ public class JoinDesc {
     }
 
     public void setForeignKeyColumns(TblColRef[] foreignKeyColumns) {
+        checkSameTable(primaryKeyColumns);
         this.foreignKeyColumns = foreignKeyColumns;
     }
 
+    private void checkSameTable(TblColRef[] cols) {
+        if (cols == null || cols.length == 0)
+            return;
+        
+        TableRef tableRef = cols[0].getTableRef();
+        for (int i = 1; i < cols.length; i++)
+            Preconditions.checkState(tableRef == cols[i].getTableRef());
+    }
+
     public void sortByFK() {
         Preconditions.checkState(primaryKey.length == foreignKey.length && primaryKey.length == primaryKeyColumns.length && foreignKey.length == foreignKeyColumns.length);
         boolean cont = true;

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java
new file mode 100644
index 0000000..630156f
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java
@@ -0,0 +1,77 @@
+/*
+ * 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.metadata.model;
+
+import org.apache.kylin.metadata.model.DataModelDesc.TableKind;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class JoinTableDesc {
+
+    @JsonProperty("table")
+    private String table;
+
+    @JsonProperty("kind")
+    @JsonInclude(JsonInclude.Include.NON_NULL)
+    private TableKind kind = TableKind.LOOKUP;
+    
+    @JsonProperty("alias")
+    @JsonInclude(JsonInclude.Include.NON_NULL)
+    private String alias;
+    
+    @JsonProperty("join")
+    private JoinDesc join;
+    
+    private TableRef tableRef;
+
+    public String getTable() {
+        return table;
+    }
+
+    void setTable(String table) {
+        this.table = table;
+    }
+
+    public TableKind getKind() {
+        return kind;
+    }
+    
+    public String getAlias() {
+        return alias;
+    }
+    
+    public JoinDesc getJoin() {
+        return join;
+    }
+
+    public TableRef getTableRef() {
+        return tableRef;
+    }
+
+    void setTableRef(TableRef ref) {
+        this.tableRef = ref;
+    }
+    
+    
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-metadata/src/main/java/org/apache/kylin/metadata/model/LookupDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/LookupDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/LookupDesc.java
deleted file mode 100644
index b8828f7..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/LookupDesc.java
+++ /dev/null
@@ -1,67 +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.metadata.model;
-
-import com.fasterxml.jackson.annotation.JsonAutoDetect;
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class LookupDesc {
-
-    @JsonProperty("table")
-    private String table;
-
-    @JsonProperty("alias")
-    @JsonInclude(JsonInclude.Include.NON_NULL)
-    private String alias;
-    
-    @JsonProperty("join")
-    private JoinDesc join;
-    
-    private TableRef tableRef;
-
-    public String getTable() {
-        return table;
-    }
-
-    void setTable(String table) {
-        this.table = table;
-    }
-
-    public String getAlias() {
-        return alias;
-    }
-    
-    public JoinDesc getJoin() {
-        return join;
-    }
-
-    public TableRef getTableRef() {
-        return tableRef;
-    }
-
-    void setTableRef(TableRef ref) {
-        this.tableRef = ref;
-    }
-    
-    
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
index d6f18e5..0261f41 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
@@ -278,13 +278,16 @@ public class PartitionDesc {
         }
     }
 
-    public static PartitionDesc getCopyOf(PartitionDesc partitionDesc) {
-        PartitionDesc newPartDesc = new PartitionDesc();
-        newPartDesc.setCubePartitionType(partitionDesc.getCubePartitionType());
-        newPartDesc.setPartitionDateColumn(partitionDesc.getPartitionDateColumn());
-        newPartDesc.setPartitionDateFormat(partitionDesc.getPartitionDateFormat());
-        newPartDesc.setPartitionDateStart(partitionDesc.getPartitionDateStart());
-        return newPartDesc;
+    public static PartitionDesc getCopyOf(PartitionDesc orig) {
+        PartitionDesc ret = new PartitionDesc();
+        ret.partitionDateColumn = orig.partitionDateColumn;
+        ret.partitionTimeColumn = orig.partitionTimeColumn;
+        ret.partitionDateStart = orig.partitionDateStart; //Deprecated
+        ret.partitionDateFormat = orig.partitionDateFormat;
+        ret.partitionTimeFormat = orig.partitionTimeFormat;
+        ret.partitionType = orig.partitionType;
+        ret.partitionConditionBuilderClz = orig.partitionConditionBuilderClz;
+        return ret;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectL2Cache.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectL2Cache.java b/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectL2Cache.java
index 15fa8ce..82c0de3 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectL2Cache.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectL2Cache.java
@@ -126,22 +126,11 @@ class ProjectL2Cache {
             return Collections.unmodifiableSet(tableCache.realizations);
     }
 
-    public List<IRealization> getOnlineRealizationByFactTable(String project, String factTable) {
-        Set<IRealization> realizations = getRealizationsByTable(project, factTable);
-        List<IRealization> result = Lists.newArrayListWithCapacity(realizations.size());
-        for (IRealization r : realizations) {
-            if (r.getFactTable().equalsIgnoreCase(factTable) && r.isReady()) {
-                result.add(r);
-            }
-        }
-        return result;
-    }
-
     public List<MeasureDesc> listEffectiveRewriteMeasures(String project, String factTable, boolean onlyRewriteMeasure) {
         Set<IRealization> realizations = getRealizationsByTable(project, factTable);
         List<MeasureDesc> result = Lists.newArrayList();
         for (IRealization r : realizations) {
-            if (r.getFactTable().equalsIgnoreCase(factTable) && r.isReady()) {
+            if (r.getModel().isFactTable(factTable) && r.isReady()) {
                 for (MeasureDesc m : r.getMeasures()) {
                     FunctionDesc func = m.getFunction();
                     if (onlyRewriteMeasure) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java
index 2838e56..b547843 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java
@@ -429,10 +429,6 @@ public class ProjectManager {
         return l2Cache.getRealizationsByTable(norm(project), tableName.toUpperCase());
     }
 
-    public List<IRealization> getOnlineRealizationByFactTable(String project, String factTable) {
-        return l2Cache.getOnlineRealizationByFactTable(norm(project), factTable.toUpperCase());
-    }
-
     public List<MeasureDesc> listEffectiveRewriteMeasures(String project, String factTable) {
         return l2Cache.listEffectiveRewriteMeasures(norm(project), factTable.toUpperCase(), true);
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealization.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealization.java b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealization.java
index f1770d7..a0243f4 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealization.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealization.java
@@ -39,9 +39,7 @@ public interface IRealization extends IStorageAware {
      */
     public RealizationType getType();
 
-    public DataModelDesc getDataModelDesc();
-
-    public String getFactTable();
+    public DataModelDesc getModel();
 
     public Set<TblColRef> getAllColumns();
     

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java
index d7007ac..4f0e446 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java
@@ -203,18 +203,13 @@ public class HybridInstance extends RootPersistentEntity implements IRealization
     }
 
     @Override
-    public DataModelDesc getDataModelDesc() {
+    public DataModelDesc getModel() {
         if (this.getLatestRealization() != null)
-            return this.getLatestRealization().getDataModelDesc();
+            return this.getLatestRealization().getModel();
         return null;
     }
 
     @Override
-    public String getFactTable() {
-        return getRealizations()[0].getFactTable();
-    }
-
-    @Override
     public Set<TblColRef> getAllColumns() {
         init();
         return allColumns;

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
index 69387f8..80636d3 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
@@ -27,11 +27,12 @@ import static org.apache.hadoop.util.StringUtils.formatTime;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -67,8 +68,8 @@ import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.job.JobInstance;
 import org.apache.kylin.job.exception.JobException;
-import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.source.SourceFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -449,22 +450,20 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
     }
 
     protected void attachKylinPropsAndMetadata(TableDesc table, Configuration conf) throws IOException {
-        ArrayList<String> dumpList = new ArrayList<String>();
+        Set<String> dumpList = new LinkedHashSet<>();
         dumpList.add(table.getResourcePath());
         attachKylinPropsAndMetadata(dumpList, KylinConfig.getInstanceFromEnv(), conf);
     }
 
     protected void attachKylinPropsAndMetadata(CubeInstance cube, Configuration conf) throws IOException {
-        MetadataManager metaMgr = MetadataManager.getInstance(cube.getConfig());
-
         // write cube / model_desc / cube_desc / dict / table
-        ArrayList<String> dumpList = new ArrayList<String>();
+        Set<String> dumpList = new LinkedHashSet<>();
         dumpList.add(cube.getResourcePath());
         dumpList.add(cube.getDescriptor().getModel().getResourcePath());
         dumpList.add(cube.getDescriptor().getResourcePath());
 
-        for (String tableName : cube.getDescriptor().getModel().getAllTables()) {
-            TableDesc table = metaMgr.getTableDesc(tableName);
+        for (TableRef tableRef: cube.getDescriptor().getModel().getAllTables()) {
+            TableDesc table = tableRef.getTableDesc();
             dumpList.add(table.getResourcePath());
             List<String> dependentResources = SourceFactory.getMRDependentResources(table);
             dumpList.addAll(dependentResources);
@@ -476,7 +475,7 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
         attachKylinPropsAndMetadata(dumpList, cube.getConfig(), conf);
     }
 
-    protected void attachKylinPropsAndMetadata(ArrayList<String> dumpList, KylinConfig kylinConfig, Configuration conf) throws IOException {
+    protected void attachKylinPropsAndMetadata(Set<String> dumpList, KylinConfig kylinConfig, Configuration conf) throws IOException {
         File tmp = File.createTempFile("kylin_job_meta", "");
         FileUtils.forceDelete(tmp); // we need a directory, so delete the file first
 
@@ -524,7 +523,7 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
         }
     }
 
-    private void dumpResources(KylinConfig kylinConfig, File metaDir, ArrayList<String> dumpList) throws IOException {
+    private void dumpResources(KylinConfig kylinConfig, File metaDir, Set<String> dumpList) throws IOException {
         ResourceStore from = ResourceStore.getStore(kylinConfig);
         KylinConfig localConfig = KylinConfig.createInstanceFromUri(metaDir.getAbsolutePath());
         ResourceStore to = ResourceStore.getStore(localConfig);

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
index 67c0f4c..a79d5aa 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
@@ -51,6 +51,7 @@ import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.MeasureIngester;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
 
 import com.google.common.collect.Lists;
@@ -250,14 +251,14 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
         Boolean ret = dimensionsNeedDict.get(col);
         if (ret != null)
             return ret;
-        else {
-            ret = cubeDesc.getRowkey().isUseDictionary(col);
-            if (ret) {
-                String dictTable = DictionaryManager.getInstance(config).decideSourceData(cubeDesc.getModel(), col).getTable();
-                ret = cubeDesc.getFactTable().equalsIgnoreCase(dictTable);
-            }
-            dimensionsNeedDict.put(col, ret);
-            return ret;
+        
+        ret = cubeDesc.getRowkey().isUseDictionary(col);
+        if (ret) {
+            TableRef srcTable = DictionaryManager.getInstance(config).decideSourceData(cubeDesc.getModel(), col).getTableRef();
+            ret = cubeDesc.getModel().isFactTable(srcTable);
         }
+        
+        dimensionsNeedDict.put(col, ret);
+        return ret;
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java
index 7280d39..2281612 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java
@@ -37,6 +37,7 @@ import org.apache.kylin.job.exception.ExecuteException;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableContext;
 import org.apache.kylin.job.execution.ExecuteResult;
+import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -110,8 +111,8 @@ public class MergeDictionaryStep extends AbstractExecutable {
         CubeDesc cubeDesc = cube.getDescriptor();
 
         for (TblColRef col : cubeDesc.getAllColumnsNeedDictionaryBuilt()) {
-            String dictTable = dictMgr.decideSourceData(cubeDesc.getModel(), col).getTable();
-            if (cubeDesc.getFactTable().equalsIgnoreCase(dictTable)) {
+            TableRef srcTable = dictMgr.decideSourceData(cubeDesc.getModel(), col).getTableRef();
+            if (cubeDesc.getModel().isFactTable(srcTable)) {
                 colsNeedMeringDict.add(col);
             } else {
                 colsNeedCopyDict.add(col);

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
index c2f53e1..8abb84c 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
@@ -108,7 +108,7 @@ public class BuildCubeWithStream {
         cubeManager = CubeManager.getInstance(kylinConfig);
 
         final CubeInstance cubeInstance = CubeManager.getInstance(kylinConfig).getCube(cubeName);
-        final String factTable = cubeInstance.getFactTable();
+        final String factTable = cubeInstance.getRootFactTable();
 
         final StreamingManager streamingManager = StreamingManager.getInstance(kylinConfig);
         final StreamingConfig streamingConfig = streamingManager.getStreamingConfig(factTable);

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITStorageTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITStorageTest.java b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITStorageTest.java
index 78b87fd..db9d133 100644
--- a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITStorageTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITStorageTest.java
@@ -76,7 +76,7 @@ public class ITStorageTest extends HBaseMetadataTestCase {
         String url = KylinConfig.getInstanceFromEnv().getStorageUrl();
         context = new StorageContext();
         context.setConnUrl(url);
-        mockup = new StorageMockUtils(cube.getDataModelDesc());
+        mockup = new StorageMockUtils(cube.getModel());
     }
 
     @After

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
index 941ef6a..d9cfe02 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
@@ -163,8 +163,14 @@ public interface OLAPRel extends RelNode {
         }
 
         public static boolean needRewrite(OLAPContext ctx) {
-            boolean hasFactTable = ctx.hasJoin || ctx.firstTableScan.getTableName().equals(ctx.realization.getFactTable());
-            return hasFactTable;
+            if (ctx.hasJoin)
+                return true;
+            
+            String realRootFact = ctx.realization.getModel().getRootFactTable().getTableIdentity();
+            if (ctx.firstTableScan.getTableName().equals(realRootFact))
+                return true;
+            
+            return false;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
index e63a978..3c2bf48 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
@@ -271,7 +271,7 @@ public class OLAPTableScan extends TableScan implements OLAPRel, EnumerableRel {
 
     private String genExecFunc() {
         // if the table to scan is not the fact table of cube, then it's a lookup table
-        if (context.hasJoin == false && tableName.equalsIgnoreCase(context.realization.getFactTable()) == false) {
+        if (context.hasJoin == false && context.realization.getModel().isLookupTable(tableName)) {
             return "executeLookupTableQuery";
         } else {
             return "executeOLAPQuery";

http://git-wip-us.apache.org/repos/asf/kylin/blob/cfa3d0d4/query/src/main/java/org/apache/kylin/query/routing/ModelChooser.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/routing/ModelChooser.java b/query/src/main/java/org/apache/kylin/query/routing/ModelChooser.java
index 08616bd..f1f5a48 100644
--- a/query/src/main/java/org/apache/kylin/query/routing/ModelChooser.java
+++ b/query/src/main/java/org/apache/kylin/query/routing/ModelChooser.java
@@ -29,7 +29,7 @@ import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.metadata.model.ColumnDesc;
 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.JoinTableDesc;
 import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.project.ProjectManager;
@@ -74,7 +74,7 @@ public class ModelChooser {
         Map<String, String> result = Maps.newHashMap();
 
         // the greedy match is not perfect but works for the moment
-        Map<String, List<JoinDesc>> modelJoinsMap = model.getJoinsMap();
+        Map<String, List<JoinDesc>> modelJoinsMap = model.getFKSideJoinMap();
         for (OLAPContext ctx : contexts) {
             for (JoinDesc queryJoin : ctx.joins) {
                 String fkTable = queryJoin.getForeignKeyColumns()[0].getTable();
@@ -133,7 +133,7 @@ public class ModelChooser {
                 continue;
 
             RealizationCost cost = new RealizationCost(real);
-            DataModelDesc m = real.getDataModelDesc();
+            DataModelDesc m = real.getModel();
             Set<IRealization> set = models.get(m);
             if (set == null) {
                 set = Sets.newHashSet();
@@ -184,8 +184,8 @@ public class ModelChooser {
 
             // ref CubeInstance.getCost()
             int c = real.getAllDimensions().size() * CubeInstance.COST_WEIGHT_DIMENSION + real.getMeasures().size() * CubeInstance.COST_WEIGHT_MEASURE;
-            for (LookupDesc lookup : real.getDataModelDesc().getLookups()) {
-                if (lookup.getJoin().isInnerJoin())
+            for (JoinTableDesc join : real.getModel().getJoinTables()) {
+                if (join.getJoin().isInnerJoin())
                     c += CubeInstance.COST_WEIGHT_INNER_JOIN;
             }
             this.cost = c;