You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by xx...@apache.org on 2020/06/15 02:51:16 UTC

[kylin] branch master updated (faa15ad -> b5ef6dd)

This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git.


    from faa15ad  KYLIN-4355 add ut
     new 6aeaf23  KYLIN-4420 Add model compatibility check to allow more compatible update
     new c9c723f  KYLIN-4421 Add table alias field for root fact table in DataModelDesc
     new d494b8d  KYLIN-4421 Allow to update table & database name
     new 70521fd  KYLIN-4421 Add a rest API for updating table & database name
     new 6bd5b43  KYLIN-4422 Allow to change data type from varchar to datetime
     new f13771e  KYLIN-4484 Relax checkValidationInModel() in TableSchemaUpdateChecker
     new 67f347b  KYLIN-4419 Refine IT sqls with adding order by
     new a0ef189  KYLIN-4487 Backend support for auto-migration to allow user to do cube migration by self service
     new c5d1a39  KYLIN-4499 Extract kylin server self discovery service from CuratorScheduler
     new e0d92d1  KYLIN-4488 Revert KYLIN-2999
     new 569552d  KYLIN-4488 Frontend support for auto-migration to allow user to do cube migration by self service
     new d502c52  KYLIN-4489 Add hive table compatibility check rest service
     new 2e2a4df  KYLIN-4489 Create a tool for migration cross clusters
     new 36fa31b  KYLIN-4485 Minor fix
     new b5ef6dd  KYLIN-4429 Auto convert filter date values to date format string when the related column type is string

The 15 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 core-common/pom.xml                                |  11 +
 .../java/org/apache/kylin/common/KylinConfig.java  |  34 +-
 .../org/apache/kylin/common/KylinConfigBase.java   |  74 +-
 .../apache/kylin/common/restclient/RestClient.java |  77 ++-
 .../org/apache/kylin/common/util/CheckUtil.java    |  14 +
 .../common/zookeeper/KylinServerDiscovery.java     | 164 +++++
 .../src/main/resources/kylin-defaults.properties   |   3 +
 .../kylin/common/zookeeper}/ExampleServer.java     |  26 +-
 .../common/zookeeper/KylinServerDiscoveryTest.java |  46 +-
 .../java/org/apache/kylin/cube/CubeInstance.java   |  65 +-
 .../java/org/apache/kylin/cube/CubeSegment.java    |  41 ++
 .../kylin/cube/gridtable/CubeCodeSystem.java       |  21 +
 .../apache/kylin/cube/model/AggregationGroup.java  |   2 +
 .../java/org/apache/kylin/cube/model/CubeDesc.java |  65 ++
 .../apache/kylin/cube/model/DictionaryDesc.java    |  17 +
 .../kylin/cube/model/HBaseColumnFamilyDesc.java    |  16 +
 .../org/apache/kylin/cube/model/RowKeyColDesc.java |  12 +-
 .../org/apache/kylin/cube/model/RowKeyDesc.java    |   4 +
 .../org/apache/kylin/cube/model/SelectRule.java    |  40 ++
 .../apache/kylin/cube/model/SnapshotTableDesc.java |  33 +-
 .../org/apache/kylin/dict/DictionaryGenerator.java | 104 +--
 .../kylin/dict/lookup/LookupStringTable.java       |   8 +-
 .../apache/kylin/dict/DictionaryProviderTest.java  |  26 +-
 .../java/org/apache/kylin/job/JoinedFlatTable.java |   3 +-
 .../kylin/job/impl/curator/CuratorScheduler.java   | 100 +--
 .../kylin/metadata/TableMetadataManager.java       |  29 +-
 .../apache/kylin/metadata/model/DataModelDesc.java |  59 +-
 .../kylin/metadata/model/DataModelManager.java     |   3 +
 .../apache/kylin/metadata/model/JoinTableDesc.java |   9 +
 .../kylin/metadata/model/ModelDimensionDesc.java   |  18 +
 .../apache/kylin/metadata/model/ParameterDesc.java |   7 +-
 .../apache/kylin/metadata/model/PartitionDesc.java |  72 ++
 .../org/apache/kylin/metadata/model/TableDesc.java |  14 +-
 .../kylin/metadata/project/ProjectManager.java     |  22 +-
 .../org/apache/kylin/metadata/tuple/Tuple.java     |  19 +-
 .../org/apache/kylin/metadata/tuple/TupleInfo.java |   4 +
 .../kylin/storage/gtrecord/CubeTupleConverter.java |  42 +-
 .../storage/translate/DerivedFilterTranslator.java |  27 +-
 {server-base => cube-migration}/pom.xml            | 132 +---
 .../kylin/rest/controller/MigrationController.java | 202 ++++++
 .../kylin/rest/exception/ConflictException.java    |  29 +-
 .../rest/exception/RuleValidationException.java    |  30 +-
 .../kylin/rest/request/MigrationRequest.java       |  46 +-
 .../kylin/rest/service/MigrationRuleSet.java       | 469 +++++++++++++
 .../kylin/rest/service/MigrationService.java       | 225 ++++++
 .../kylin/rest/util/MailNotificationUtil.java      |  30 +-
 .../mail_templates/MIGRATION_APPROVED.ftl          | 189 +++++
 .../mail_templates/MIGRATION_COMPLETED.ftl         | 192 ++++++
 .../resources/mail_templates/MIGRATION_FAILED.ftl  | 220 ++++++
 .../mail_templates/MIGRATION_REJECTED.ftl          | 196 ++++++
 .../resources/mail_templates/MIGRATION_REQUEST.ftl | 192 ++++++
 .../sql_casewhen/{query56.sql => query58.sql}      |   4 +-
 .../test/resources/query/sql_timestamp/query01.sql |   1 +
 .../test/resources/query/sql_timestamp/query02.sql |   1 +
 .../test/resources/query/sql_timestamp/query03.sql |   1 +
 .../test/resources/query/sql_timestamp/query05.sql |   1 +
 .../test/resources/query/sql_timestamp/query11.sql |   1 +
 .../test/resources/query/sql_timestamp/query12.sql |   1 +
 .../test/resources/query/sql_timestamp/query13.sql |   1 +
 .../test/resources/query/sql_timestamp/query14.sql |   1 +
 .../test/resources/query/sql_timestamp/query21.sql |   1 +
 .../resources/query/sql_timestamp/query21_a.sql    |   1 +
 .../test/resources/query/sql_timestamp/query22.sql |   1 +
 .../resources/query/sql_timestamp/query22_a.sql    |   1 +
 .../resources/query/sql_timestamp/query22_b.sql    |   1 +
 .../resources/query/sql_timestamp/query22_c.sql    |   1 +
 .../resources/query/sql_timestamp/query22_d.sql    |   1 +
 .../resources/query/sql_timestamp/query22_e.sql    |   1 +
 .../test/resources/query/sql_timestamp/query23.sql |   1 +
 .../resources/query/sql_timestamp/query23_a.sql    |   1 +
 .../test/resources/query/sql_timestamp/query25.sql |   1 +
 .../resources/query/sql_timestamp/query25_a.sql    |   1 +
 pom.xml                                            |  11 +
 .../query/relnode/visitor/TupleFilterVisitor.java  |  22 +
 .../kylin/rest/controller/AdminController.java     |  22 +
 .../kylin/rest/controller/CubeController.java      |   7 -
 .../kylin/rest/controller/TableController.java     |  17 +
 ...MetricsRequest.java => TableUpdateRequest.java} |  49 +-
 .../apache/kylin/rest/service/AdminService.java    |  14 +
 .../org/apache/kylin/rest/service/CubeService.java |  49 --
 .../apache/kylin/rest/service/HBaseInfoUtil.java   |  17 +-
 .../org/apache/kylin/rest/service/JobService.java  |   6 +
 .../rest/service/ModelSchemaUpdateChecker.java     | 258 +++++++
 .../apache/kylin/rest/service/ModelService.java    |  30 +
 .../rest/service/TableSchemaUpdateChecker.java     | 113 ++-
 .../apache/kylin/rest/service/TableService.java    | 146 ++++
 .../service/update/TableSchemaUpdateMapping.java   |  72 ++
 .../rest/service/update/TableSchemaUpdater.java    | 191 ++++++
 .../service/update/TableSchemaUpdaterTest.java     | 153 +++++
 .../resources/update/TableSchemaUpdateMapping.json |  19 +
 .../test_kylin_cube_with_slr_left_join_ready.json  |  20 +-
 .../update}/cube_desc/ci_left_join_cube.json       |  14 +-
 .../update}/model_desc/ci_inner_join_model.json    |  24 +-
 .../table/EDW.CAL_DT.json}                         | 108 +--
 .../table/TEST.COUNTRY.json}                       |   8 +-
 .../resources/update/table/TEST.KYLIN_FACT.json    |  20 +-
 .../resources/update/table/TEST.TEST_ACCOUNT.json  |  10 +-
 .../table/TEST.TEST_CATEGORY_GROUPINGS.json}       |   2 +-
 server/pom.xml                                     |  14 +
 server/src/main/resources/kylinSecurity.xml        |   5 +
 .../kylin/rest/service/AdminServiceTest.java       |   2 +-
 .../kylin/storage/hbase/HBaseResourceStore.java    |   7 +-
 .../storage/hbase/util/DeployCoprocessorCLI.java   |  21 +-
 tool/pom.xml                                       |  11 +
 .../org/apache/kylin/tool/CubeMigrationCLI.java    |   4 +-
 .../apache/kylin/tool/migration/ClusterUtil.java   | 165 +++++
 .../tool/migration/CompatibilityCheckRequest.java  |  38 +-
 .../migration/CubeMigrationCrossClusterCLI.java    | 757 +++++++++++++++++++++
 .../kylin/tool/migration/DstClusterUtil.java       | 371 ++++++++++
 .../kylin/tool/migration/SrcClusterUtil.java       | 148 ++++
 .../kylin/tool/query/ProbabilityGenerator.java     |  88 +++
 .../kylin/tool/query/ProbabilityGeneratorCLI.java  |  99 +++
 .../apache/kylin/tool/query/QueryGenerator.java    | 189 +++++
 .../apache/kylin/tool/query/QueryGeneratorCLI.java | 138 ++++
 .../tool/query/ProbabilityGeneratorCLITest.java    |  18 +-
 .../kylin/tool/query/QueryGeneratorCLITest.java    |  22 +-
 webapp/app/js/controllers/cubes.js                 | 186 ++++-
 webapp/app/js/controllers/models.js                |  10 +-
 webapp/app/js/services/cubes.js                    |  18 +-
 webapp/app/js/services/kylinProperties.js          |  16 +-
 webapp/app/partials/cubes/cube_migrate.html        |  91 +++
 webapp/app/partials/cubes/cubes.html               |   6 +-
 122 files changed, 6518 insertions(+), 815 deletions(-)
 create mode 100644 core-common/src/main/java/org/apache/kylin/common/zookeeper/KylinServerDiscovery.java
 rename {core-job/src/test/java/org/apache/kylin/job/impl/curator => core-common/src/test/java/org/apache/kylin/common/zookeeper}/ExampleServer.java (61%)
 rename core-job/src/test/java/org/apache/kylin/job/impl/curator/CuratorSchedulerTest.java => core-common/src/test/java/org/apache/kylin/common/zookeeper/KylinServerDiscoveryTest.java (71%)
 copy {server-base => cube-migration}/pom.xml (54%)
 mode change 100644 => 100755
 create mode 100644 cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java
 copy server-base/src/main/java/org/apache/kylin/rest/exception/NotFoundException.java => cube-migration/src/main/java/org/apache/kylin/rest/exception/ConflictException.java (71%)
 copy server-base/src/main/java/org/apache/kylin/rest/exception/NotFoundException.java => cube-migration/src/main/java/org/apache/kylin/rest/exception/RuleValidationException.java (65%)
 copy server-base/src/main/java/org/apache/kylin/rest/request/MetricsRequest.java => cube-migration/src/main/java/org/apache/kylin/rest/request/MigrationRequest.java (60%)
 create mode 100644 cube-migration/src/main/java/org/apache/kylin/rest/service/MigrationRuleSet.java
 create mode 100644 cube-migration/src/main/java/org/apache/kylin/rest/service/MigrationService.java
 copy server-base/src/main/java/org/apache/kylin/rest/util/PagingUtil.java => cube-migration/src/main/java/org/apache/kylin/rest/util/MailNotificationUtil.java (59%)
 create mode 100644 cube-migration/src/main/resources/mail_templates/MIGRATION_APPROVED.ftl
 create mode 100644 cube-migration/src/main/resources/mail_templates/MIGRATION_COMPLETED.ftl
 create mode 100644 cube-migration/src/main/resources/mail_templates/MIGRATION_FAILED.ftl
 create mode 100644 cube-migration/src/main/resources/mail_templates/MIGRATION_REJECTED.ftl
 create mode 100644 cube-migration/src/main/resources/mail_templates/MIGRATION_REQUEST.ftl
 copy kylin-it/src/test/resources/query/sql_casewhen/{query56.sql => query58.sql} (80%)
 copy server-base/src/main/java/org/apache/kylin/rest/request/{MetricsRequest.java => TableUpdateRequest.java} (57%)
 create mode 100644 server-base/src/main/java/org/apache/kylin/rest/service/ModelSchemaUpdateChecker.java
 create mode 100644 server-base/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdateMapping.java
 create mode 100644 server-base/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdater.java
 create mode 100644 server-base/src/test/java/org/apache/kylin/rest/service/update/TableSchemaUpdaterTest.java
 create mode 100644 server-base/src/test/resources/update/TableSchemaUpdateMapping.json
 copy {examples/test_case_data/localmeta => server-base/src/test/resources/update}/cube/test_kylin_cube_with_slr_left_join_ready.json (50%)
 copy {examples/test_case_data/localmeta => server-base/src/test/resources/update}/cube_desc/ci_left_join_cube.json (98%)
 copy {examples/test_case_data/localmeta => server-base/src/test/resources/update}/model_desc/ci_inner_join_model.json (89%)
 copy server-base/src/test/resources/{test_meta/table/EDW.TEST_CAL_DT.json => update/table/EDW.CAL_DT.json} (80%)
 copy server-base/src/test/resources/{test_meta/table/DEFAULT.TEST_COUNTRY.json => update/table/TEST.COUNTRY.json} (76%)
 copy examples/test_case_data/localmeta/table/DEFAULT.TEST_KYLIN_FACT.json => server-base/src/test/resources/update/table/TEST.KYLIN_FACT.json (81%)
 copy examples/test_case_data/localmeta/table/DEFAULT.TEST_ACCOUNT.json => server-base/src/test/resources/update/table/TEST.TEST_ACCOUNT.json (81%)
 copy server-base/src/test/resources/{test_meta/table/DEFAULT.TEST_CATEGORY_GROUPINGS.json => update/table/TEST.TEST_CATEGORY_GROUPINGS.json} (99%)
 create mode 100644 tool/src/main/java/org/apache/kylin/tool/migration/ClusterUtil.java
 copy server-base/src/main/java/org/apache/kylin/rest/request/LookupSnapshotBuildRequest.java => tool/src/main/java/org/apache/kylin/tool/migration/CompatibilityCheckRequest.java (54%)
 create mode 100644 tool/src/main/java/org/apache/kylin/tool/migration/CubeMigrationCrossClusterCLI.java
 create mode 100644 tool/src/main/java/org/apache/kylin/tool/migration/DstClusterUtil.java
 create mode 100644 tool/src/main/java/org/apache/kylin/tool/migration/SrcClusterUtil.java
 create mode 100644 tool/src/main/java/org/apache/kylin/tool/query/ProbabilityGenerator.java
 create mode 100644 tool/src/main/java/org/apache/kylin/tool/query/ProbabilityGeneratorCLI.java
 create mode 100644 tool/src/main/java/org/apache/kylin/tool/query/QueryGenerator.java
 create mode 100644 tool/src/main/java/org/apache/kylin/tool/query/QueryGeneratorCLI.java
 copy core-job/src/test/java/org/apache/kylin/job/BasicLocalMetaTest.java => tool/src/test/java/org/apache/kylin/tool/query/ProbabilityGeneratorCLITest.java (65%)
 mode change 100644 => 100755
 copy core-job/src/test/java/org/apache/kylin/job/BasicLocalMetaTest.java => tool/src/test/java/org/apache/kylin/tool/query/QueryGeneratorCLITest.java (61%)
 mode change 100644 => 100755
 create mode 100644 webapp/app/partials/cubes/cube_migrate.html


[kylin] 12/15: KYLIN-4489 Add hive table compatibility check rest service

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit d502c52ae3198c0bad39bde9dd3e244b67b95520
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Tue May 19 12:02:47 2020 +0800

    KYLIN-4489 Add hive table compatibility check rest service
---
 .../apache/kylin/common/restclient/RestClient.java |  7 +++
 .../kylin/rest/controller/MigrationController.java | 19 ++++++++
 .../rest/service/TableSchemaUpdateChecker.java     | 56 ++++++++++++++++++++--
 .../apache/kylin/rest/service/TableService.java    | 22 +++++++++
 server/src/main/resources/kylinSecurity.xml        |  2 +
 5 files changed, 103 insertions(+), 3 deletions(-)

diff --git a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
index a9971dd..d908f58 100644
--- a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
+++ b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
@@ -353,6 +353,13 @@ public class RestClient {
     }
 
     public void checkCompatibility(String jsonRequest) throws IOException {
+        checkCompatibility(jsonRequest, false);
+    }
+
+    public void checkCompatibility(String jsonRequest, boolean ifHiveCheck) throws IOException {
+        if (ifHiveCheck) {
+            checkCompatibility(jsonRequest, baseUrl + "/cubes/checkCompatibility/hiveTable");
+        }
         checkCompatibility(jsonRequest, baseUrl + "/cubes/checkCompatibility");
     }
 
diff --git a/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java b/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java
index 106d51f..efef5cf 100644
--- a/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java
+++ b/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java
@@ -163,6 +163,25 @@ public class MigrationController extends BasicController {
         }
     }
 
+    /**
+     * Check the schema compatibility for table
+     */
+    @RequestMapping(value = "/checkCompatibility/hiveTable", method = { RequestMethod.POST })
+    @ResponseBody
+    public void checkHiveTableCompatibility(@RequestBody CompatibilityCheckRequest request) {
+        try {
+            List<TableDesc> tableDescList = deserializeTableDescList(request);
+            for (TableDesc tableDesc : tableDescList) {
+                logger.info("Schema compatibility check for table {}", tableDesc.getName());
+                tableService.checkHiveTableCompatibility(request.getProjectName(), tableDesc);
+                logger.info("Pass schema compatibility check for table {}", tableDesc.getName());
+            }
+        } catch (Exception e) {
+            logger.error(e.getMessage(), e);
+            throw new ConflictException(e.getMessage(), e);
+        }
+    }
+
     private List<TableDesc> deserializeTableDescList(CompatibilityCheckRequest request) {
         List<TableDesc> result = Lists.newArrayList();
         try {
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java b/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java
index f03acb8..3a45f49 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java
@@ -23,22 +23,23 @@ import static java.lang.String.format;
 
 import java.util.List;
 import java.util.Locale;
+import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 
 import javax.annotation.Nullable;
 
-import org.apache.kylin.shaded.com.google.common.base.Preconditions;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.metadata.TableMetadataManager;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.DataModelDesc;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.model.DataModelManager;
 import org.apache.kylin.metadata.model.ModelDimensionDesc;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TblColRef;
 
+import org.apache.kylin.shaded.com.google.common.base.Preconditions;
 import org.apache.kylin.shaded.com.google.common.base.Predicate;
 import org.apache.kylin.shaded.com.google.common.collect.ImmutableList;
 import org.apache.kylin.shaded.com.google.common.collect.Iterables;
@@ -340,4 +341,53 @@ public class TableSchemaUpdateChecker {
 
         return usedColumns;
     }
+
+    public CheckResult allowMigrate(TableDesc newTableDesc, TableDesc hiveTableDesc) throws Exception {
+        final String fullTableName = newTableDesc.getIdentity();
+
+        List<String> issues = Lists.newArrayList();
+        checkAllColumnsInHiveTableDesc(hiveTableDesc, newTableDesc, issues);
+        if (issues.isEmpty()) {
+            return new CheckResult(true,
+                    format(Locale.ROOT, "Table '%s' is compatible with existing hive table", fullTableName));
+        } else {
+            return new CheckResult(false, format(Locale.ROOT,
+                    "Table '%s' is incompatible with existing hive table due to '%s'", fullTableName, issues));
+        }
+    }
+
+    private void checkAllColumnsInHiveTableDesc(TableDesc hiveTable, TableDesc newTable, List<String> issues) {
+        ColumnDesc[] hiveTableCols = hiveTable.getColumns();
+        ColumnDesc[] newTableCols = newTable.getColumns();
+
+        if (hiveTableCols.length < newTableCols.length) {
+            Set<String> colNamesNew = Lists.newArrayList(newTableCols).stream().map(input -> input.getName())
+                    .collect(Collectors.toSet());
+            Set<String> colNamesHive = Lists.newArrayList(hiveTableCols).stream().map(input -> input.getName())
+                    .collect(Collectors.toSet());
+            colNamesNew.removeAll(colNamesHive);
+            issues.add(format(Locale.ROOT, "columns %s are not existing in hive table", colNamesNew));
+            return;
+        }
+
+        Map<String, ColumnDesc> hiveColMap = Lists.newArrayList(hiveTableCols).stream()
+                .collect(Collectors.toMap(input -> input.getName().toUpperCase(Locale.ROOT), input -> input));
+        Map<String, ColumnDesc> newColMap = Lists.newArrayList(newTableCols).stream()
+                .collect(Collectors.toMap(input -> input.getName().toUpperCase(Locale.ROOT), input -> input));
+
+        List<String> violateColumns = Lists.newArrayList();
+        for (String colName : newColMap.keySet()) {
+            ColumnDesc hiveCol = hiveColMap.get(colName);
+            if (hiveCol == null) {
+                issues.add(format(Locale.ROOT, "column %s is not existing in hive table", colName));
+                continue;
+            }
+            if (!isColumnCompatible(hiveCol, newColMap.get(colName))) {
+                violateColumns.add(colName);
+            }
+        }
+        if (!violateColumns.isEmpty()) {
+            issues.add(format(Locale.ROOT, "Columns %s are incompatible " + "in hive", violateColumns));
+        }
+    }
 }
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java b/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
index 764a32a..0420ad8 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
@@ -125,6 +125,28 @@ public class TableService extends BasicService {
         TableSchemaUpdateChecker.CheckResult result = getSchemaUpdateChecker().allowReload(tableDesc, prj);
         result.raiseExceptionWhenInvalid();
     }
+
+    public void checkHiveTableCompatibility(String prj, TableDesc tableDesc) throws Exception {
+        Preconditions.checkNotNull(tableDesc.getDatabase());
+        Preconditions.checkNotNull(tableDesc.getName());
+
+        String database = tableDesc.getDatabase().toUpperCase(Locale.ROOT);
+        String tableName = tableDesc.getName().toUpperCase(Locale.ROOT);
+        ProjectInstance projectInstance = getProjectManager().getProject(prj);
+        ISourceMetadataExplorer explr = SourceManager.getSource(projectInstance).getSourceMetadataExplorer();
+
+        TableDesc hiveTableDesc;
+        try {
+            Pair<TableDesc, TableExtDesc> pair = explr.loadTableMetadata(database, tableName, prj);
+            hiveTableDesc = pair.getFirst();
+        } catch (Exception e) {
+            logger.error("Fail to get metadata for hive table {} due to ", tableDesc.getIdentity(), e);
+            throw new RuntimeException("Fail to get metadata for hive table" + tableDesc.getIdentity());
+        }
+
+        TableSchemaUpdateChecker.CheckResult result = getSchemaUpdateChecker().allowMigrate(tableDesc, hiveTableDesc);
+        result.raiseExceptionWhenInvalid();
+    }
     
     public List<TableDesc> getTableDescByProject(String project, boolean withExt) throws IOException {
         aclEvaluate.checkProjectReadPermission(project);
diff --git a/server/src/main/resources/kylinSecurity.xml b/server/src/main/resources/kylinSecurity.xml
index baf7172..912f700 100644
--- a/server/src/main/resources/kylinSecurity.xml
+++ b/server/src/main/resources/kylinSecurity.xml
@@ -237,6 +237,7 @@
 
             <scr:intercept-url pattern="/api/user/authentication*/**" access="permitAll"/>
             <scr:intercept-url pattern="/api/cubes/checkCompatibility" access="permitAll"/>
+            <scr:intercept-url pattern="/api/cubes/checkCompatibility/hiveTable" access="permitAll"/>
             <scr:intercept-url pattern="/api/query/runningQueries" access="hasRole('ROLE_ADMIN')"/>
             <scr:intercept-url pattern="/api/query/*/stop" access="hasRole('ROLE_ADMIN')"/>
             <scr:intercept-url pattern="/api/query*/**" access="isAuthenticated()"/>
@@ -290,6 +291,7 @@
 
             <scr:intercept-url pattern="/api/user/authentication*/**" access="permitAll"/>
             <scr:intercept-url pattern="/api/cubes/checkCompatibility" access="permitAll"/>
+            <scr:intercept-url pattern="/api/cubes/checkCompatibility/hiveTable" access="permitAll"/>
             <scr:intercept-url pattern="/api/query/runningQueries" access="hasRole('ROLE_ADMIN')"/>
             <scr:intercept-url pattern="/api/query/*/stop" access="hasRole('ROLE_ADMIN')"/>
             <scr:intercept-url pattern="/api/query*/**" access="isAuthenticated()"/>


[kylin] 11/15: KYLIN-4488 Frontend support for auto-migration to allow user to do cube migration by self service

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit 569552d974105b55547bc9941ffbe004f4e0486d
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Mon May 18 13:35:57 2020 +0800

    KYLIN-4488 Frontend support for auto-migration to allow user to do cube migration by self service
---
 .../kylin/rest/controller/MigrationController.java |  16 +++
 webapp/app/js/controllers/cubes.js                 | 157 +++++++++++++++++++++
 webapp/app/js/services/cubes.js                    |  17 ++-
 webapp/app/partials/cubes/cube_migrate.html        |  91 ++++++++++++
 webapp/app/partials/cubes/cubes.html               |   5 +
 5 files changed, 285 insertions(+), 1 deletion(-)

diff --git a/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java b/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java
index 9588e51..106d51f 100644
--- a/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java
+++ b/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java
@@ -45,6 +45,7 @@ import org.springframework.web.bind.annotation.PathVariable;
 import org.springframework.web.bind.annotation.RequestBody;
 import org.springframework.web.bind.annotation.RequestMapping;
 import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.RequestParam;
 import org.springframework.web.bind.annotation.ResponseBody;
 
 import com.fasterxml.jackson.core.JsonParseException;
@@ -82,6 +83,21 @@ public class MigrationController extends BasicController {
         return cube;
     }
 
+    @RequestMapping(value = "/{cubeName}/migrateRuleCheck", method = { RequestMethod.GET })
+    @ResponseBody
+    public String migrationRuleCheck(@PathVariable String cubeName, @RequestParam String projectName,
+            @RequestParam(value = "targetHost", required = false) String targetHost) {
+        CubeInstance cube = getCubeInstance(cubeName);
+        try {
+            MigrationRuleSet.Context ctx = new MigrationRuleSet.Context(queryService, cube, getTargetHost(targetHost),
+                    projectName);
+            return migrationService.checkRule(ctx);
+        } catch (Exception e) {
+            logger.error("Request migration failed.", e);
+            throw new BadRequestException(e.getMessage());
+        }
+    }
+    
     @RequestMapping(value = "/{cubeName}/migrateRequest", method = { RequestMethod.PUT })
     @ResponseBody
     public String requestMigration(@PathVariable String cubeName, @RequestBody MigrationRequest request) {
diff --git a/webapp/app/js/controllers/cubes.js b/webapp/app/js/controllers/cubes.js
index 2d9c032..ddc037e 100644
--- a/webapp/app/js/controllers/cubes.js
+++ b/webapp/app/js/controllers/cubes.js
@@ -581,6 +581,39 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
       });
     };
 
+    $scope.startMigrateCube = function(cube, action) {
+      $scope.loadDetail(cube).then(function () {
+        switch (action) {
+          case 0:
+            var template = 'cubeMigrate.html';
+            break;
+          case 1:
+            var template = 'migrateApprove.html';
+            break;
+          case -1:
+            var template = 'migrateReject.html';
+            break;
+          default:
+            var template = '';
+        }
+
+        if (!template) {
+          return;
+        }
+
+        $modal.open({
+          templateUrl: template,
+          controller: cubeMigrateCtrl,
+          windowClass:"clone-cube-window",
+          resolve: {
+            cube: function () {
+              return cube;
+            }
+          }
+        });
+      });
+    };
+
     $scope.listCubeAccess = function (cube) {
       //check project auth for user
       $scope.cubeProjectEntity = _.find($scope.projectModel.projects, function(project) {return project.name == $scope.projectModel.selectedProject;});
@@ -1080,3 +1113,127 @@ var lookupRefreshCtrl = function($scope, scope, CubeList, $modalInstance, CubeSe
   };
 
 };
+
+var cubeMigrateCtrl = function ($scope, $modalInstance, CubeService, cube, ProjectModel, loadingRequest, SweetAlert) {
+  $scope.migrate={
+    targetProject: ProjectModel.selectedProject,
+    cubeValidate: true,
+    lockProjectName: false
+  }
+
+  $scope.cancel = function () {
+    $modalInstance.dismiss('cancel');
+  };
+
+  $scope.validate = function () {
+    $scope.MigrationRequest = {
+      projectName:$scope.migrate.targetProject
+    }
+    loadingRequest.show();
+    CubeService.ruleCheck({cubeId: cube.name, projectName: $scope.migrate.targetProject}, function(result) {
+      loadingRequest.hide();
+      $scope.migrate.cubeValidate = false;
+      $scope.migrate.lockProjectName = true;
+      if (result.length > 0) {
+        SweetAlert.swal('Attention', result, 'warning');
+      }
+    }, function (e) {
+      loadingRequest.hide();
+      if (e.data && e.data.exception) {
+        var message = e.data.exception;
+        var msg = !!(message) ? message : 'Failed to take action.';
+        SweetAlert.swal('Oops...', msg, 'error');
+      } else {
+        SweetAlert.swal('Oops...', 'Failed to take action.', 'error');
+      }
+    });
+  }
+
+  $scope.migrateCube = function(){
+    if(!$scope.migrate.targetProject){
+      SweetAlert.swal('Oops...', "Please input target project name.", 'info');
+      return;
+    }
+
+    $scope.MigrationRequest = {
+      projectName:$scope.migrate.targetProject
+    }
+
+    SweetAlert.swal({
+      title: '',
+      text: 'Are you sure to migrate the cube? ',
+      type: '',
+      showCancelButton: true,
+      confirmButtonColor: '#DD6B55',
+      confirmButtonText: "Yes",
+      closeOnConfirm: true
+    }, function (isConfirm) {
+      if (isConfirm) {
+        loadingRequest.show();
+        CubeService.migrate({cubeId: cube.name}, $scope.MigrationRequest, function (result) {
+          loadingRequest.hide();
+          $modalInstance.dismiss('cancel');
+          SweetAlert.swal('Success!', 'Your Migration Request has been well received. Please check your email to get timely update of this request.', 'success');
+        }, function (e) {
+          loadingRequest.hide();
+          if (e.data && e.data.exception) {
+            var msg = e.data.exception;
+            if(e.status == '400' && (e.data.exception.indexOf("QueryLatencyRule") > -1 || e.data.exception.indexOf("ExpansionRateRule") > -1)){
+              msg += '. Please refer to the guidance to optimize your cube design';
+            }
+            msg += '. For any question, please contact support team.';
+            SweetAlert.swal('Oops...', msg, 'error');
+          } else {
+            SweetAlert.swal('Oops...', 'Failed to take action.', 'error');
+          }
+        });
+      }
+    });
+  };
+
+  $scope.migrateApprove = function(){
+    $scope.MigrationRequest = {
+      projectName:$scope.migrate.targetProject
+    }
+    loadingRequest.show();
+    CubeService.approve({cubeId: cube.name}, $scope.MigrationRequest, function (result) {
+      loadingRequest.hide();
+      $modalInstance.dismiss('cancel');
+      SweetAlert.swal('Success!', 'Approve cube migration successfully', 'success');
+    }, function (e) {
+      loadingRequest.hide();
+      if (e.data && e.data.exception) {
+        var message = e.data.exception;
+        var msg = !!(message) ? message : 'Failed to take action.';
+        SweetAlert.swal('Oops...', msg, 'error');
+      } else {
+        SweetAlert.swal('Oops...', 'Failed to take action.', 'error');
+      }
+    });
+  };
+
+  $scope.migrateReject = function(){
+    if(!$scope.migrate.reason) {
+      SweetAlert.swal('Oops...', 'Please enter reason for refusal.', 'info');
+      return;
+    }
+    $scope.MigrationRequest = {
+      reason: $scope.migrate.reason
+    };
+    loadingRequest.show();
+    CubeService.reject({cubeId: cube.name}, $scope.MigrationRequest, function (result) {
+      loadingRequest.hide();
+      $modalInstance.dismiss('cancel');
+      SweetAlert.swal('Success!', 'Reject cube migration successfully', 'success');
+    }, function (e) {
+      loadingRequest.hide();
+      if (e.data && e.data.exception) {
+        var message = e.data.exception;
+        var msg = !!(message) ? message : 'Failed to take action.';
+        SweetAlert.swal('Oops...', msg, 'error');
+      } else {
+        SweetAlert.swal('Oops...', 'Failed to take action.', 'error');
+      }
+    });
+  }
+};
diff --git a/webapp/app/js/services/cubes.js b/webapp/app/js/services/cubes.js
index 65a4d5d..522d043 100644
--- a/webapp/app/js/services/cubes.js
+++ b/webapp/app/js/services/cubes.js
@@ -79,7 +79,22 @@ KylinApp.factory('CubeService', ['$resource', function ($resource, config) {
       }
     },
     optimize: {method: 'PUT', params: {action: 'optimize'}, isArray: false},
+    ruleCheck: {
+      method: 'GET',
+      params: {
+        action: 'migrateRuleCheck'
+      },
+      isArray: false,
+      interceptor: {
+        response: function(response) {
+          return response.data;
+        }
+      }
+    },
     lookupRefresh: {method: 'PUT', params: {action: 'refresh_lookup'}, isArray: false},
-    checkDuplicateCubeName: {method: 'GET', params: {action: 'validate'}, isArray: false}
+    checkDuplicateCubeName: {method: 'GET', params: {action: 'validate'}, isArray: false},
+    migrate: {method: 'PUT', params: {action: 'migrateRequest'}, isArray: false},
+    approve: {method: 'PUT', params: {action: 'migrateApprove'}, isArray: false},
+    reject: {method: 'PUT', params: {action: 'migrateReject'}, isArray: false}
   });
 }]);
diff --git a/webapp/app/partials/cubes/cube_migrate.html b/webapp/app/partials/cubes/cube_migrate.html
new file mode 100644
index 0000000..51b0900
--- /dev/null
+++ b/webapp/app/partials/cubes/cube_migrate.html
@@ -0,0 +1,91 @@
+
+<!--
+* 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.
+-->
+<script type="text/ng-template" id="cubeMigrate.html">
+  <div class="modal-header">
+    <h4 tooltip="submit">MIGRATE CUBE</h4>
+  </div>
+  <div class="modal-body" style="background-color: white">
+    <div class="row">
+      <div class="col-md-8 col-md-offset-2">
+        <div class="form-group">
+          <b>Target Project Name:</b>
+          <br/>
+          <p style="color:red;font-size:12px;">You can change the below project name if it's different on Kylin Prod.</p>
+          <input type="text" class="form-control" ng-readonly="migrate.lockProjectName" ng-model="migrate.targetProject" />
+        </div>
+      </div>
+    </div>
+    <div class="row">
+      <div class="col-md-8 col-md-offset-2">
+        <div class="form-group">
+          <div ng-show="migrate.cubeValidate" >
+            <b>Cube Validation:</b>
+            <br/>
+            <button style="height:40px;width:150px;" class="btn btn-info" ng-click="validate()">Validate</button>
+          </div>
+        </div>
+      </div>
+    </div>
+  </div>
+  <div class="modal-footer">
+    <button class="btn btn-success"  ng-disabled="migrate.cubeValidate" ng-click="migrateCube()">Submit</button>
+    <button class="btn btn-primary" ng-click="cancel()">Close</button>
+  </div>
+</script>
+
+<script type="text/ng-template" id="migrateApprove.html">
+  <div class="modal-header">
+    <h4 tooltip="submit">MIGRATE APPROVE</h4>
+  </div>
+  <div class="modal-body" style="background-color: white">
+    <div class="row">
+      <div class="col-md-8 col-md-offset-2">
+        <div class="form-group">
+          <b>Target Project Name:</b>
+          <br/>
+          <input type="text" class="form-control" ng-model="migrate.targetProject" value={{migrate.targetProject}}/>
+        </div>
+      </div>
+    </div>
+  <div class="modal-footer">
+    <button class="btn btn-success" ng-click="migrateApprove()">Approve</button>
+    <button class="btn btn-primary" ng-click="cancel()">Close</button>
+  </div>
+</script>
+
+<script type="text/ng-template" id="migrateReject.html">
+  <div class="modal-header">
+    <h4 tooltip="submit">MIGRATE REJECT</h4>
+  </div>
+  <div class="modal-body" style="background-color: white">
+    <div class="row">
+      <div class="col-md-8 col-md-offset-2">
+        <div class="form-group">
+          <b>Please enter reject reason:</b>
+          <br/>
+          <textarea ng-model="migrate.reason" placeholder="Reject Reason..." rows="5" cols="30" class="form-control" style="overflow:scroll"></textarea>
+        </div>
+      </div>
+    </div>
+  </div>
+  <div class="modal-footer">
+    <button class="btn btn-success" ng-click="migrateReject()">Reject</button>
+    <button class="btn btn-primary" ng-click="cancel()">Close</button>
+  </div>
+</script>
diff --git a/webapp/app/partials/cubes/cubes.html b/webapp/app/partials/cubes/cubes.html
index 65ebc87..dba3c8b 100644
--- a/webapp/app/partials/cubes/cubes.html
+++ b/webapp/app/partials/cubes/cubes.html
@@ -102,6 +102,7 @@
                         <li ng-if="cube.status=='DISABLED' && (userService.hasRole('ROLE_ADMIN') || hasPermission('cube',cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask))"><a ng-click="startDeleteSegment(cube)">Delete Segment</a></li>
                         <li ng-if="cube.status=='DISABLED' && (userService.hasRole('ROLE_ADMIN') || hasPermission('cube',cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask))"><a ng-click="purge(cube)">Purge</a></li>
                         <li ng-if="cube.status!='DESCBROKEN' && (userService.hasRole('ROLE_ADMIN') || hasPermission('cube',cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask))"><a ng-click="cloneCube(cube)">Clone</a></li>
+                        <li ng-if="cube.status=='READY' && (userService.hasRole('ROLE_ADMIN') || hasPermission(cube, permissions.ADMINISTRATION.mask) || hasPermission(cubeProjectEntity, permissions.ADMINISTRATION.mask))"><a ng-click="startMigrateCube(cube, 0);">Migrate</a></li>
 
                     </ul>
                     <ul ng-if="(userService.hasRole('ROLE_ADMIN') || hasPermission('cube', cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask, permissions.OPERATION.mask)) && cube.streamingV2 && actionLoaded" class="dropdown-menu" role="menu" style="right:0;left:auto;">
@@ -116,6 +117,7 @@
                         <li ng-if="cube.status=='DISABLED'"><a ng-click="enable(cube, $index)">Enable</a></li>
                         <li ng-if="cube.status=='DISABLED'"><a ng-click="purge(cube, $index)">Purge</a></li>
                         <li><a ng-click="cloneCube(cube)">Clone</a></li>
+                        <li ng-if="cube.status=='READY' && (userService.hasRole('ROLE_ADMIN') || hasPermission(cube, permissions.ADMINISTRATION.mask))"><a ng-click="startMigrateCube(cube, 0);">Migrate</a></li>
                     </ul>
                     <span ng-if="!(userService.hasRole('ROLE_ADMIN') || hasPermission('cube',cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask, permissions.OPERATION.mask))" class="dropdown-menu" role="menu">
                       N/A
@@ -130,6 +132,8 @@
                     <ul class="dropdown-menu" role="menu" style="right:0;left:auto;">
                         <li ng-if="cube.status!='READY'"><a href="cubes/edit/{{cube.name}}/descriptionjson">Edit CubeDesc</a></li>
                         <li><a href="cubes/view/{{cube.name}}/instancejson">View Cube</a></li>
+                        <li ng-if="cube.status=='READY' && kylinConfig.getDeployEnv().indexOf('QA') > -1"><a ng-click="startMigrateCube(cube, 1)">Approve Migration</a></li>
+                        <li ng-if="cube.status=='READY' && kylinConfig.getDeployEnv().indexOf('QA') > -1"><a ng-click="startMigrateCube(cube, -1)">Reject Migration</a></li>
                     </ul>
                 </div>
             </td>
@@ -160,6 +164,7 @@
 <div ng-include="'partials/models/model_detail.html'"></div>
 <div ng-include="'partials/cubes/cube_clone.html'"></div>
 <div ng-include="'partials/cubes/cube_delete_segment.html'"></div>
+<div ng-include="'partials/cubes/cube_migrate.html'"></div>
 <div ng-include="'partials/jobs/lookup_refresh.html'"></div>
 <div ng-include="'partials/streaming/cubeAssignment.html'"></div>
 </div>


[kylin] 01/15: KYLIN-4420 Add model compatibility check to allow more compatible update

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit 6aeaf2335adc97138075d177f9a7dd50536bae4e
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Thu Apr 9 17:22:20 2020 +0800

    KYLIN-4420 Add model compatibility check to allow more compatible update
---
 .../org/apache/kylin/common/KylinConfigBase.java   |   4 +
 .../org/apache/kylin/common/util/CheckUtil.java    |  14 ++
 .../apache/kylin/metadata/model/PartitionDesc.java |  72 ++++++
 .../rest/service/ModelSchemaUpdateChecker.java     | 252 +++++++++++++++++++++
 .../apache/kylin/rest/service/ModelService.java    |  17 ++
 webapp/app/js/controllers/models.js                |  10 +-
 6 files changed, 362 insertions(+), 7 deletions(-)

diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 9e17de6..99123ec 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -503,6 +503,10 @@ public abstract class KylinConfigBase implements Serializable {
         return getOptional("kylin.metadata.hbase-client-retries-number", "1");
     }
 
+    public boolean isModelSchemaUpdaterCheckerEnabled() {
+        return Boolean.parseBoolean(getOptional("kylin.metadata.model-schema-updater-checker-enabled", "false"));
+    }
+
     // ============================================================================
     // DICTIONARY & SNAPSHOT
     // ============================================================================
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/CheckUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/CheckUtil.java
index f727566..9b813e2 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/CheckUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/CheckUtil.java
@@ -70,4 +70,18 @@ public class CheckUtil {
 
         return false;
     }
+
+    public static boolean equals(String s1, String s2) {
+        if (s1 != null && s2 != null) {
+            return s1.trim().equalsIgnoreCase(s2.trim());
+        }
+        return s1 == null && s2 == null;
+    }
+
+    public static <T> boolean equals(T o1, T o2) {
+        if (o1 != null && o2 != null) {
+            return o1.equals(o2);
+        }
+        return o1 == null && o2 == null;
+    }
 }
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 56b3ec1..f1c22fb 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
@@ -24,6 +24,7 @@ import java.util.function.Function;
 
 import org.apache.kylin.shaded.com.google.common.base.Preconditions;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.kylin.common.util.CheckUtil;
 import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.metadata.datatype.DataType;
@@ -189,6 +190,77 @@ public class PartitionDesc implements Serializable {
         return partitionTimeColumnRef;
     }
 
+    public boolean equalsRaw(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        PartitionDesc other = (PartitionDesc) obj;
+
+        if (!this.partitionType.equals(other.getCubePartitionType()))
+            return false;
+        if (!this.partitionConditionBuilderClz.equals(other.partitionConditionBuilderClz))
+            return false;
+        if (!CheckUtil.equals(this.partitionDateColumn, other.getPartitionDateColumn()))
+            return false;
+        if (!CheckUtil.equals(this.partitionDateFormat, other.getPartitionDateFormat()))
+            return false;
+        if (!CheckUtil.equals(this.partitionTimeColumn, other.getPartitionTimeColumn()))
+            return false;
+        if (!CheckUtil.equals(this.partitionTimeFormat, other.getPartitionTimeFormat()))
+            return false;
+        return true;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        PartitionDesc other = (PartitionDesc) obj;
+
+        if (!this.partitionType.equals(other.getCubePartitionType()))
+            return false;
+        if (!CheckUtil.equals(this.partitionDateColumn, other.getPartitionDateColumn()))
+            return false;
+        if (!CheckUtil.equals(this.partitionDateFormat, other.getPartitionDateFormat()))
+            return false;
+        if (this.partitionDateColumn != null) {
+            if (!this.partitionConditionBuilder.getClass().equals(other.getPartitionConditionBuilder().getClass()))
+                return false;
+            if (this.partitionConditionBuilder instanceof DefaultPartitionConditionBuilder) {
+                if (!CheckUtil.equals(this.partitionTimeColumn, other.getPartitionTimeColumn())) {
+                    return false;
+                }
+                if (!CheckUtil.equals(this.partitionTimeFormat, other.getPartitionTimeFormat())) {
+                    return false;
+                }
+            }
+        }
+
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + partitionType.hashCode();
+        result = prime * result + partitionConditionBuilderClz.hashCode();
+        result = prime * result + ((partitionDateColumn == null) ? 0 : partitionDateColumn.hashCode());
+        result = prime * result + ((partitionDateFormat == null) ? 0 : partitionDateFormat.hashCode());
+        if (partitionConditionBuilder instanceof DefaultPartitionConditionBuilder) {
+            result = prime * result + ((partitionTimeColumn == null) ? 0 : partitionTimeColumn.hashCode());
+            result = prime * result + ((partitionTimeFormat == null) ? 0 : partitionTimeFormat.hashCode());
+        }
+        return result;
+    }
+
     // ============================================================================
 
     public static interface IPartitionConditionBuilder {
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/ModelSchemaUpdateChecker.java b/server-base/src/main/java/org/apache/kylin/rest/service/ModelSchemaUpdateChecker.java
new file mode 100644
index 0000000..e8c04fd
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/ModelSchemaUpdateChecker.java
@@ -0,0 +1,252 @@
+/*
+ * 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.rest.service;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static java.lang.String.format;
+import static org.apache.kylin.measure.topn.TopNMeasureType.FUNC_TOP_N;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kylin.common.util.CheckUtil;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.metadata.TableMetadataManager;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.DataModelManager;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.JoinTableDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.ModelDimensionDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+public class ModelSchemaUpdateChecker {
+
+    private final TableMetadataManager metadataManager;
+    private final CubeManager cubeManager;
+    private final DataModelManager dataModelManager;
+
+    static class CheckResult {
+        private final boolean valid;
+        private final String reason;
+
+        private CheckResult(boolean valid, String reason) {
+            this.valid = valid;
+            this.reason = reason;
+        }
+
+        void raiseExceptionWhenInvalid() {
+            if (!valid) {
+                throw new RuntimeException(reason);
+            }
+        }
+
+        static CheckResult validOnFirstCreate(String modelName) {
+            return new CheckResult(true, format(Locale.ROOT, "Model '%s' hasn't been created before", modelName));
+        }
+
+        static CheckResult validOnCompatibleSchema(String modelName) {
+            return new CheckResult(true,
+                    format(Locale.ROOT, "Table '%s' is compatible with all existing cubes", modelName));
+        }
+
+        static CheckResult invalidOnIncompatibleSchema(String modelName, List<String> reasons) {
+            StringBuilder buf = new StringBuilder();
+            for (String reason : reasons) {
+                buf.append("- ").append(reason).append("\n");
+            }
+
+            return new CheckResult(false,
+                    format(Locale.ROOT,
+                            "Found %d issue(s) with '%s':%n%s Please disable and purge related cube(s) first",
+                            reasons.size(), modelName, buf.toString()));
+        }
+    }
+
+    ModelSchemaUpdateChecker(TableMetadataManager metadataManager, CubeManager cubeManager,
+            DataModelManager dataModelManager) {
+        this.metadataManager = checkNotNull(metadataManager, "metadataManager is null");
+        this.cubeManager = checkNotNull(cubeManager, "cubeManager is null");
+        this.dataModelManager = checkNotNull(dataModelManager, "dataModelManager is null");
+    }
+
+    private List<CubeInstance> findCubeByModel(final String modelName) {
+        Iterable<CubeInstance> relatedCubes = Iterables.filter(cubeManager.listAllCubes(), cube -> {
+            if (cube == null || cube.allowBrokenDescriptor()) {
+                return false;
+            }
+            DataModelDesc model = cube.getModel();
+            if (model == null)
+                return false;
+            return model.getName().equals(modelName);
+        });
+
+        return ImmutableList.copyOf(relatedCubes);
+    }
+
+    /**
+     * Model compatible rule includes:
+     * 1. the same fact table
+     * 2. the same lookup table
+     * 3. the same joins
+     * 4. the same partition
+     * 5. the same filter
+     */
+    private static void checkDataModelCompatible(DataModelDesc existing, DataModelDesc newModel, List<String> issues) {
+        // Check fact table
+        if (!existing.getRootFactTableName().equalsIgnoreCase(newModel.getRootFactTableName())) {
+            issues.add(format(Locale.ROOT,
+                    "The fact table %s used in existing model is not the same as the updated one %s",
+                    existing.getRootFactTableName(), newModel.getRootFactTableName()));
+        }
+        // Check join table
+        Map<String, JoinTableDesc> existingLookupMap = Maps.newHashMap();
+        for (JoinTableDesc joinTableDesc : existing.getJoinTables()) {
+            existingLookupMap.put(joinTableDesc.getAlias(), joinTableDesc);
+        }
+        for (JoinTableDesc joinTableDesc : newModel.getJoinTables()) {
+            if (existingLookupMap.get(joinTableDesc.getAlias()) == null) {
+                issues.add(format(Locale.ROOT, "The join table %s does not existing in existing model",
+                        joinTableDesc.getTable()));
+                continue;
+            }
+            JoinTableDesc existingLookup = existingLookupMap.remove(joinTableDesc.getAlias());
+            if (!existingLookup.getTable().equals(joinTableDesc.getTable())) {
+                issues.add(format(Locale.ROOT,
+                        "The join table %s used in existing model is not the same as the updated one %s",
+                        existingLookup.getTable(), joinTableDesc.getTable()));
+                continue;
+            }
+            if (!existingLookup.getKind().equals(joinTableDesc.getKind())) {
+                issues.add(format(Locale.ROOT,
+                        "The TableKind %s in existing model is not the same as the updated one %s for table %s",
+                        existingLookup.getKind(), joinTableDesc.getKind(), existingLookup.getTable()));
+                continue;
+            }
+            if (!existingLookup.getJoin().equals(joinTableDesc.getJoin())) {
+                issues.add(format(Locale.ROOT, "The join %s is not the same as the existing one %s",
+                        joinTableDesc.getJoin(), existingLookup.getJoin()));
+            }
+        }
+        if (existingLookupMap.size() > 0) {
+            issues.add(format(Locale.ROOT, "Missing lookup tables %s", existingLookupMap.keySet()));
+        }
+        // Check partition column
+        if (!CheckUtil.equals(existing.getPartitionDesc(), newModel.getPartitionDesc())) {
+            issues.add(format(Locale.ROOT, "The partition desc %s is not the same as the existing one %s",
+                    newModel.getPartitionDesc(), existing.getPartitionDesc()));
+        }
+        // Check filter
+        if (!CheckUtil.equals(existing.getFilterCondition(), newModel.getFilterCondition())) {
+            issues.add(format(Locale.ROOT, "The filter %s is not the same as the existing one %s",
+                    newModel.getFilterCondition(), existing.getFilterCondition()));
+        }
+    }
+
+    public CheckResult allowEdit(DataModelDesc modelDesc, String prj) {
+
+        final String modelName = modelDesc.getName();
+        // No model
+        DataModelDesc existing = dataModelManager.getDataModelDesc(modelName);
+        if (existing == null) {
+            return CheckResult.validOnFirstCreate(modelName);
+        }
+        modelDesc.init(metadataManager.getConfig(), metadataManager.getAllTablesMap(prj));
+
+        // No cube
+        List<CubeInstance> cubes = findCubeByModel(modelName);
+        if (cubes.size() <= 0) {
+            return CheckResult.validOnCompatibleSchema(modelName);
+        }
+
+        existing = cubes.get(0).getModel();
+        List<String> issues = Lists.newArrayList();
+        // Check model related
+        checkDataModelCompatible(existing, modelDesc, issues);
+        if (!issues.isEmpty()) {
+            return CheckResult.invalidOnIncompatibleSchema(modelName, issues);
+        }
+
+        // Check cube related
+        Set<String> dimensionColumns = Sets.newHashSet();
+        for (ModelDimensionDesc modelDimensionDesc : modelDesc.getDimensions()) {
+            for (String columnName : modelDimensionDesc.getColumns()) {
+                dimensionColumns.add(modelDimensionDesc.getTable() + "." + columnName);
+            }
+        }
+        // Add key related columns
+        for (JoinTableDesc joinTableDesc : modelDesc.getJoinTables()) {
+            List<TblColRef> keyCols = Lists.newArrayList(joinTableDesc.getJoin().getForeignKeyColumns());
+            keyCols.addAll(Lists.newArrayList(joinTableDesc.getJoin().getPrimaryKeyColumns()));
+            dimensionColumns.addAll(Lists.transform(keyCols, entry -> entry.getIdentity()));
+        }
+        Set<String> measureColumns = Sets.newHashSet(modelDesc.getMetrics());
+        for (CubeInstance cube : cubes) {
+            // Check dimensions
+            List<String> cubeDimensionColumns = Lists.newLinkedList();
+            for (TblColRef entry : cube.getAllDimensions()) {
+                cubeDimensionColumns.add(entry.getIdentity());
+            }
+            for (MeasureDesc input : cube.getMeasures()) {
+                FunctionDesc funcDesc = input.getFunction();
+                if (FUNC_TOP_N.equalsIgnoreCase(funcDesc.getExpression())) {
+                    List<TblColRef> ret = funcDesc.getParameter().getColRefs();
+                    cubeDimensionColumns
+                            .addAll(Lists.transform(ret.subList(1, ret.size()), entry -> entry.getIdentity()));
+                }
+            }
+            if (!dimensionColumns.containsAll(cubeDimensionColumns)) {
+                cubeDimensionColumns.removeAll(dimensionColumns);
+                issues.add(format(Locale.ROOT, "Missing some dimension columns %s for cube %s", cubeDimensionColumns,
+                        cube.getName()));
+            }
+            // Check measures
+            List<List<TblColRef>> cubeMeasureTblColRefLists = Lists.transform(cube.getMeasures(), entry -> {
+                FunctionDesc funcDesc = entry.getFunction();
+                List<TblColRef> ret = funcDesc.getParameter().getColRefs();
+                if (FUNC_TOP_N.equalsIgnoreCase(funcDesc.getExpression())) {
+                    return Lists.newArrayList(ret.get(0));
+                } else {
+                    return funcDesc.getParameter().getColRefs();
+                }
+            });
+            List<String> cubeMeasureColumns = Lists.transform(
+                    Lists.newArrayList(Iterables.concat(cubeMeasureTblColRefLists)), entry -> entry.getIdentity());
+            if (!measureColumns.containsAll(cubeMeasureColumns)) {
+                cubeMeasureColumns.removeAll(measureColumns);
+                issues.add(format(Locale.ROOT, "Missing some measure columns %s for cube %s", cubeMeasureColumns,
+                        cube.getName()));
+            }
+        }
+
+        if (issues.isEmpty()) {
+            return CheckResult.validOnCompatibleSchema(modelName);
+        }
+        return CheckResult.invalidOnIncompatibleSchema(modelName, issues);
+    }
+}
\ No newline at end of file
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
index 2bb803a..d1c583f 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
@@ -39,6 +39,7 @@ import org.apache.kylin.metadata.model.JoinsTree;
 import org.apache.kylin.metadata.model.ModelDimensionDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.rest.exception.BadRequestException;
 import org.apache.kylin.rest.exception.ForbiddenException;
 import org.apache.kylin.rest.msg.Message;
@@ -147,10 +148,26 @@ public class ModelService extends BasicService {
     public DataModelDesc updateModelAndDesc(String project, DataModelDesc desc) throws IOException {
         aclEvaluate.checkProjectWritePermission(project);
         validateModel(project, desc);
+        checkModelCompatible(project, desc);
         getDataModelManager().updateDataModelDesc(desc);
         return desc;
     }
 
+    public void checkModelCompatible(String project, DataModelDesc dataModalDesc) {
+        ProjectInstance prjInstance = getProjectManager().getProject(project);
+        if (prjInstance == null) {
+            throw new BadRequestException("Project " + project + " does not exist");
+        }
+        if (!prjInstance.getConfig().isModelSchemaUpdaterCheckerEnabled()) {
+            logger.info("Skip the check for model schema update");
+            return;
+        }
+        ModelSchemaUpdateChecker checker = new ModelSchemaUpdateChecker(getTableManager(), getCubeManager(),
+                getDataModelManager());
+        ModelSchemaUpdateChecker.CheckResult result = checker.allowEdit(dataModalDesc, project);
+        result.raiseExceptionWhenInvalid();
+    }
+
     public void validateModel(String project, DataModelDesc desc) throws IllegalArgumentException {
         String factTableName = desc.getRootFactTableName();
         TableDesc tableDesc = getTableManager().getTableDesc(factTableName, project);
diff --git a/webapp/app/js/controllers/models.js b/webapp/app/js/controllers/models.js
index 84e99e6..274bbaa 100644
--- a/webapp/app/js/controllers/models.js
+++ b/webapp/app/js/controllers/models.js
@@ -145,14 +145,10 @@ KylinApp.controller('ModelsCtrl', function ($scope, $q, $routeParams, $location,
         })
       }
 
-      if (modelstate==false){
-    	  if (isEditJson) {
-    		  $location.path("/models/edit/" + model.name + "/descriptionjson");
-    	  } else {
-    		  $location.path("/models/edit/" + model.name);
-    	  }
+      if (isEditJson) {
+        $location.path("/models/edit/" + model.name + "/descriptionjson");
       } else {
-        SweetAlert.swal('Sorry','This model is still used by '+ cubename.join(','));
+        $location.path("/models/edit/" + model.name);
       }
     })
 


[kylin] 06/15: KYLIN-4484 Relax checkValidationInModel() in TableSchemaUpdateChecker

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit f13771e058e947b3e551323c9de1ae4ce0eff9e1
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Sat May 9 15:23:32 2020 +0800

    KYLIN-4484 Relax checkValidationInModel() in TableSchemaUpdateChecker
---
 .../rest/service/TableSchemaUpdateChecker.java     | 49 ++++++++++++++--------
 1 file changed, 32 insertions(+), 17 deletions(-)

diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java b/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java
index c0acff4..1b205be 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java
@@ -24,6 +24,7 @@ import static java.lang.String.format;
 import java.util.List;
 import java.util.Locale;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import javax.annotation.Nullable;
 
@@ -243,37 +244,51 @@ public class TableSchemaUpdateChecker {
         }
     }
 
-    private void checkValidationInModel(TableDesc newTableDesc, List<String> issues, DataModelDesc usedModel){
+    private void checkValidationInModel(TableDesc newTableDesc, List<String> issues, DataModelDesc usedModel) {
+        List<String> violateColumns = Lists.newArrayList();
+
         final String fullTableName = newTableDesc.getIdentity();
-        // if user reloads a fact table used by model, then all used columns must match current schema
         if (usedModel.isFactTable(fullTableName)) {
             TableDesc factTable = usedModel.findFirstTable(fullTableName).getTableDesc();
-            List<String> violateColumns = checkAllColumnsInFactTable(usedModel, factTable, newTableDesc);
-            if (!violateColumns.isEmpty()) {
-                issues.add(format(Locale.ROOT, "Column %s used in model[%s], but changed " + "in hive",
-                        violateColumns, usedModel.getName()));
-            }
+            violateColumns.addAll(checkAllColumnsInFactTable(usedModel, factTable, newTableDesc));
         }
 
-        // 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)
         if (usedModel.isLookupTable(fullTableName)) {
-            TableDesc lookupTable = usedModel.findFirstTable(fullTableName).getTableDesc();
-            if (!checkAllColumnsInTableDesc(lookupTable, newTableDesc)) {
-                issues.add(format(Locale.ROOT, "Table '%s' is used as Lookup Table in model[%s], but "
-                                + "changed in " + "hive, only append operation are supported on hive table as lookup table",
-                        lookupTable.getIdentity(), usedModel.getName()));
-            }
+            violateColumns.addAll(checkAllColumnsInLookupTable(usedModel, newTableDesc));
+        }
+
+        if (!violateColumns.isEmpty()) {
+            issues.add(format(Locale.ROOT, "Column %s used in model[%s], but not exist " + "in hive", violateColumns,
+                    usedModel.getName()));
         }
     }
 
-    private List<String> checkAllColumnsInFactTable(DataModelDesc usedModel, TableDesc factTable, TableDesc newTableDesc) {
+    // columns in usedModel should not be deleted in new table 
+    private List<String> checkAllColumnsInLookupTable(DataModelDesc usedModel, TableDesc newTableDesc) {
+        List<String> violateColumns = Lists.newArrayList();
+
+        Set<String> newColumns = Sets.newHashSet(newTableDesc.getColumns()).stream()
+                .map(c -> c.getName().toUpperCase(Locale.ROOT)).collect(Collectors.toSet());
+        for (ModelDimensionDesc dim : usedModel.getDimensions()) {
+            if (dim.getTable().equalsIgnoreCase(newTableDesc.getIdentity())) {
+                Set<String> oldColumns = Sets.newHashSet(dim.getColumns()).stream().map(c -> c.toUpperCase(Locale.ROOT))
+                        .collect(Collectors.toSet());
+                oldColumns.removeAll(newColumns);
+                violateColumns.addAll(oldColumns);
+            }
+        }
+
+        return violateColumns;
+    }
+    
+    private List<String> checkAllColumnsInFactTable(DataModelDesc usedModel, TableDesc factTable,
+            TableDesc newTableDesc) {
         List<String> violateColumns = Lists.newArrayList();
 
         for (ColumnDesc column : findUsedColumnsInFactTable(usedModel, factTable)) {
             if (!column.isComputedColumn()) {
                 ColumnDesc newCol = newTableDesc.findColumnByName(column.getName());
-                if (newCol == null || !isColumnCompatible(column, newCol)) {
+                if (newCol == null) {
                     violateColumns.add(column.getName());
                 }
             }


[kylin] 02/15: KYLIN-4421 Add table alias field for root fact table in DataModelDesc

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit c9c723fe7164983476bcd39381ae216ea44f7867
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Thu Apr 9 18:49:28 2020 +0800

    KYLIN-4421 Add table alias field for root fact table in DataModelDesc
---
 .../java/org/apache/kylin/metadata/model/DataModelDesc.java | 13 ++++++++++---
 1 file changed, 10 insertions(+), 3 deletions(-)

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 71f380a..e117176 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
@@ -31,6 +31,7 @@ import java.util.Objects;
 import java.util.Queue;
 import java.util.Set;
 
+import com.google.common.base.Strings;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
@@ -82,6 +83,9 @@ public class DataModelDesc extends RootPersistentEntity {
     @JsonProperty("fact_table")
     private String rootFactTable;
 
+    @JsonProperty("fact_table_alias")
+    private String rootFactTableAlias;
+
     @JsonProperty("lookups")
     @JsonInclude(JsonInclude.Include.NON_NULL)
     private JoinTableDesc[] joinTables;
@@ -403,7 +407,10 @@ public class DataModelDesc extends RootPersistentEntity {
             throw new IllegalStateException("Root fact table does not exist:" + rootFactTable);
 
         TableDesc rootDesc = tables.get(rootFactTable);
-        rootFactTableRef = new TableRef(this, rootDesc.getName(), rootDesc, false);
+        if (Strings.isNullOrEmpty(rootFactTableAlias)) {
+            rootFactTableAlias = rootDesc.getName();
+        }
+        rootFactTableRef = new TableRef(this, rootFactTableAlias, rootDesc, false);
 
         addAlias(rootFactTableRef);
         factTableRefs.add(rootFactTableRef);
@@ -601,8 +608,7 @@ public class DataModelDesc extends RootPersistentEntity {
         int orderedIndex = 0;
 
         Queue<JoinTableDesc> joinTableBuff = new ArrayDeque<JoinTableDesc>();
-        TableDesc rootDesc = tables.get(rootFactTable);
-        joinTableBuff.addAll(fkMap.get(rootDesc.getName()));
+        joinTableBuff.addAll(fkMap.get(rootFactTableAlias));
         while (!joinTableBuff.isEmpty()) {
             JoinTableDesc head = joinTableBuff.poll();
             orderedJoinTables[orderedIndex++] = head;
@@ -795,6 +801,7 @@ public class DataModelDesc extends RootPersistentEntity {
         copy.owner = orig.owner;
         copy.description = orig.description;
         copy.rootFactTable = orig.rootFactTable;
+        copy.rootFactTableAlias = orig.rootFactTableAlias;
         copy.joinTables = orig.joinTables;
         copy.dimensions = orig.dimensions;
         copy.metrics = orig.metrics;


[kylin] 05/15: KYLIN-4422 Allow to change data type from varchar to datetime

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit 6bd5b43bb06b6bf19d1a096d99146396aab8d5b2
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Mon Apr 13 16:35:07 2020 +0800

    KYLIN-4422 Allow to change data type from varchar to datetime
---
 .../org/apache/kylin/common/KylinConfigBase.java   |   8 ++
 .../kylin/cube/gridtable/CubeCodeSystem.java       |  21 +++++
 .../org/apache/kylin/cube/model/RowKeyColDesc.java |  12 ++-
 .../org/apache/kylin/cube/model/RowKeyDesc.java    |   4 +
 .../org/apache/kylin/dict/DictionaryGenerator.java | 104 ++-------------------
 .../kylin/dict/lookup/LookupStringTable.java       |   8 +-
 .../apache/kylin/dict/DictionaryProviderTest.java  |  26 +-----
 .../org/apache/kylin/metadata/tuple/Tuple.java     |  19 ++--
 .../org/apache/kylin/metadata/tuple/TupleInfo.java |   4 +
 .../kylin/storage/gtrecord/CubeTupleConverter.java |  42 ++++++++-
 .../test/resources/query/sql_casewhen/query58.sql  |  22 +++++
 .../rest/service/TableSchemaUpdateChecker.java     |   4 +
 .../service/update/TableSchemaUpdaterTest.java     |   1 -
 13 files changed, 146 insertions(+), 129 deletions(-)

diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 99123ec..13e73d9 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -507,6 +507,10 @@ public abstract class KylinConfigBase implements Serializable {
         return Boolean.parseBoolean(getOptional("kylin.metadata.model-schema-updater-checker-enabled", "false"));
     }
 
+    public boolean isAbleChangeStringToDateTime() {
+        return Boolean.parseBoolean(getOptional("kylin.metadata.able-change-string-to-datetime", "false"));
+    }
+
     // ============================================================================
     // DICTIONARY & SNAPSHOT
     // ============================================================================
@@ -706,6 +710,10 @@ public abstract class KylinConfigBase implements Serializable {
         return getOptional("kylin.cube.cuboid-scheduler", "org.apache.kylin.cube.cuboid.DefaultCuboidScheduler");
     }
 
+    public boolean isRowKeyEncodingAutoConvert() {
+        return Boolean.parseBoolean(getOptional("kylin.cube.kylin.cube.rowkey-encoding-auto-convert", "true"));
+    }
+    
     public String getSegmentAdvisor() {
         return getOptional("kylin.cube.segment-advisor", "org.apache.kylin.cube.CubeSegmentAdvisor");
     }
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CubeCodeSystem.java b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CubeCodeSystem.java
index 4c71fea..c049027 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CubeCodeSystem.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/CubeCodeSystem.java
@@ -18,12 +18,15 @@
 
 package org.apache.kylin.cube.gridtable;
 
+import static org.apache.kylin.metadata.filter.FilterOptimizeTransformer.logger;
+
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.util.Collections;
 import java.util.Map;
 
 import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.dimension.DictionaryDimEnc;
 import org.apache.kylin.dimension.DictionaryDimEnc.DictionarySerializer;
@@ -33,6 +36,7 @@ import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.gridtable.IGTCodeSystem;
 import org.apache.kylin.gridtable.IGTComparator;
 import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 import org.apache.kylin.metadata.datatype.DynamicDimSerializer;
 
@@ -127,6 +131,23 @@ public class CubeCodeSystem implements IGTCodeSystem {
             if (dictEnc.getRoundingFlag() != roundingFlag) {
                 serializer = dictEnc.copy(roundingFlag).asDataTypeSerializer();
             }
+
+            // Deal with data type change from string to datetime
+            DataType dataType = info.getColumnType(col);
+            if (dataType.isDateTimeFamily()) {
+                try {
+                    long ts = DateFormat.stringToMillis((String) value);
+                    if (dataType.isDate()) {
+                        value = DateFormat.formatToDateStr(ts);
+                    } else {
+                        value = DateFormat.formatToTimeWithoutMilliStr(ts);
+                    }
+                    logger.info("Convert value from {} to {}", ts, value);
+                } catch (Exception e) {
+                    logger.warn("Fail to convert value {} to string due to {}", value, e);
+                }
+            }
+            
             try {
                 serializer.serialize(value, buf);
             } catch (IllegalArgumentException ex) {
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
index f1d5645..1e95f51 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
@@ -45,6 +45,14 @@ import org.apache.kylin.shaded.com.google.common.base.Preconditions;
 public class RowKeyColDesc implements java.io.Serializable {
     private static final Logger logger = LoggerFactory.getLogger(RowKeyColDesc.class);
 
+    public static boolean isDateDimEnc(RowKeyColDesc rowKeyColDesc) {
+        return DateDimEnc.ENCODING_NAME.equals(rowKeyColDesc.getEncodingName());
+    }
+
+    public static boolean isTimeDimEnc(RowKeyColDesc rowKeyColDesc) {
+        return TimeDimEnc.ENCODING_NAME.equals(rowKeyColDesc.getEncodingName());
+    }
+
     @JsonProperty("column")
     private String column;
     @JsonProperty("encoding")
@@ -81,12 +89,14 @@ public class RowKeyColDesc implements java.io.Serializable {
         // convert date/time dictionary on date/time column to DimensionEncoding implicitly
         // however date/time dictionary on varchar column is still required
         DataType type = colRef.getType();
-        if (DictionaryDimEnc.ENCODING_NAME.equals(encodingName)) {
+        if (DictionaryDimEnc.ENCODING_NAME.equals(encodingName) && cubeDesc.getConfig().isRowKeyEncodingAutoConvert()) {
             if (type.isDate()) {
                 encoding = encodingName = DateDimEnc.ENCODING_NAME;
+                logger.info("Implicitly convert encoding to {}", encodingName);
             }
             if (type.isTimeFamily()) {
                 encoding = encodingName = TimeDimEnc.ENCODING_NAME;
+                logger.info("Implicitly convert encoding to {}", encodingName);
             }
         }
 
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
index c28c828..8934a3b 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
@@ -67,6 +67,10 @@ public class RowKeyDesc implements java.io.Serializable {
         return desc;
     }
 
+    public RowKeyColDesc getColDescUncheck(TblColRef col) {
+        return columnMap.get(col);
+    }
+
     public boolean isUseDictionary(TblColRef col) {
         return getColDesc(col).isUsingDictionary();
     }
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
index f4aaa45..a0730ff 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
@@ -22,19 +22,18 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
+import javax.annotation.Nullable;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.metadata.datatype.DataType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-
-import javax.annotation.Nullable;
+import com.google.common.collect.Lists;
 
 /**
  * @author yangli9
@@ -49,18 +48,11 @@ public class DictionaryGenerator {
 
         // build dict, case by data type
         IDictionaryBuilder builder;
-        if (dataType.isDateTimeFamily()) {
-            if (dataType.isDate())
-                builder = new DateDictBuilder();
-            else
-                builder = new TimeDictBuilder();
-        } else {
-            boolean useForest = KylinConfig.getInstanceFromEnv().isUseForestTrieDictionary();
-            if (dataType.isNumberFamily())
-                builder = useForest ? new NumberTrieDictForestBuilder() : new NumberTrieDictBuilder();
-            else
-                builder = useForest ? new StringTrieDictForestBuilder() : new StringTrieDictBuilder();
-        }
+        boolean useForest = KylinConfig.getInstanceFromEnv().isUseForestTrieDictionary();
+        if (dataType.isNumberFamily())
+            builder = useForest ? new NumberTrieDictForestBuilder() : new NumberTrieDictBuilder();
+        else
+            builder = useForest ? new StringTrieDictForestBuilder() : new StringTrieDictBuilder();
         return builder;
     }
 
@@ -123,84 +115,6 @@ public class DictionaryGenerator {
         return buildDictionary(dataType, new MultipleDictionaryValueEnumerator(dataType, dictList));
     }
 
-    private static class DateDictBuilder implements IDictionaryBuilder {
-        private static final String[] DATE_PATTERNS = new String[] { "yyyy-MM-dd", "yyyyMMdd" };
-
-        private int baseId;
-        private String datePattern;
-
-        @Override
-        public void init(DictionaryInfo info, int baseId, String hdfsDir) throws IOException {
-            this.baseId = baseId;
-        }
-
-        @Override
-        public boolean addValue(String value) {
-            if (StringUtils.isBlank(value)) // empty string is treated as null
-                return false;
-
-            // detect date pattern on the first value
-            if (datePattern == null) {
-                for (String p : DATE_PATTERNS) {
-                    try {
-                        DateFormat.stringToDate(value, p);
-                        datePattern = p;
-                        break;
-                    } catch (Exception e) {
-                        // continue;
-                    }
-                }
-                if (datePattern == null)
-                    throw new IllegalArgumentException("Unknown date pattern for input value: " + value);
-            }
-
-            // check the date format
-            DateFormat.stringToDate(value, datePattern);
-            return true;
-        }
-
-        @Override
-        public Dictionary<String> build() throws IOException {
-            if (datePattern == null)
-                datePattern = DATE_PATTERNS[0];
-
-            return new DateStrDictionary(datePattern, baseId);
-        }
-
-
-        @Override
-        public void clear() {
-            // do nothing
-        }
-    }
-
-    private static class TimeDictBuilder implements IDictionaryBuilder {
-
-        @Override
-        public void init(DictionaryInfo info, int baseId, String hdfsDir) throws IOException {
-        }
-
-        @Override
-        public boolean addValue(String value) {
-            if (StringUtils.isBlank(value)) // empty string is treated as null
-                return false;
-
-            // check the time format
-            DateFormat.stringToMillis(value);
-            return true;
-        }
-
-        @Override
-        public Dictionary<String> build() throws IOException {
-            return new TimeStrDictionary(); // base ID is always 0
-        }
-
-        @Override
-        public void clear() {
-
-        }
-    }
-
     private static class StringTrieDictBuilder implements IDictionaryBuilder {
         int baseId;
         TrieDictionaryBuilder builder;
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/LookupStringTable.java b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/LookupStringTable.java
index 1d0348a..fae1cfb 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/LookupStringTable.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/LookupStringTable.java
@@ -84,8 +84,12 @@ public class LookupStringTable extends LookupTable<String> implements ILookupTab
     protected String[] convertRow(String[] cols) {
         for (int i = 0; i < cols.length; i++) {
             if (colIsDateTime[i]) {
-                if (cols[i] != null)
-                    cols[i] = String.valueOf(DateFormat.stringToMillis(cols[i]));
+                if (cols[i] != null) {
+                    if (cols[i].isEmpty())
+                        cols[i] = null;
+                    else
+                        cols[i] = String.valueOf(DateFormat.stringToMillis(cols[i]));
+                }
             }
         }
         return cols;
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/DictionaryProviderTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/DictionaryProviderTest.java
index 7e2e218..82ce587 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/DictionaryProviderTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/DictionaryProviderTest.java
@@ -19,7 +19,6 @@
 package org.apache.kylin.dict;
 
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -37,7 +36,7 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-public class DictionaryProviderTest extends LocalFileMetadataTestCase{
+public class DictionaryProviderTest extends LocalFileMetadataTestCase {
 
     @Before
     public void setUp() throws Exception {
@@ -52,28 +51,13 @@ public class DictionaryProviderTest extends LocalFileMetadataTestCase{
     @Test
     public void testReadWrite() throws Exception {
         //string dict
-        Dictionary<String> dict = getDict(DataType.getType("string"), Arrays.asList(new String[] { "a", "b" }).iterator());
+        Dictionary<String> dict = getDict(DataType.getType("string"),
+                Arrays.asList(new String[] { "a", "b" }).iterator());
         readWriteTest(dict);
         //number dict
-        Dictionary<String> dict2 = getDict(DataType.getType("long"), Arrays.asList(new String[] { "1", "2" }).iterator());
+        Dictionary<String> dict2 = getDict(DataType.getType("long"),
+                Arrays.asList(new String[] { "1", "2" }).iterator());
         readWriteTest(dict2);
-
-        //date dict
-        Dictionary<String> dict3 = getDict(DataType.getType("datetime"), Arrays.asList(new String[] { "20161122", "20161123" }).iterator());
-        readWriteTest(dict3);
-
-        //date dict
-        Dictionary<String> dict4 = getDict(DataType.getType("datetime"), Arrays.asList(new String[] { "2016-11-22", "2016-11-23" }).iterator());
-        readWriteTest(dict4);
-
-        //date dict
-        try {
-            Dictionary<String> dict5 = getDict(DataType.getType("date"), Arrays.asList(new String[] { "2016-11-22", "20161122" }).iterator());
-            readWriteTest(dict5);
-            fail("Date format not correct.Should throw exception");
-        } catch (IllegalArgumentException e) {
-            //correct
-        }
     }
 
     @Test
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
index 6a0cda9..506e7ca 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
@@ -103,6 +103,10 @@ public class Tuple implements ITuple {
         values[idx] = objectValue;
     }
 
+    public void setDimensionValueDirectly(int idx, Object objectValue) {
+        values[idx] = objectValue;
+    }
+
     public void setMeasureValue(String fieldName, Object fieldValue) {
         setMeasureValue(info.getFieldIndex(fieldName), fieldValue);
     }
@@ -182,10 +186,14 @@ public class Tuple implements ITuple {
         }
     }
 
-    private static long epicDaysToMillis(int days) {
+    public static long epicDaysToMillis(int days) {
         return 1L * days * (1000 * 3600 * 24);
     }
 
+    public static int millisToEpicDays(long millis) {
+        return (int) (millis / (1000 * 3600 * 24));
+    }
+
     public static Object convertOptiqCellValue(String strValue, String dataTypeName) {
         if (strValue == null)
             return null;
@@ -197,10 +205,10 @@ public class Tuple implements ITuple {
         switch (dataTypeName) {
         case "date":
             // convert epoch time
-            return Integer.valueOf(dateToEpicDays(strValue));// Optiq expects Integer instead of Long. by honma
+            return millisToEpicDays(DateFormat.stringToMillis(strValue));// Optiq expects Integer instead of Long. by honma
         case "datetime":
         case "timestamp":
-            return Long.valueOf(DateFormat.stringToMillis(strValue));
+            return DateFormat.stringToMillis(strValue);
         case "tinyint":
             return Byte.valueOf(strValue);
         case "smallint":
@@ -222,9 +230,4 @@ public class Tuple implements ITuple {
         }
     }
 
-    private static int dateToEpicDays(String strValue) {
-        long millis = DateFormat.stringToMillis(strValue);
-        return (int) (millis / (1000 * 3600 * 24));
-    }
-
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/TupleInfo.java b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/TupleInfo.java
index c3e88b5..213688d 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/TupleInfo.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/TupleInfo.java
@@ -52,6 +52,10 @@ public class TupleInfo {
         return columns.get(idx);
     }
 
+    public TblColRef getColumn(int idx) {
+        return columns.get(idx);
+    }
+
     public int getColumnIndex(TblColRef col) {
         return columnMap.get(col);
     }
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeTupleConverter.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeTupleConverter.java
index ddf2a5a..5d6f750 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeTupleConverter.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeTupleConverter.java
@@ -28,11 +28,14 @@ import java.util.TimeZone;
 
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Array;
+import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
+import org.apache.kylin.cube.model.RowKeyColDesc;
+import org.apache.kylin.cube.model.RowKeyDesc;
 import org.apache.kylin.dict.lookup.ILookupTable;
 import org.apache.kylin.dimension.TimeDerivedColumnType;
 import org.apache.kylin.measure.MeasureType;
@@ -75,6 +78,8 @@ public class CubeTupleConverter implements ITupleConverter {
 
     public final int nSelectedDims;
 
+    private final RowKeyDesc rowKeyDesc;
+
     public CubeTupleConverter(CubeSegment cubeSeg, Cuboid cuboid, //
             Set<TblColRef> selectedDimensions, Set<FunctionDesc> selectedMetrics, int[] gtColIdx, TupleInfo returnTupleInfo) {
         this.cubeSeg = cubeSeg;
@@ -148,6 +153,8 @@ public class CubeTupleConverter implements ITupleConverter {
                 }
             }
         }
+
+        rowKeyDesc = cubeSeg.getCubeDesc().getRowkey();
     }
 
     // load only needed dictionaries
@@ -169,6 +176,7 @@ public class CubeTupleConverter implements ITupleConverter {
             if (ti >= 0) {
                 // add offset to return result according to timezone
                 if (autoJustByTimezone && timestampColumn.contains(ti)) {
+                    // For streaming
                     try {
                         String v = toString(gtValues[i]);
                         if (v != null) {
@@ -179,7 +187,8 @@ public class CubeTupleConverter implements ITupleConverter {
                         tuple.setDimensionValue(ti, toString(gtValues[i]));
                     }
                 } else {
-                    tuple.setDimensionValue(ti, toString(gtValues[i]));
+                    // For batch
+                    setDimensionValue(tuple, ti, toString(gtValues[i]));
                 }
             }
         }
@@ -209,6 +218,33 @@ public class CubeTupleConverter implements ITupleConverter {
         }
     }
 
+    private void setDimensionValue(Tuple tuple, int idx, String valueStr) {
+        if (valueStr == null) {
+            tuple.setDimensionValueDirectly(idx, valueStr);
+            return;
+        }
+
+        Object valueConvert = null;
+        TblColRef col = tupleInfo.getColumn(idx);
+        RowKeyColDesc rowKeyColDesc = rowKeyDesc.getColDescUncheck(col);
+        if (rowKeyColDesc != null) {
+            // convert value if inconsistency exists between rowkey col encoding & col data type
+            if (col.getType().isDate() && !RowKeyColDesc.isDateDimEnc(rowKeyColDesc)) {
+                long tmpValue = (Long) Tuple.convertOptiqCellValue(valueStr, "timestamp");
+                valueConvert = Tuple.millisToEpicDays(tmpValue);
+            } else if (col.getType().isDatetime() && !RowKeyColDesc.isTimeDimEnc(rowKeyColDesc)) {
+                int tmpValue = (Integer) Tuple.convertOptiqCellValue(valueStr, "date");
+                valueConvert = Tuple.epicDaysToMillis(tmpValue);
+            }
+        }
+
+        if (valueConvert != null) {
+            tuple.setDimensionValueDirectly(idx, valueConvert);
+        } else {
+            tuple.setDimensionValue(idx, valueStr);
+        }
+    }
+
     @Override
     public void close() throws IOException {
         for (ILookupTable usedLookupTable : usedLookupTables) {
@@ -262,6 +298,10 @@ public class CubeTupleConverter implements ITupleConverter {
                 public void fillDerivedColumns(Object[] gtValues, Tuple tuple) {
                     for (int i = 0; i < hostTmpIdx.length; i++) {
                         lookupKey.data[i] = CubeTupleConverter.toString(gtValues[hostTmpIdx[i]]);
+                        // if the primary key of lookup table is date time type, do this change in case of data type inconsistency
+                        if (deriveInfo.join.getPrimaryKeyColumns()[i].getType().isDateTimeFamily()) {
+                            lookupKey.data[i] = String.valueOf(DateFormat.stringToMillis(lookupKey.data[i]));
+                        }
                     }
 
                     String[] lookupRow = lookupTable.getRow(lookupKey);
diff --git a/kylin-it/src/test/resources/query/sql_casewhen/query58.sql b/kylin-it/src/test/resources/query/sql_casewhen/query58.sql
new file mode 100644
index 0000000..c65c911
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_casewhen/query58.sql
@@ -0,0 +1,22 @@
+--
+-- 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.
+--
+
+SELECT (CASE WHEN ("TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" = 'Auction') THEN 'Auction2' ELSE 'Auction1' END) AS "LSTG_FORMAT_NAME__group_",
+  SUM("TEST_KYLIN_FACT"."PRICE") AS "sum_PRICE_ok"
+FROM "TEST_KYLIN_FACT" "TEST_KYLIN_FACT"
+GROUP BY (CASE WHEN ("TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" =  'Auction') THEN 'Auction2' ELSE 'Auction1' END)
\ No newline at end of file
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java b/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java
index 84cc19b..c0acff4 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java
@@ -128,6 +128,10 @@ public class TableSchemaUpdateChecker {
         } else if (column.getType().isNumberFamily()) {
             // Both are float/double should be fine.
             return newCol.getType().isNumberFamily();
+        } else if ((column.getType().isStringFamily() && newCol.getType().isDateTimeFamily())
+                && metadataManager.getConfig().isAbleChangeStringToDateTime()) {
+            // String can be converted to Date or Time
+            return true;
         } else {
             // only compare base type name, changing precision or scale should be fine
             return column.getTypeName().equals(newCol.getTypeName());
diff --git a/server-base/src/test/java/org/apache/kylin/rest/service/update/TableSchemaUpdaterTest.java b/server-base/src/test/java/org/apache/kylin/rest/service/update/TableSchemaUpdaterTest.java
index 7b8eecb..288877c 100644
--- a/server-base/src/test/java/org/apache/kylin/rest/service/update/TableSchemaUpdaterTest.java
+++ b/server-base/src/test/java/org/apache/kylin/rest/service/update/TableSchemaUpdaterTest.java
@@ -24,7 +24,6 @@ import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;


[kylin] 03/15: KYLIN-4421 Allow to update table & database name

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit d494b8d05f75fd648d2dbb1dcb4b72f452de8b55
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Fri Apr 10 10:43:02 2020 +0800

    KYLIN-4421 Allow to update table & database name
---
 .../java/org/apache/kylin/cube/CubeInstance.java   |  65 ++-
 .../java/org/apache/kylin/cube/CubeSegment.java    |  41 ++
 .../apache/kylin/cube/model/AggregationGroup.java  |   2 +
 .../java/org/apache/kylin/cube/model/CubeDesc.java |  65 +++
 .../apache/kylin/cube/model/DictionaryDesc.java    |  17 +
 .../kylin/cube/model/HBaseColumnFamilyDesc.java    |  16 +
 .../org/apache/kylin/cube/model/SelectRule.java    |  40 ++
 .../apache/kylin/cube/model/SnapshotTableDesc.java |  33 +-
 .../apache/kylin/metadata/model/DataModelDesc.java |  46 ++
 .../apache/kylin/metadata/model/JoinTableDesc.java |   9 +
 .../kylin/metadata/model/ModelDimensionDesc.java   |  18 +
 .../apache/kylin/metadata/model/ParameterDesc.java |   7 +-
 .../org/apache/kylin/metadata/model/TableDesc.java |  14 +-
 .../service/update/TableSchemaUpdateMapping.java   |  71 +++
 .../rest/service/update/TableSchemaUpdater.java    | 191 +++++++
 .../service/update/TableSchemaUpdaterTest.java     | 154 +++++
 .../resources/update/TableSchemaUpdateMapping.json |  19 +
 .../test_kylin_cube_with_slr_left_join_ready.json  |  44 ++
 .../update/cube_desc/ci_left_join_cube.json        | 619 +++++++++++++++++++++
 .../update/model_desc/ci_inner_join_model.json     | 238 ++++++++
 .../test/resources/update/table/EDW.CAL_DT.json    | 413 ++++++++++++++
 .../test/resources/update/table/TEST.COUNTRY.json  |  26 +
 .../resources/update/table/TEST.KYLIN_FACT.json    |  73 +++
 .../resources/update/table/TEST.TEST_ACCOUNT.json  |  36 ++
 .../update/table/TEST.TEST_CATEGORY_GROUPINGS.json | 155 ++++++
 25 files changed, 2393 insertions(+), 19 deletions(-)

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 eb90d56..454816d 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
@@ -24,6 +24,7 @@ import static org.apache.kylin.cube.cuboid.CuboidModeEnum.RECOMMEND;
 
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -273,6 +274,33 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
         return true;
     }
 
+    public boolean equalsRaw(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        CubeInstance that = (CubeInstance) o;
+        if (!java.util.Objects.equals(name, that.name))
+            return false;
+        if (!java.util.Objects.equals(owner, that.owner))
+            return false;
+        if (!java.util.Objects.equals(descName, that.descName))
+            return false;
+        if (!java.util.Objects.equals(displayName, that.displayName))
+            return false;
+        if (!java.util.Objects.equals(status, that.status))
+            return false;
+
+        if (!java.util.Objects.equals(segments, that.segments))
+            return false;
+        if (!java.util.Arrays.equals(cuboidBytes, that.cuboidBytes))
+            return false;
+        if (!java.util.Arrays.equals(cuboidBytesRecommend, that.cuboidBytesRecommend))
+            return false;
+        return java.util.Objects.equals(snapshots, that.snapshots);
+    }
+
     // ============================================================================
 
     @Override
@@ -695,6 +723,10 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
         return snapshots;
     }
 
+    public void resetSnapshots() {
+        snapshots = Maps.newHashMap();
+    }
+
     public String getSnapshotResPath(String tableName) {
         return getSnapshots().get(tableName);
     }
@@ -703,18 +735,27 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
         getSnapshots().put(table, snapshotResPath);
     }
 
-    public static CubeInstance getCopyOf(CubeInstance cubeInstance) {
-        CubeInstance newCube = new CubeInstance();
-        newCube.setName(cubeInstance.getName());
-        newCube.setSegments(cubeInstance.getSegments());
-        newCube.setDescName(cubeInstance.getDescName());
-        newCube.setConfig((KylinConfigExt) cubeInstance.getConfig());
-        newCube.setStatus(cubeInstance.getStatus());
-        newCube.setOwner(cubeInstance.getOwner());
-        newCube.setCost(cubeInstance.getCost());
-        newCube.setCreateTimeUTC(System.currentTimeMillis());
-        newCube.updateRandomUuid();
-        return newCube;
+    public static CubeInstance getCopyOf(CubeInstance other) {
+        CubeInstance ret = new CubeInstance();
+        ret.setName(other.getName());
+        ret.setOwner(other.getOwner());
+        ret.setDescName(other.getDescName());
+        ret.setCost(other.getCost());
+        ret.setStatus(other.getStatus());
+        ret.setSegments(other.getSegments());
+        ret.setCreateTimeUTC(System.currentTimeMillis());
+        if (other.cuboidBytes != null) {
+            ret.cuboidBytes = Arrays.copyOf(other.cuboidBytes, other.cuboidBytes.length);
+        }
+        if (other.cuboidBytesRecommend != null) {
+            ret.cuboidBytesRecommend = Arrays.copyOf(other.cuboidBytesRecommend, other.cuboidBytesRecommend.length);
+        }
+        ret.cuboidLastOptimized = other.cuboidLastOptimized;
+        ret.getSnapshots().putAll(other.getSnapshots());
+
+        ret.setConfig((KylinConfigExt) other.getConfig());
+        ret.updateRandomUuid();
+        return ret;
     }
 
     public static CubeSegment findSegmentWithJobId(String jobID, CubeInstance cubeInstance) {
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
index 52f2034..e423f57 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
@@ -321,6 +321,10 @@ public class CubeSegment implements IBuildable, ISegment, Serializable {
         return snapshots;
     }
 
+    public void resetSnapshots() {
+        snapshots = new ConcurrentHashMap<String, String>();
+    }
+
     public String getSnapshotResPath(String table) {
         return getSnapshots().get(table);
     }
@@ -660,4 +664,41 @@ public class CubeSegment implements IBuildable, ISegment, Serializable {
     public void setStreamSourceCheckpoint(String streamSourceCheckpoint) {
         this.streamSourceCheckpoint = streamSourceCheckpoint;
     }
+
+    public static CubeSegment getCopyOf(CubeSegment other) {
+        CubeSegment copy = new CubeSegment();
+        copy.cubeInstance = other.cubeInstance;
+        copy.uuid = other.uuid;
+        copy.name = other.name;
+        copy.storageLocationIdentifier = other.storageLocationIdentifier;
+        copy.dateRangeStart = other.dateRangeStart;
+        copy.dateRangeEnd = other.dateRangeEnd;
+        copy.sourceOffsetStart = other.sourceOffsetStart;
+        copy.sourceOffsetEnd = other.sourceOffsetEnd;
+        copy.status = other.status;
+        copy.sizeKB = other.sizeKB;
+        copy.isMerged = other.isMerged;
+        copy.estimateRatio = other.estimateRatio == null ? null : Lists.newArrayList(other.estimateRatio);
+        copy.inputRecords = other.inputRecords;
+        copy.inputRecordsSize = other.inputRecordsSize;
+        copy.lastBuildTime = other.lastBuildTime;
+        copy.lastBuildJobID = other.lastBuildJobID;
+        copy.createTimeUTC = other.createTimeUTC;
+        copy.cuboidShardNums.putAll(other.cuboidShardNums);
+        copy.totalShards = other.totalShards;
+        copy.blackoutCuboids.addAll(other.blackoutCuboids);
+        copy.getDictionaries().putAll(other.getDictionaries());
+        copy.getSnapshots().putAll(other.getSnapshots());
+        copy.rowkeyStats.addAll(other.rowkeyStats);
+        copy.sourcePartitionOffsetStart.putAll(other.sourcePartitionOffsetStart);
+        copy.sourcePartitionOffsetEnd.putAll(other.sourcePartitionOffsetEnd);
+        if (other.streamSourceCheckpoint != null) {
+            copy.streamSourceCheckpoint = other.streamSourceCheckpoint;
+        }
+        copy.additionalInfo.putAll(other.additionalInfo);
+        copy.dimensionRangeInfoMap = other.dimensionRangeInfoMap == null ? null
+                : Maps.newHashMap(other.dimensionRangeInfoMap);
+        copy.binarySignature = other.binarySignature;
+        return copy;
+    }
 }
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java b/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
index 7de211e..c03cca1 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
@@ -540,4 +540,6 @@ public class AggregationGroup implements Serializable {
     public int getDimCap() {
         return this.selectRule.dimCap == null ? 0 : this.selectRule.dimCap;
     }
+
+
 }
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 364ad6d..13311c0 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
@@ -42,6 +42,7 @@ import java.util.Map.Entry;
 import java.util.Objects;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.stream.IntStream;
 
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.collections.CollectionUtils;
@@ -484,6 +485,70 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
         return mandatoryCuboids;
     }
 
+    public boolean equalsRaw(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        CubeDesc that = (CubeDesc) o;
+
+        if (!Objects.equals(name, that.name))
+            return false;
+        if (!Objects.equals(modelName, that.modelName))
+            return false;
+        if (!Objects.equals(description, that.description))
+            return false;
+        if (!Objects.equals(dimensions, that.dimensions))
+            return false;
+        if (!Objects.equals(measures, that.measures))
+            return false;
+        if (!Objects.equals(dictionaries, that.dictionaries))
+            return false;
+        if (!Arrays.equals(rowkey.getRowKeyColumns(), that.rowkey.getRowKeyColumns()))
+            return false;
+        if (!Objects.equals(nullStrings, that.nullStrings))
+            return false;
+        if (!Arrays.equals(hbaseMapping.getColumnFamily(), that.hbaseMapping.getColumnFamily()))
+            return false;
+        if (aggregationGroups != that.aggregationGroups) {
+            if (aggregationGroups == null || that.aggregationGroups == null) {
+                return false;
+            } else if (!IntStream.range(0, aggregationGroups.size())
+                    .allMatch(i -> Arrays.equals(aggregationGroups.get(i).getIncludes(),
+                            that.aggregationGroups.get(i).getIncludes())
+                            && Objects.equals(aggregationGroups.get(i).getSelectRule(),
+                                    that.aggregationGroups.get(i).getSelectRule()))) {
+                return false;
+            }
+        }
+        if (!Objects.equals(notifyList, that.notifyList))
+            return false;
+        if (!Objects.equals(statusNeedNotify, that.statusNeedNotify))
+            return false;
+        if (!Arrays.equals(autoMergeTimeRanges, that.autoMergeTimeRanges))
+            return false;
+        if (!Objects.equals(retentionRange, that.retentionRange))
+            return false;
+        if (!Objects.equals(engineType, that.engineType))
+            return false;
+        if (!Objects.equals(storageType, that.storageType))
+            return false;
+        if (!Objects.equals(overrideKylinProps, that.overrideKylinProps))
+            return false;
+        if (!Objects.equals(snapshotTableDescList, that.snapshotTableDescList))
+            return false;
+        if (!Objects.equals(partitionDateStart, that.partitionDateStart))
+            return false;
+        if (!Objects.equals(partitionDateEnd, that.partitionDateEnd))
+            return false;
+        if (!Objects.equals(parentForward, that.parentForward))
+            return false;
+        if (!Objects.equals(mandatoryDimensionSetList, that.mandatoryDimensionSetList))
+            return false;
+        return Objects.equals(cuboidBlackSet, that.cuboidBlackSet);
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o)
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java
index 2d1ba99..248984e 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java
@@ -118,4 +118,21 @@ public class DictionaryDesc implements java.io.Serializable {
         desc.builderClass = builderClass;
         return desc;
     }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        DictionaryDesc that = (DictionaryDesc) o;
+        return Objects.equals(column, that.column) &&
+                Objects.equals(reuseColumn, that.reuseColumn) &&
+                Objects.equals(builderClass, that.builderClass) &&
+                Objects.equals(cube, that.cube) &&
+                Objects.equals(model, that.model);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(column, reuseColumn, builderClass, cube, model);
+    }
 }
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnFamilyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnFamilyDesc.java
index 85c2c17..7550751 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnFamilyDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnFamilyDesc.java
@@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.kylin.metadata.model.MeasureDesc;
 
 import java.util.Arrays;
+import java.util.Objects;
 
 /**
  */
@@ -67,4 +68,19 @@ public class HBaseColumnFamilyDesc implements java.io.Serializable {
         return false;
     }
 
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        HBaseColumnFamilyDesc that = (HBaseColumnFamilyDesc) o;
+        return Objects.equals(name, that.name) &&
+                Arrays.equals(columns, that.columns);
+    }
+
+    @Override
+    public int hashCode() {
+        int result = Objects.hash(name);
+        result = 31 * result + Arrays.hashCode(columns);
+        return result;
+    }
 }
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java
index d78da9f..b3527ae 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java
@@ -19,6 +19,9 @@
 package org.apache.kylin.cube.model;
 
 import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Objects;
+import java.util.stream.IntStream;
 
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -35,4 +38,41 @@ public class SelectRule implements Serializable {
     @JsonProperty("dim_cap")
     @JsonInclude(JsonInclude.Include.NON_NULL)
     public Integer dimCap;
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        SelectRule that = (SelectRule) o;
+        if (hierarchyDims != that.hierarchyDims) {
+            if (hierarchyDims == null || that.hierarchyDims == null) {
+                return false;
+            } else if (!IntStream.range(0, hierarchyDims.length)
+                    .allMatch(i -> Arrays.equals(hierarchyDims[i], that.hierarchyDims[i]))) {
+                return false;
+            }
+        }
+
+        if (jointDims != that.jointDims) {
+            if (jointDims == null || that.jointDims == null) {
+                return false;
+            } else if (!IntStream.range(0, jointDims.length)
+                    .allMatch(i -> Arrays.equals(jointDims[i], that.jointDims[i]))) {
+                return false;
+            }
+        }
+        return Arrays.equals(mandatoryDims, that.mandatoryDims) && Objects.equals(dimCap, that.dimCap);
+    }
+
+    @Override
+    public int hashCode() {
+        int result = Objects.hash(dimCap);
+        result = 31 * result + Arrays.hashCode(hierarchyDims);
+        result = 31 * result + Arrays.hashCode(mandatoryDims);
+        result = 31 * result + Arrays.hashCode(jointDims);
+        return result;
+    }
 }
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/SnapshotTableDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/SnapshotTableDesc.java
index e61240b..30f533b 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/SnapshotTableDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/SnapshotTableDesc.java
@@ -18,13 +18,17 @@
 
 package org.apache.kylin.cube.model;
 
+import org.apache.kylin.dict.lookup.SnapshotTable;
+
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.kylin.dict.lookup.SnapshotTable;
+
+import java.util.Objects;
 
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class SnapshotTableDesc implements java.io.Serializable{
+public class
+SnapshotTableDesc implements java.io.Serializable{
     @JsonProperty("table_name")
     private String tableName;
 
@@ -72,4 +76,29 @@ public class SnapshotTableDesc implements java.io.Serializable{
     public void setEnableLocalCache(boolean enableLocalCache) {
         this.enableLocalCache = enableLocalCache;
     }
+
+    public static SnapshotTableDesc getCopyOf(SnapshotTableDesc other) {
+        SnapshotTableDesc copy = new SnapshotTableDesc();
+        copy.tableName = other.tableName;
+        copy.storageType = other.storageType;
+        copy.enableLocalCache = other.enableLocalCache;
+        copy.global = other.global;
+        return copy;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        SnapshotTableDesc that = (SnapshotTableDesc) o;
+        return enableLocalCache == that.enableLocalCache &&
+                global == that.global &&
+                Objects.equals(tableName, that.tableName) &&
+                Objects.equals(storageType, that.storageType);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(tableName, storageType, enableLocalCache, global);
+    }
 }
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 e117176..d126d3d 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
@@ -720,6 +720,52 @@ public class DataModelDesc extends RootPersistentEntity {
         return this.errors;
     }
 
+    private Map<String, JoinTableDesc> getJoinTableMap(JoinTableDesc[] joinTables) {
+        if (joinTables == null) {
+            return Maps.newHashMap();
+        }
+        Map<String, JoinTableDesc> ret = Maps.newHashMapWithExpectedSize(joinTables.length);
+        for (JoinTableDesc joinTable : joinTables) {
+            ret.put(joinTable.getAlias(), joinTable);
+        }
+        return ret;
+    }
+
+    public boolean equalsRaw(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        DataModelDesc that = (DataModelDesc) o;
+
+        if (isDraft != that.isDraft)
+            return false;
+        if (name != null ? !name.equals(that.name) : that.name != null)
+            return false;
+        if (owner != null ? !owner.equals(that.owner) : that.owner != null)
+            return false;
+        if (description != null ? !description.equals(that.description) : that.description != null)
+            return false;
+        if (rootFactTable != null ? !rootFactTable.equals(that.rootFactTable) : that.rootFactTable != null)
+            return false;
+        if (rootFactTableAlias != null ? !rootFactTableAlias.equals(that.rootFactTableAlias)
+                : that.rootFactTableAlias != null)
+            return false;
+        if (!getJoinTableMap(joinTables).equals(getJoinTableMap(that.joinTables)))
+            return false;
+        if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null)
+            return false;
+        // Probably incorrect - comparing Object[] arrays with Arrays.equals
+        if (!Arrays.equals(metrics, that.metrics))
+            return false;
+        if (filterCondition != null ? !filterCondition.equals(that.filterCondition) : that.filterCondition != null)
+            return false;
+        if (partitionDesc != null ? !partitionDesc.equalsRaw(that.partitionDesc) : that.partitionDesc != null)
+            return false;
+        return capacity == that.capacity;
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o)
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
index dc4710e..f0d9111 100644
--- 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
@@ -87,6 +87,15 @@ public class JoinTableDesc implements Serializable {
         this.tableRef = ref;
     }
 
+    public static JoinTableDesc getCopyOf(JoinTableDesc other) {
+        JoinTableDesc copy = new JoinTableDesc();
+        copy.table = other.table;
+        copy.kind = other.kind;
+        copy.alias = other.alias;
+        copy.join = other.join;
+        return copy;
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o)
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java
index caadbd6..cf729a4 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java
@@ -19,9 +19,12 @@
 package org.apache.kylin.metadata.model;
 
 import java.io.Serializable;
+import java.util.Arrays;
 import java.util.List;
 
 import java.util.Locale;
+import java.util.Objects;
+
 import org.apache.kylin.common.util.StringUtil;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
@@ -94,4 +97,19 @@ public class ModelDimensionDesc implements Serializable {
         return count;
     }
 
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        ModelDimensionDesc that = (ModelDimensionDesc) o;
+        return Objects.equals(table, that.table) &&
+                Arrays.equals(columns, that.columns);
+    }
+
+    @Override
+    public int hashCode() {
+        int result = Objects.hash(table);
+        result = 31 * result + Arrays.hashCode(columns);
+        return result;
+    }
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java
index 0633d1e..7748719 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Locale;
+import java.util.Objects;
 import java.util.Set;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
@@ -266,19 +267,19 @@ public class ParameterDesc implements Serializable {
 
             PlainParameter that = (PlainParameter) o;
 
-            if (type != null ? !type.equals(that.type) : that.type != null)
+            if (!Objects.equals(type, that.type))
                 return false;
 
             if (this.isColumnType()) {
                 if (!that.isColumnType())
                     return false;
-                if (!this.colRef.equals(that.colRef)) {
+                if (!Objects.equals(colRef, that.colRef)) {
                     return false;
                 }
             } else {
                 if (that.isColumnType())
                     return false;
-                if (!this.value.equals(that.value))
+                if (!Objects.equals(value, that.value))
                     return false;
             }
 
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java
index 588cf1e..d99ff54 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java
@@ -237,8 +237,7 @@ public class TableDesc extends RootPersistentEntity implements ISourceAware {
 
     public String getIdentity() {
         if (identity == null) {
-            identity = String.format(Locale.ROOT, "%s.%s", this.getDatabase().toUpperCase(Locale.ROOT), this.getName())
-                    .toUpperCase(Locale.ROOT);
+            setIdentity();
         }
         return identity;
     }
@@ -279,6 +278,9 @@ public class TableDesc extends RootPersistentEntity implements ISourceAware {
         } else {
             this.name = null;
         }
+        if (identity != null) {
+            setIdentity();
+        }
     }
 
     @JsonProperty("database")
@@ -289,6 +291,14 @@ public class TableDesc extends RootPersistentEntity implements ISourceAware {
     @JsonProperty("database")
     public void setDatabase(String database) {
         this.database.setName(database);
+        if (identity != null) {
+            setIdentity();
+        }
+    }
+
+    private void setIdentity() {
+        identity = String.format(Locale.ROOT, "%s.%s", this.getDatabase().toUpperCase(Locale.ROOT), this.getName())
+                .toUpperCase(Locale.ROOT);
     }
 
     public ColumnDesc[] getColumns() {
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdateMapping.java b/server-base/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdateMapping.java
new file mode 100644
index 0000000..62b4a35
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdateMapping.java
@@ -0,0 +1,71 @@
+/*
+ * 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.rest.service.update;
+
+import java.util.Locale;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+
+public class TableSchemaUpdateMapping {
+
+    private String database;
+
+    private String tableName;
+
+    public boolean isDatabaseChanged() {
+        return !Strings.isNullOrEmpty(database);
+    }
+
+    public String getDatabase(String dbName) {
+        String ret = isDatabaseChanged() ? database : dbName;
+        return ret.toUpperCase(Locale.ROOT);
+    }
+
+    public void setDatabase(String database) {
+        this.database = database;
+    }
+
+    public boolean isTableNameChanged() {
+        return !Strings.isNullOrEmpty(tableName);
+    }
+
+    public String getTableName(String tblName) {
+        String ret = isTableNameChanged() ? tableName : tblName;
+        return ret.toUpperCase(Locale.ROOT);
+    }
+
+    public void setTableName(String tableName) {
+        this.tableName = tableName;
+    }
+
+    public boolean isTableIdentityChanged() {
+        return isDatabaseChanged() || isTableNameChanged();
+    }
+
+    public String getTableIdentity(String tableIdentity) {
+        String[] tableNameEs = tableIdentity.split("\\.");
+        Preconditions.checkArgument(tableNameEs.length == 2);
+        return getTableIdentity(tableNameEs[0], tableNameEs[1]);
+    }
+
+    public String getTableIdentity(String database, String tableName) {
+        return getDatabase(database) + "." + getTableName(tableName);
+    }
+}
\ No newline at end of file
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdater.java b/server-base/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdater.java
new file mode 100644
index 0000000..99c88a0
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdater.java
@@ -0,0 +1,191 @@
+/*
+ * 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.rest.service.update;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.SnapshotTableDesc;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.JoinTableDesc;
+import org.apache.kylin.metadata.model.PartitionDesc;
+import org.apache.kylin.metadata.model.Segments;
+import org.apache.kylin.metadata.model.TableDesc;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class TableSchemaUpdater {
+
+    public static TableDesc dealWithMappingForTable(TableDesc other, Map<String, TableSchemaUpdateMapping> mappings) {
+        TableSchemaUpdateMapping mapping = getTableSchemaUpdateMapping(mappings, other.getIdentity());
+        if (mapping == null) {
+            return other;
+        }
+
+        TableDesc copy = new TableDesc(other);
+
+        copy.setDatabase(mapping.getDatabase(other.getDatabase()));
+
+        copy.setName(mapping.getTableName(other.getName()));
+
+        // It will always be a new one
+        copy.setLastModified(0L);
+
+        return copy;
+    }
+
+    // the input data model should be initialized, then table names & col names will be normalized
+    public static DataModelDesc dealWithMappingForModel(DataModelDesc other,
+            Map<String, TableSchemaUpdateMapping> mappings) {
+        // For filter condition, not support
+        if (!Strings.isNullOrEmpty(other.getFilterCondition())) {
+            throw new UnsupportedOperationException("Cannot deal with filter condition " + other.getFilterCondition());
+        }
+
+        DataModelDesc copy = DataModelDesc.getCopyOf(other);
+        copy.setLastModified(other.getLastModified());
+
+        // mapping for root fact table identity
+        TableSchemaUpdateMapping rootMapping = getTableSchemaUpdateMapping(mappings, other.getRootFactTableName());
+        if (rootMapping != null) {
+            TableDesc rootFactTable = other.getRootFactTable().getTableDesc();
+            copy.setRootFactTableName(
+                    rootMapping.getTableIdentity(rootFactTable.getDatabase(), rootFactTable.getName()));
+        }
+
+        // mapping for joins
+        JoinTableDesc[] joinTables = other.getJoinTables();
+        JoinTableDesc[] joinTablesCopy = new JoinTableDesc[joinTables.length];
+        for (int i = 0; i < joinTables.length; i++) {
+            JoinTableDesc joinTable = joinTables[i];
+            joinTablesCopy[i] = JoinTableDesc.getCopyOf(joinTable);
+            String tableIdentity = joinTable.getTable();
+            TableSchemaUpdateMapping mapping = getTableSchemaUpdateMapping(mappings, tableIdentity);
+            if (mapping != null && mapping.isTableIdentityChanged()) {
+                joinTablesCopy[i].setTable(mapping.getTableIdentity(tableIdentity));
+            }
+        }
+        copy.setJoinTables(joinTablesCopy);
+
+        // mapping for partition columns
+        PartitionDesc partDesc = other.getPartitionDesc();
+        PartitionDesc partCopy = PartitionDesc.getCopyOf(partDesc);
+        if (partDesc.getPartitionDateColumnRef() != null) {
+            partCopy.setPartitionDateColumn(
+                    replacePartitionCol(partDesc.getPartitionDateColumnRef().getCanonicalName(), mappings));
+        }
+        if (partDesc.getPartitionTimeColumnRef() != null) {
+            partCopy.setPartitionTimeColumn(
+                    replacePartitionCol(partDesc.getPartitionTimeColumnRef().getCanonicalName(), mappings));
+        }
+        copy.setPartitionDesc(partCopy);
+
+        return copy;
+    }
+
+    public static CubeDesc dealWithMappingForCubeDesc(CubeDesc other, Map<String, TableSchemaUpdateMapping> mappings) {
+        CubeDesc copy = CubeDesc.getCopyOf(other);
+        copy.setLastModified(other.getLastModified());
+
+        // mapping for cube-level snapshot tables
+        if (other.getSnapshotTableDescList() != null && !other.getSnapshotTableDescList().isEmpty()) {
+            List<SnapshotTableDesc> snapshotTableDescListCopy = Lists
+                    .newArrayListWithExpectedSize(other.getSnapshotTableDescList().size());
+            for (SnapshotTableDesc snapshotDesc : other.getSnapshotTableDescList()) {
+                TableSchemaUpdateMapping mapping = getTableSchemaUpdateMapping(mappings, snapshotDesc.getTableName());
+                if (mapping != null && mapping.isTableIdentityChanged()) {
+                    snapshotDesc = SnapshotTableDesc.getCopyOf(snapshotDesc);
+                    snapshotDesc.setTableName(mapping.getTableIdentity(snapshotDesc.getTableName()));
+                }
+                snapshotTableDescListCopy.add(snapshotDesc);
+            }
+            copy.setSnapshotTableDescList(snapshotTableDescListCopy);
+        }
+
+        return copy;
+    }
+
+    public static CubeInstance dealWithMappingForCube(CubeInstance other,
+            Map<String, TableSchemaUpdateMapping> mappings) {
+        CubeInstance copy = CubeInstance.getCopyOf(other);
+        copy.setLastModified(other.getLastModified());
+
+        // mapping for cube-level snapshot tables
+        if (other.getSnapshots() != null && !other.getSnapshots().isEmpty()) {
+            Map<String, String> snapshotsCopy = replaceTableIdentityForTableSnapshots(other.getSnapshots(), mappings);
+            copy.resetSnapshots();
+            copy.getSnapshots().putAll(snapshotsCopy);
+        }
+
+        // mapping for segment-level snapshot tables
+        if (other.getSegments() != null && !other.getSegments().isEmpty()) {
+            Segments<CubeSegment> segmentsCopy = new Segments<>();
+            for (CubeSegment segment : other.getSegments()) {
+                CubeSegment segmentCopy = CubeSegment.getCopyOf(segment);
+                segmentCopy.setCubeInstance(copy);
+                Map<String, String> snapshotsCopy = replaceTableIdentityForTableSnapshots(segment.getSnapshots(),
+                        mappings);
+                segmentCopy.resetSnapshots();
+                segmentCopy.getSnapshots().putAll(snapshotsCopy);
+                segmentsCopy.add(segmentCopy);
+            }
+            copy.setSegments(segmentsCopy);
+        }
+
+        return copy;
+    }
+
+    private static Map<String, String> replaceTableIdentityForTableSnapshots(Map<String, String> snapshots,
+            Map<String, TableSchemaUpdateMapping> mappings) {
+        Map<String, String> snapshotsCopy = Maps.newHashMapWithExpectedSize(snapshots.size());
+        for (String tableIdentity : snapshots.keySet()) {
+            String resPath = snapshots.get(tableIdentity);
+            TableSchemaUpdateMapping mapping = getTableSchemaUpdateMapping(mappings, tableIdentity);
+            if (mapping != null && mapping.isTableIdentityChanged()) {
+                tableIdentity = mapping.getTableIdentity(tableIdentity);
+            }
+            snapshotsCopy.put(tableIdentity, resPath);
+        }
+        return snapshotsCopy;
+    }
+
+    private static String replacePartitionCol(String partCol, Map<String, TableSchemaUpdateMapping> mappings) {
+        int cut = partCol.lastIndexOf('.');
+        if (cut < 0) {
+            return partCol;
+        }
+        String partTableIdentity = partCol.substring(0, cut);
+        TableSchemaUpdateMapping mapping = getTableSchemaUpdateMapping(mappings, partTableIdentity);
+        if (mapping != null) {
+            return mapping.getTableIdentity(partTableIdentity) + "." + partCol.substring(cut + 1);
+        }
+        return partCol;
+    }
+
+    public static TableSchemaUpdateMapping getTableSchemaUpdateMapping(Map<String, TableSchemaUpdateMapping> mappings,
+            String key) {
+        return mappings.get(key.toUpperCase(Locale.ROOT));
+    }
+}
diff --git a/server-base/src/test/java/org/apache/kylin/rest/service/update/TableSchemaUpdaterTest.java b/server-base/src/test/java/org/apache/kylin/rest/service/update/TableSchemaUpdaterTest.java
new file mode 100644
index 0000000..7b8eecb
--- /dev/null
+++ b/server-base/src/test/java/org/apache/kylin/rest/service/update/TableSchemaUpdaterTest.java
@@ -0,0 +1,154 @@
+/*
+ * 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.rest.service.update;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.persistence.RootPersistentEntity;
+import org.apache.kylin.common.persistence.Serializer;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.cube.CubeDescManager;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.metadata.TableMetadataManager;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.DataModelManager;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.collect.Sets;
+
+public class TableSchemaUpdaterTest extends LocalFileMetadataTestCase {
+
+    private final String mappingRootPath = "src/test/resources/update";
+    private final String mappingFileName = "TableSchemaUpdateMapping.json";
+    private Map<String, TableSchemaUpdateMapping> mappings;
+
+    @Before
+    public void setUp() throws IOException {
+        this.createTestMetadata();
+
+        File mappingFile = new File(mappingRootPath + "/" + mappingFileName);
+        String content = new String(Files.readAllBytes(mappingFile.toPath()), StandardCharsets.UTF_8);
+        mappings = JsonUtil.readValue(content, new TypeReference<Map<String, TableSchemaUpdateMapping>>() {
+        });
+    }
+
+    @Test
+    public void testDealWithMappingForTable() throws IOException {
+        TableMetadataManager tableMetaManager = TableMetadataManager.getInstance(getTestConfig());
+        ResourceStore store = tableMetaManager.getStore();
+
+        Set<TableDesc> tables = Sets.newHashSet();
+        for (String tableIdentity : mappings.keySet()) {
+            tables.add(store.getResource(TableDesc.concatResourcePath(tableIdentity, null),
+                    TableMetadataManager.TABLE_SERIALIZER));
+        }
+
+        for (TableDesc tableDesc : tables) {
+            TableDesc updated = TableSchemaUpdater.dealWithMappingForTable(tableDesc, mappings);
+            updated = reinit(updated, TableMetadataManager.TABLE_SERIALIZER);
+
+            try (DataInputStream bis = new DataInputStream(new FileInputStream(
+                    new File(mappingRootPath + TableDesc.concatResourcePath(updated.getIdentity(), null))))) {
+                TableDesc expected = TableMetadataManager.TABLE_SERIALIZER.deserialize(bis);
+                Assert.assertEquals(expected, updated);
+            } catch (Exception e) {
+                Assert.fail("Table identity is not updated correctly");
+            }
+        }
+    }
+
+    @Test
+    public void testDealWithMappingForModel() throws IOException {
+        DataModelManager dataModelManager = DataModelManager.getInstance(getTestConfig());
+        DataModelDesc model = dataModelManager.getDataModelDesc("ci_inner_join_model");
+
+        DataModelDesc updated = TableSchemaUpdater.dealWithMappingForModel(model, mappings);
+        updated = reinit(updated, dataModelManager.getDataModelSerializer());
+
+        try (DataInputStream bis = new DataInputStream(
+                new FileInputStream(new File(mappingRootPath + DataModelDesc.concatResourcePath(updated.getName()))))) {
+            DataModelDesc expected = dataModelManager.getDataModelSerializer().deserialize(bis);
+            Assert.assertTrue(expected.equalsRaw(updated));
+        } catch (Exception e) {
+            Assert.fail("Model is not updated correctly");
+        }
+    }
+
+    @Test
+    public void testDealWithMappingForCubeDesc() throws IOException {
+        CubeDescManager cubeDescManager = CubeDescManager.getInstance(getTestConfig());
+        CubeDesc cubeDesc = cubeDescManager.getCubeDesc("ci_left_join_cube");
+
+        CubeDesc updated = TableSchemaUpdater.dealWithMappingForCubeDesc(cubeDesc, mappings);
+        updated = reinit(updated, cubeDescManager.CUBE_DESC_SERIALIZER);
+
+        try (DataInputStream bis = new DataInputStream(
+                new FileInputStream(new File(mappingRootPath + CubeDesc.concatResourcePath(updated.getName()))))) {
+            CubeDesc expected = cubeDescManager.CUBE_DESC_SERIALIZER.deserialize(bis);
+            Assert.assertTrue(expected.equalsRaw(updated));
+        } catch (Exception e) {
+            Assert.fail("CubeDesc is not updated correctly");
+        }
+    }
+
+    @Test
+    public void testDealWithMappingForCube() throws IOException {
+        CubeManager cubeManager = CubeManager.getInstance(getTestConfig());
+        CubeInstance cube = cubeManager.getCube("test_kylin_cube_with_slr_left_join_ready");
+
+        CubeInstance updated = TableSchemaUpdater.dealWithMappingForCube(cube, mappings);
+        updated = reinit(updated, cubeManager.CUBE_SERIALIZER);
+
+        try (DataInputStream bis = new DataInputStream(
+                new FileInputStream(new File(mappingRootPath + CubeInstance.concatResourcePath(updated.getName()))))) {
+            CubeInstance expected = cubeManager.CUBE_SERIALIZER.deserialize(bis);
+            Assert.assertTrue(expected.equalsRaw(updated));
+        } catch (Exception e) {
+            Assert.fail("CubeInstance is not updated correctly");
+        }
+    }
+
+    private <T extends RootPersistentEntity> T reinit(T obj, Serializer<T> serializer) throws IOException {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        serializer.serialize(obj, dos);
+        ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+        DataInputStream dis = new DataInputStream(bais);
+        return serializer.deserialize(dis);
+    }
+}
diff --git a/server-base/src/test/resources/update/TableSchemaUpdateMapping.json b/server-base/src/test/resources/update/TableSchemaUpdateMapping.json
new file mode 100644
index 0000000..bd601c8
--- /dev/null
+++ b/server-base/src/test/resources/update/TableSchemaUpdateMapping.json
@@ -0,0 +1,19 @@
+{
+  "DEFAULT.TEST_KYLIN_FACT": {
+    "database": "TEST",
+    "tableName": "KYLIN_FACT"
+  },
+  "DEFAULT.TEST_ACCOUNT": {
+    "database": "TEST"
+  },
+  "DEFAULT.TEST_COUNTRY": {
+    "database": "TEST",
+    "tableName": "COUNTRY"
+  },
+  "EDW.TEST_CAL_DT": {
+    "tableName": "CAL_DT"
+  },
+  "DEFAULT.TEST_CATEGORY_GROUPINGS": {
+    "database": "TEST"
+  }
+}
\ No newline at end of file
diff --git a/server-base/src/test/resources/update/cube/test_kylin_cube_with_slr_left_join_ready.json b/server-base/src/test/resources/update/cube/test_kylin_cube_with_slr_left_join_ready.json
new file mode 100644
index 0000000..8cd7548
--- /dev/null
+++ b/server-base/src/test/resources/update/cube/test_kylin_cube_with_slr_left_join_ready.json
@@ -0,0 +1,44 @@
+{
+  "uuid" : "kkkka32a-a33e-4b69-83dd-0bb8b1f8c53b",
+ 
+  "last_modified" : 1404097095621,
+  "name" : "test_kylin_cube_with_slr_left_join_ready",
+  "owner" : null,
+  "descriptor" : "test_kylin_cube_with_slr_left_join_desc",
+  "cost" : 50,
+  "segments" : [ {
+    "uuid" : "1eaca32a-a33e-4b69-83dd-xxe8b1f8dddd",
+	"name" : "1eaca32a-a33e-4b69-83dd-xxe8b1f8dddd",
+	"storage_location_identifier" : "KYLIN-CUBE-TEST_KYLIN_CUBE_WITH_SLR_LEFT_JOIN_READY-BCF2F125-9B0B-40DD-9509-95EC59B31333",
+    "date_range_start" : 0,
+    "date_range_end" : 1384243200000,
+	"status" : "READY",
+	"size_kb" : 7690,
+	"source_records" : 10000,
+	"source_records_size" : 610288,
+	"last_build_time" : 1404097095455,
+	"last_build_job_id" : "bcf2f125-9b0b-40dd-9509-95ec59b31333",
+	"binary_signature" : null,
+      "dictionaries" : {
+        "TEST.TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/c12ae49d-9dbe-4a58-b169-19afac317696.dict",
+        "TEST.TEST_FACT/SLR_SEGMENT_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/14fe66b3-5956-498c-bd93-40182cac5510.dict",
+        "TEST.TEST_FACT/LSTG_SITE_ID" : "/dict/TEST_SITES/SITE_ID/0bec6bb3-1b0d-469c-8289-b8c4ca5d5001.dict",
+        "EDW.TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict",
+        "TEST.TEST_FACT/CAL_DT" : "/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict",
+        "TEST.TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/eacccee7-d120-4f4d-97d0-c99a5b83ec32.dict",
+        "EDW.CAL_DT/CAL_DT" : "/dict/TEST_CAL_DT/CAL_DT/ed0c3451-593c-494c-9019-64f63fcb0b8e.dict",
+        "TEST.TEST_FACT/LEAF_CATEG_ID" : "/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/8b4b1c06-fb74-486b-a2ad-74420afebcda.dict",
+        "EDW.TEST_SITES/SITE_ID" : "/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict",
+        "TEST.TEST_CATEGORY_GROUPINGS/META_CATEG_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/c2af25cf-6c79-45e6-a6f6-6d2a8ecc6592.dict"
+      },
+      "snapshots" : {
+        "EDW.TEST_SELLER_TYPE_DIM" : "/table_snapshot/TEST_SELLER_TYPE_DIM.csv/b43dd3f1-9a34-4721-8abc-df90962e94d8.snapshot",
+        "EDW.CAL_DT" : "/table_snapshot/TEST_CAL_DT.csv/4af48c94-86de-4e22-a4fd-c49b06cbaa4f.snapshot",
+        "TEST.TEST_CATEGORY_GROUPINGS" : "/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/89715f33-15c9-4745-83f9-f2b9817d9100.snapshot",
+        "EDW.TEST_SITES" : "/table_snapshot/TEST_SITES.csv/7d70b82e-43f6-4999-b012-0e91a9bb8408.snapshot"
+      }
+  } ],
+  "status" : "READY",
+  "create_time" : null,
+  "notify_list" : null
+}
\ No newline at end of file
diff --git a/server-base/src/test/resources/update/cube_desc/ci_left_join_cube.json b/server-base/src/test/resources/update/cube_desc/ci_left_join_cube.json
new file mode 100644
index 0000000..f7f5959
--- /dev/null
+++ b/server-base/src/test/resources/update/cube_desc/ci_left_join_cube.json
@@ -0,0 +1,619 @@
+{
+  "uuid": "629ab7a8-3929-4dff-b59d-2100aadccd1a",
+  "name": "ci_left_join_cube",
+  "model_name": "ci_left_join_model",
+  "description": null,
+  "dimensions": [
+    {
+      "name": "CAL_DT",
+      "table": "TEST_CAL_DT",
+      "column": "{FK}",
+      "derived": [
+        "WEEK_BEG_DT"
+      ]
+    },
+    {
+      "name": "ORDER_ID",
+      "table": "TEST_KYLIN_FACT",
+      "column": "ORDER_ID"
+    },
+    {
+      "name": "TEST_DATE_ENC",
+      "table": "TEST_ORDER",
+      "column": "TEST_DATE_ENC"
+    },
+    {
+      "name": "TEST_TIME_ENC",
+      "table": "TEST_ORDER",
+      "column": "TEST_TIME_ENC"
+    },
+    {
+      "name": "CATEGORY",
+      "table": "TEST_CATEGORY_GROUPINGS",
+      "column": "{FK}",
+      "derived": [
+        "USER_DEFINED_FIELD1",
+        "USER_DEFINED_FIELD3",
+        "UPD_DATE",
+        "UPD_USER"
+      ]
+    },
+    {
+      "name": "CATEGORY_HIERARCHY",
+      "table": "TEST_CATEGORY_GROUPINGS",
+      "column": "META_CATEG_NAME",
+      "derived": null
+    },
+    {
+      "name": "CATEGORY_HIERARCHY",
+      "table": "TEST_CATEGORY_GROUPINGS",
+      "column": "CATEG_LVL2_NAME",
+      "derived": null
+    },
+    {
+      "name": "CATEGORY_HIERARCHY",
+      "table": "TEST_CATEGORY_GROUPINGS",
+      "column": "CATEG_LVL3_NAME",
+      "derived": null
+    },
+    {
+      "name": "LSTG_FORMAT_NAME",
+      "table": "TEST_KYLIN_FACT",
+      "column": "LSTG_FORMAT_NAME",
+      "derived": null
+    },
+    {
+      "name": "SITE_ID",
+      "table": "TEST_SITES",
+      "column": "{FK}",
+      "derived": [
+        "SITE_NAME",
+        "CRE_USER"
+      ]
+    },
+    {
+      "name": "SELLER_TYPE_CD",
+      "table": "TEST_SELLER_TYPE_DIM",
+      "column": "{FK}",
+      "derived": [
+        "SELLER_TYPE_DESC"
+      ]
+    },
+    {
+      "name": "SELLER_ID",
+      "table": "TEST_KYLIN_FACT",
+      "column": "SELLER_ID"
+    },
+    {
+      "name": "SELLER_BUYER_LEVEL",
+      "table": "SELLER_ACCOUNT",
+      "column": "ACCOUNT_BUYER_LEVEL"
+    },
+    {
+      "name": "SELLER_SELLER_LEVEL",
+      "table": "SELLER_ACCOUNT",
+      "column": "ACCOUNT_SELLER_LEVEL"
+    },
+    {
+      "name": "SELLER_COUNTRY",
+      "table": "SELLER_ACCOUNT",
+      "column": "ACCOUNT_COUNTRY"
+    },
+    {
+      "name": "SELLER_COUNTRY_NAME",
+      "table": "SELLER_COUNTRY",
+      "column": "NAME"
+    },
+    {
+      "name": "BUYER_ID",
+      "table": "TEST_ORDER",
+      "column": "BUYER_ID"
+    },
+    {
+      "name": "BUYER_BUYER_LEVEL",
+      "table": "BUYER_ACCOUNT",
+      "column": "ACCOUNT_BUYER_LEVEL"
+    },
+    {
+      "name": "BUYER_SELLER_LEVEL",
+      "table": "BUYER_ACCOUNT",
+      "column": "ACCOUNT_SELLER_LEVEL"
+    },
+    {
+      "name": "BUYER_COUNTRY",
+      "table": "BUYER_ACCOUNT",
+      "column": "ACCOUNT_COUNTRY"
+    },
+    {
+      "name": "BUYER_COUNTRY_NAME",
+      "table": "BUYER_COUNTRY",
+      "column": "NAME"
+    }
+  ],
+  "measures": [
+    {
+      "name": "TRANS_CNT",
+      "function": {
+        "expression": "COUNT",
+        "parameter": {
+          "type": "constant",
+          "value": "1"
+        },
+        "returntype": "bigint"
+      }
+    },
+    {
+      "name": "ITEM_COUNT_SUM",
+      "function": {
+        "expression": "SUM",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.ITEM_COUNT"
+        },
+        "returntype": "bigint"
+      }
+    },
+    {
+      "name": "GMV_SUM",
+      "function": {
+        "expression": "SUM",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.PRICE"
+        },
+        "returntype": "decimal(19,4)"
+      }
+    },
+    {
+      "name": "GMV_CNT",
+      "function": {
+        "expression": "COUNT",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.PRICE"
+        },
+        "returntype": "bigint"
+      }
+    },
+    {
+      "name": "GMV_MIN",
+      "function": {
+        "expression": "MIN",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.PRICE"
+        },
+        "returntype": "decimal(19,4)"
+      }
+    },
+    {
+      "name": "GMV_MAX",
+      "function": {
+        "expression": "MAX",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.PRICE"
+        },
+        "returntype": "decimal(19,4)"
+      }
+    },
+    {
+      "name": "SELLER_HLL",
+      "function": {
+        "expression": "COUNT_DISTINCT",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.SELLER_ID"
+        },
+        "returntype": "hllc(10)"
+      }
+    },
+    {
+      "name": "SELLER_FORMAT_HLL",
+      "function": {
+        "expression": "COUNT_DISTINCT",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.LSTG_FORMAT_NAME",
+          "next_parameter": {
+            "type": "column",
+            "value": "TEST_KYLIN_FACT.SELLER_ID"
+          }
+        },
+        "returntype": "hllc(10)"
+      }
+    },
+    {
+      "name": "TOP_SELLER",
+      "function": {
+        "expression": "TOP_N",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.PRICE",
+          "next_parameter": {
+            "type": "column",
+            "value": "TEST_KYLIN_FACT.SELLER_ID"
+          }
+        },
+        "returntype": "topn(100, 4)",
+        "configuration": {
+          "topn.encoding.TEST_KYLIN_FACT.SELLER_ID": "int:4"
+        }
+      }
+    },
+    {
+      "name": "TEST_COUNT_DISTINCT_BITMAP",
+      "function": {
+        "expression": "COUNT_DISTINCT",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.TEST_COUNT_DISTINCT_BITMAP"
+        },
+        "returntype": "bitmap"
+      }
+    },
+    {
+      "name": "TEST_EXTENDED_COLUMN",
+      "function": {
+        "expression": "EXTENDED_COLUMN",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.ORDER_ID",
+          "next_parameter": {
+            "type": "column",
+            "value": "TEST_ORDER.TEST_EXTENDED_COLUMN"
+          }
+        },
+        "returntype": "extendedcolumn(100)"
+      }
+    },
+    {
+      "name": "BUYER_CONTACT",
+      "function": {
+        "expression": "EXTENDED_COLUMN",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_ORDER.BUYER_ID",
+          "next_parameter": {
+            "type": "column",
+            "value": "BUYER_ACCOUNT.ACCOUNT_CONTACT"
+          }
+        },
+        "returntype": "extendedcolumn(100)"
+      }
+    },
+    {
+      "name": "SELLER_CONTACT",
+      "function": {
+        "expression": "EXTENDED_COLUMN",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.SELLER_ID",
+          "next_parameter": {
+            "type": "column",
+            "value": "SELLER_ACCOUNT.ACCOUNT_CONTACT"
+          }
+        },
+        "returntype": "extendedcolumn(100)"
+      }
+    },
+    {
+      "name": "TRANS_ID_RAW",
+      "function": {
+        "expression": "RAW",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.TRANS_ID"
+        },
+        "returntype": "raw"
+      }
+    },
+    {
+      "name": "PRICE_RAW",
+      "function": {
+        "expression": "RAW",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.PRICE"
+        },
+        "returntype": "raw"
+      }
+    },
+    {
+      "name": "CAL_DT_RAW",
+      "function": {
+        "expression": "RAW",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.CAL_DT"
+        },
+        "returntype": "raw"
+      }
+    },
+    {
+      "name": "GVM_PERCENTILE",
+      "function": {
+        "expression": "PERCENTILE_APPROX",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.PRICE"
+        },
+        "returntype": "percentile(100)"
+      }
+    },
+    {
+      "name": "TEST_COUNT_COLUMN_CNT",
+      "function": {
+        "expression": "COUNT",
+        "parameter": {
+          "type": "column",
+          "value": "TEST_KYLIN_FACT.TEST_COUNT_COLUMN"
+        },
+        "returntype": "bigint"
+      }
+    }
+  ],
+  "dictionaries": [
+    {
+      "column": "TEST_KYLIN_FACT.TEST_COUNT_DISTINCT_BITMAP",
+      "builder": "org.apache.kylin.dict.GlobalDictionaryBuilder"
+    }
+  ],
+  "rowkey": {
+    "rowkey_columns": [
+      {
+        "column": "TEST_KYLIN_FACT.SELLER_ID",
+        "encoding": "int:4"
+      },
+      {
+        "column": "TEST_KYLIN_FACT.ORDER_ID",
+        "encoding": "int:4"
+      },
+      {
+        "column": "TEST_KYLIN_FACT.CAL_DT",
+        "encoding": "dict"
+      },
+      {
+        "column": "TEST_KYLIN_FACT.LEAF_CATEG_ID",
+        "encoding": "dict"
+      },
+      {
+        "column": "TEST_CATEGORY_GROUPINGS.META_CATEG_NAME",
+        "encoding": "dict"
+      },
+      {
+        "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME",
+        "encoding": "dict"
+      },
+      {
+        "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME",
+        "encoding": "dict"
+      },
+      {
+        "column": "TEST_KYLIN_FACT.LSTG_FORMAT_NAME",
+        "encoding": "fixed_length:12"
+      },
+      {
+        "column": "TEST_KYLIN_FACT.LSTG_SITE_ID",
+        "encoding": "dict"
+      },
+      {
+        "column": "TEST_KYLIN_FACT.SLR_SEGMENT_CD",
+        "encoding": "dict"
+      },
+      {
+        "column": "TEST_ORDER.TEST_TIME_ENC",
+        "encoding": "time"
+      },
+      {
+        "column": "TEST_ORDER.TEST_DATE_ENC",
+        "encoding": "date"
+      },
+      {
+        "column": "TEST_ORDER.BUYER_ID",
+        "encoding": "int:4"
+      },
+      {
+        "column": "BUYER_ACCOUNT.ACCOUNT_BUYER_LEVEL",
+        "encoding": "dict"
+      },
+      {
+        "column": "BUYER_ACCOUNT.ACCOUNT_SELLER_LEVEL",
+        "encoding": "dict"
+      },
+      {
+        "column": "BUYER_ACCOUNT.ACCOUNT_COUNTRY",
+        "encoding": "dict"
+      },
+      {
+        "column": "BUYER_COUNTRY.NAME",
+        "encoding": "dict"
+      },
+      {
+        "column": "SELLER_ACCOUNT.ACCOUNT_BUYER_LEVEL",
+        "encoding": "dict"
+      },
+      {
+        "column": "SELLER_ACCOUNT.ACCOUNT_SELLER_LEVEL",
+        "encoding": "dict"
+      },
+      {
+        "column": "SELLER_ACCOUNT.ACCOUNT_COUNTRY",
+        "encoding": "dict"
+      },
+      {
+        "column": "SELLER_COUNTRY.NAME",
+        "encoding": "dict"
+      }
+    ]
+  },
+  "signature": null,
+  "last_modified": 1448959801311,
+  "null_string": null,
+  "hbase_mapping": {
+    "column_family": [
+      {
+        "name": "F1",
+        "columns": [
+          {
+            "qualifier": "M",
+            "measure_refs": [
+              "TRANS_CNT",
+              "ITEM_COUNT_SUM",
+              "GMV_SUM",
+              "GMV_CNT",
+              "GMV_MIN",
+              "GMV_MAX",
+              "TEST_COUNT_COLUMN_CNT"
+            ]
+          }
+        ]
+      },
+      {
+        "name": "F2",
+        "columns": [
+          {
+            "qualifier": "M",
+            "measure_refs": [
+              "SELLER_HLL",
+              "SELLER_FORMAT_HLL",
+              "TOP_SELLER",
+              "TEST_COUNT_DISTINCT_BITMAP"
+            ]
+          }
+        ]
+      },
+      {
+        "name": "F3",
+        "columns": [
+          {
+            "qualifier": "M",
+            "measure_refs": [
+              "TEST_EXTENDED_COLUMN",
+              "BUYER_CONTACT",
+              "SELLER_CONTACT",
+              "TRANS_ID_RAW",
+              "PRICE_RAW",
+              "CAL_DT_RAW",
+              "GVM_PERCENTILE"
+            ]
+          }
+        ]
+      }
+    ]
+  },
+  "aggregation_groups": [
+    {
+      "includes": [
+        "TEST_KYLIN_FACT.CAL_DT",
+        "TEST_KYLIN_FACT.LEAF_CATEG_ID",
+        "TEST_KYLIN_FACT.LSTG_FORMAT_NAME",
+        "TEST_KYLIN_FACT.LSTG_SITE_ID",
+        "TEST_KYLIN_FACT.SLR_SEGMENT_CD",
+        "TEST_CATEGORY_GROUPINGS.META_CATEG_NAME",
+        "TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME",
+        "TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME"
+      ],
+      "select_rule": {
+        "hierarchy_dims": [
+          [
+            "TEST_CATEGORY_GROUPINGS.META_CATEG_NAME",
+            "TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME",
+            "TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME",
+            "TEST_KYLIN_FACT.LEAF_CATEG_ID"
+          ]
+        ],
+        "mandatory_dims": [],
+        "joint_dims": [
+          [
+            "TEST_KYLIN_FACT.LSTG_FORMAT_NAME",
+            "TEST_KYLIN_FACT.LSTG_SITE_ID",
+            "TEST_KYLIN_FACT.SLR_SEGMENT_CD"
+          ]
+        ],
+        "dim_cap": 5
+      }
+    },
+    {
+      "includes": [
+        "TEST_KYLIN_FACT.CAL_DT",
+        "TEST_KYLIN_FACT.LEAF_CATEG_ID",
+        "TEST_KYLIN_FACT.LSTG_FORMAT_NAME",
+        "TEST_KYLIN_FACT.LSTG_SITE_ID",
+        "TEST_KYLIN_FACT.SLR_SEGMENT_CD",
+        "TEST_CATEGORY_GROUPINGS.META_CATEG_NAME",
+        "TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME",
+        "TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME",
+        "TEST_KYLIN_FACT.SELLER_ID",
+        "SELLER_ACCOUNT.ACCOUNT_BUYER_LEVEL",
+        "SELLER_ACCOUNT.ACCOUNT_SELLER_LEVEL",
+        "SELLER_ACCOUNT.ACCOUNT_COUNTRY",
+        "SELLER_COUNTRY.NAME",
+        "TEST_KYLIN_FACT.ORDER_ID",
+        "TEST_ORDER.TEST_DATE_ENC",
+        "TEST_ORDER.TEST_TIME_ENC",
+        "TEST_ORDER.BUYER_ID",
+        "BUYER_ACCOUNT.ACCOUNT_BUYER_LEVEL",
+        "BUYER_ACCOUNT.ACCOUNT_SELLER_LEVEL",
+        "BUYER_ACCOUNT.ACCOUNT_COUNTRY",
+        "BUYER_COUNTRY.NAME"
+      ],
+      "select_rule": {
+        "hierarchy_dims": [],
+        "mandatory_dims": [
+          "TEST_KYLIN_FACT.CAL_DT"
+        ],
+        "joint_dims": [
+          [
+            "TEST_CATEGORY_GROUPINGS.META_CATEG_NAME",
+            "TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME",
+            "TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME",
+            "TEST_KYLIN_FACT.LEAF_CATEG_ID"
+          ],
+          [
+            "TEST_KYLIN_FACT.LSTG_FORMAT_NAME",
+            "TEST_KYLIN_FACT.LSTG_SITE_ID",
+            "TEST_KYLIN_FACT.SLR_SEGMENT_CD"
+          ],
+          [
+            "TEST_KYLIN_FACT.SELLER_ID",
+            "SELLER_ACCOUNT.ACCOUNT_BUYER_LEVEL",
+            "SELLER_ACCOUNT.ACCOUNT_SELLER_LEVEL",
+            "SELLER_ACCOUNT.ACCOUNT_COUNTRY",
+            "SELLER_COUNTRY.NAME"
+          ],
+          [
+            "TEST_KYLIN_FACT.ORDER_ID",
+            "TEST_ORDER.TEST_DATE_ENC",
+            "TEST_ORDER.TEST_TIME_ENC",
+            "TEST_ORDER.BUYER_ID",
+            "BUYER_ACCOUNT.ACCOUNT_BUYER_LEVEL",
+            "BUYER_ACCOUNT.ACCOUNT_SELLER_LEVEL",
+            "BUYER_ACCOUNT.ACCOUNT_COUNTRY",
+            "BUYER_COUNTRY.NAME"
+          ]
+        ],
+        "dim_cap": 3
+      }
+    }
+  ],
+  "notify_list": null,
+  "status_need_notify": [],
+  "auto_merge_time_ranges": null,
+  "retention_range": 0,
+  "engine_type": 2,
+  "storage_type": 2,
+  "override_kylin_properties": {
+    "kylin.cube.algorithm": "INMEM",
+    "kylin.dictionary.shrunken-from-global-enabled": "true"
+  },
+  "snapshot_table_desc_list": [
+    {
+      "table_name": "TEST.TEST_CATEGORY_GROUPINGS",
+      "storage_type": "hbase",
+      "local_cache_enable": true,
+      "global": true
+    }
+  ],
+  "partition_date_start": 0
+}
diff --git a/server-base/src/test/resources/update/model_desc/ci_inner_join_model.json b/server-base/src/test/resources/update/model_desc/ci_inner_join_model.json
new file mode 100644
index 0000000..8a74b36
--- /dev/null
+++ b/server-base/src/test/resources/update/model_desc/ci_inner_join_model.json
@@ -0,0 +1,238 @@
+{
+  "uuid": "72ab4ee2-2cdb-4b07-b39e-4c298563ae27",
+  "name": "ci_inner_join_model",
+  "fact_table": "TEST.KYLIN_FACT",
+  "fact_table_alias": "TEST_KYLIN_FACT",
+  "lookups": [
+    {
+      "table": "DEFAULT.TEST_ORDER",
+      "alias": "TEST_ORDER",
+      "kind": "FACT",
+      "join": {
+        "type": "INNER",
+        "primary_key": [
+          "TEST_ORDER.ORDER_ID"
+        ],
+        "foreign_key": [
+          "TEST_KYLIN_FACT.ORDER_ID"
+        ]
+      }
+    },
+    {
+      "table": "TEST.TEST_ACCOUNT",
+      "alias": "BUYER_ACCOUNT",
+      "kind": "FACT",
+      "join": {
+        "type": "INNER",
+        "primary_key": [
+          "BUYER_ACCOUNT.ACCOUNT_ID"
+        ],
+        "foreign_key": [
+          "TEST_ORDER.BUYER_ID"
+        ]
+      }
+    },
+    {
+      "table": "EDW.CAL_DT",
+      "alias": "TEST_CAL_DT",
+      "join": {
+        "type": "INNER",
+        "primary_key": [
+          "TEST_CAL_DT.CAL_DT"
+        ],
+        "foreign_key": [
+          "TEST_KYLIN_FACT.CAL_DT"
+        ]
+      }
+    },
+    {
+      "table": "TEST.TEST_CATEGORY_GROUPINGS",
+      "alias": "TEST_CATEGORY_GROUPINGS",
+      "join": {
+        "type": "INNER",
+        "primary_key": [
+          "TEST_CATEGORY_GROUPINGS.LEAF_CATEG_ID",
+          "TEST_CATEGORY_GROUPINGS.SITE_ID"
+        ],
+        "foreign_key": [
+          "TEST_KYLIN_FACT.LEAF_CATEG_ID",
+          "TEST_KYLIN_FACT.LSTG_SITE_ID"
+        ]
+      }
+    },
+    {
+      "table": "EDW.TEST_SITES",
+      "alias": "TEST_SITES",
+      "join": {
+        "type": "INNER",
+        "primary_key": [
+          "TEST_SITES.SITE_ID"
+        ],
+        "foreign_key": [
+          "TEST_KYLIN_FACT.LSTG_SITE_ID"
+        ]
+      }
+    },
+    {
+      "table": "EDW.TEST_SELLER_TYPE_DIM",
+      "alias": "TEST_SELLER_TYPE_DIM",
+      "join": {
+        "type": "INNER",
+        "primary_key": [
+          "TEST_SELLER_TYPE_DIM.SELLER_TYPE_CD"
+        ],
+        "foreign_key": [
+          "TEST_KYLIN_FACT.SLR_SEGMENT_CD"
+        ]
+      }
+    },
+    {
+      "table": "TEST.COUNTRY",
+      "alias": "BUYER_COUNTRY",
+      "join": {
+        "type": "INNER",
+        "primary_key": [
+          "BUYER_COUNTRY.COUNTRY"
+        ],
+        "foreign_key": [
+          "BUYER_ACCOUNT.ACCOUNT_COUNTRY"
+        ]
+      }
+    },
+    {
+      "table": "TEST.COUNTRY",
+      "alias": "SELLER_COUNTRY",
+      "join": {
+        "type": "INNER",
+        "primary_key": [
+          "SELLER_COUNTRY.COUNTRY"
+        ],
+        "foreign_key": [
+          "SELLER_ACCOUNT.ACCOUNT_COUNTRY"
+        ]
+      }
+    },
+    {
+      "table": "TEST.TEST_ACCOUNT",
+      "alias": "SELLER_ACCOUNT",
+      "kind": "FACT",
+      "join": {
+        "type": "INNER",
+        "primary_key": [
+          "SELLER_ACCOUNT.ACCOUNT_ID"
+        ],
+        "foreign_key": [
+          "TEST_KYLIN_FACT.SELLER_ID"
+        ]
+      }
+    }
+  ],
+  "dimensions": [
+    {
+      "table": "TEST_KYLIN_FACT",
+      "columns": [
+        "TRANS_ID",
+        "ORDER_ID",
+        "CAL_DT",
+        "LSTG_FORMAT_NAME",
+        "LSTG_SITE_ID",
+        "LEAF_CATEG_ID",
+        "SLR_SEGMENT_CD",
+        "SELLER_ID",
+        "TEST_COUNT_DISTINCT_BITMAP"
+      ]
+    },
+    {
+      "table": "TEST_ORDER",
+      "columns": [
+        "ORDER_ID",
+        "BUYER_ID",
+        "TEST_DATE_ENC",
+        "TEST_TIME_ENC",
+        "TEST_EXTENDED_COLUMN"
+      ]
+    },
+    {
+      "table": "BUYER_ACCOUNT",
+      "columns": [
+        "ACCOUNT_ID",
+        "ACCOUNT_BUYER_LEVEL",
+        "ACCOUNT_SELLER_LEVEL",
+        "ACCOUNT_COUNTRY",
+        "ACCOUNT_CONTACT"
+      ]
+    },
+    {
+      "table": "SELLER_ACCOUNT",
+      "columns": [
+        "ACCOUNT_ID",
+        "ACCOUNT_BUYER_LEVEL",
+        "ACCOUNT_SELLER_LEVEL",
+        "ACCOUNT_COUNTRY",
+        "ACCOUNT_CONTACT"
+      ]
+    },
+    {
+      "table": "TEST_CATEGORY_GROUPINGS",
+      "columns": [
+        "LEAF_CATEG_ID",
+        "SITE_ID",
+        "META_CATEG_NAME",
+        "CATEG_LVL2_NAME",
+        "CATEG_LVL3_NAME",
+        "USER_DEFINED_FIELD1",
+        "USER_DEFINED_FIELD3",
+        "UPD_DATE",
+        "UPD_USER"
+      ]
+    },
+    {
+      "table": "TEST_SITES",
+      "columns": [
+        "SITE_ID",
+        "SITE_NAME",
+        "CRE_USER"
+      ]
+    },
+    {
+      "table": "TEST_SELLER_TYPE_DIM",
+      "columns": [
+        "SELLER_TYPE_CD",
+        "SELLER_TYPE_DESC"
+      ]
+    },
+    {
+      "table": "TEST_CAL_DT",
+      "columns": [
+        "CAL_DT",
+        "WEEK_BEG_DT"
+      ]
+    },
+    {
+      "table": "BUYER_COUNTRY",
+      "columns": [
+        "COUNTRY",
+        "NAME"
+      ]
+    },
+    {
+      "table": "SELLER_COUNTRY",
+      "columns": [
+        "COUNTRY",
+        "NAME"
+      ]
+    }
+  ],
+  "metrics": [
+    "TEST_KYLIN_FACT.TEST_COUNT_COLUMN",
+    "TEST_KYLIN_FACT.PRICE",
+    "TEST_KYLIN_FACT.ITEM_COUNT"
+  ],
+  "last_modified": 1422435345352,
+  "filter_condition": null,
+  "partition_desc": {
+    "partition_date_column": "TEST.KYLIN_FACT.CAL_DT",
+    "partition_date_start": 0,
+    "partition_type": "APPEND"
+  }
+}
diff --git a/server-base/src/test/resources/update/table/EDW.CAL_DT.json b/server-base/src/test/resources/update/table/EDW.CAL_DT.json
new file mode 100644
index 0000000..8cbcec8
--- /dev/null
+++ b/server-base/src/test/resources/update/table/EDW.CAL_DT.json
@@ -0,0 +1,413 @@
+{
+ 
+  "uuid" : "0ff420eb-79ad-40bd-bca9-12d8cd05c60a",
+  "name" : "CAL_DT",
+  "columns" : [ {
+    "id" : "1",
+    "name" : "CAL_DT",
+    "datatype" : "date",
+    "index": "T"
+  }, {
+    "id" : "2",
+    "name" : "YEAR_BEG_DT",
+    "datatype" : "date",
+    "index": "T"
+  }, {
+    "id" : "3",
+    "name" : "QTR_BEG_DT",
+    "datatype" : "date",
+    "index": "T"
+  }, {
+    "id" : "4",
+    "name" : "MONTH_BEG_DT",
+    "datatype" : "date",
+    "index": "T"
+  }, {
+    "id" : "5",
+    "name" : "WEEK_BEG_DT",
+    "datatype" : "date",
+    "index": "T"
+  }, {
+    "id" : "6",
+    "name" : "AGE_FOR_YEAR_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "7",
+    "name" : "AGE_FOR_QTR_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "8",
+    "name" : "AGE_FOR_MONTH_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "9",
+    "name" : "AGE_FOR_WEEK_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "10",
+    "name" : "AGE_FOR_DT_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "11",
+    "name" : "AGE_FOR_RTL_YEAR_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "12",
+    "name" : "AGE_FOR_RTL_QTR_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "13",
+    "name" : "AGE_FOR_RTL_MONTH_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "14",
+    "name" : "AGE_FOR_RTL_WEEK_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "15",
+    "name" : "AGE_FOR_CS_WEEK_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "16",
+    "name" : "DAY_OF_CAL_ID",
+    "datatype" : "integer"
+  }, {
+    "id" : "17",
+    "name" : "DAY_OF_YEAR_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "18",
+    "name" : "DAY_OF_QTR_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "19",
+    "name" : "DAY_OF_MONTH_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "20",
+    "name" : "DAY_OF_WEEK_ID",
+    "datatype" : "integer"
+  }, {
+    "id" : "21",
+    "name" : "WEEK_OF_YEAR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "22",
+    "name" : "WEEK_OF_CAL_ID",
+    "datatype" : "integer"
+  }, {
+    "id" : "23",
+    "name" : "MONTH_OF_QTR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "24",
+    "name" : "MONTH_OF_YEAR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "25",
+    "name" : "MONTH_OF_CAL_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "26",
+    "name" : "QTR_OF_YEAR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "27",
+    "name" : "QTR_OF_CAL_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "28",
+    "name" : "YEAR_OF_CAL_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "29",
+    "name" : "YEAR_END_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "30",
+    "name" : "QTR_END_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "31",
+    "name" : "MONTH_END_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "32",
+    "name" : "WEEK_END_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "33",
+    "name" : "CAL_DT_NAME",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "34",
+    "name" : "CAL_DT_DESC",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "35",
+    "name" : "CAL_DT_SHORT_NAME",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "36",
+    "name" : "YTD_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "37",
+    "name" : "QTD_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "38",
+    "name" : "MTD_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "39",
+    "name" : "WTD_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "40",
+    "name" : "SEASON_BEG_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "41",
+    "name" : "DAY_IN_YEAR_COUNT",
+    "datatype" : "smallint"
+  }, {
+    "id" : "42",
+    "name" : "DAY_IN_QTR_COUNT",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "43",
+    "name" : "DAY_IN_MONTH_COUNT",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "44",
+    "name" : "DAY_IN_WEEK_COUNT",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "45",
+    "name" : "RTL_YEAR_BEG_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "46",
+    "name" : "RTL_QTR_BEG_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "47",
+    "name" : "RTL_MONTH_BEG_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "48",
+    "name" : "RTL_WEEK_BEG_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "49",
+    "name" : "CS_WEEK_BEG_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "50",
+    "name" : "CAL_DATE",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "51",
+    "name" : "DAY_OF_WEEK",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "52",
+    "name" : "MONTH_ID",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "53",
+    "name" : "PRD_DESC",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "54",
+    "name" : "PRD_FLAG",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "55",
+    "name" : "PRD_ID",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "56",
+    "name" : "PRD_IND",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "57",
+    "name" : "QTR_DESC",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "58",
+    "name" : "QTR_ID",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "59",
+    "name" : "QTR_IND",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "60",
+    "name" : "RETAIL_WEEK",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "61",
+    "name" : "RETAIL_YEAR",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "62",
+    "name" : "RETAIL_START_DATE",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "63",
+    "name" : "RETAIL_WK_END_DATE",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "64",
+    "name" : "WEEK_IND",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "65",
+    "name" : "WEEK_NUM_DESC",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "66",
+    "name" : "WEEK_BEG_DATE",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "67",
+    "name" : "WEEK_END_DATE",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "68",
+    "name" : "WEEK_IN_YEAR_ID",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "69",
+    "name" : "WEEK_ID",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "70",
+    "name" : "WEEK_BEG_END_DESC_MDY",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "71",
+    "name" : "WEEK_BEG_END_DESC_MD",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "72",
+    "name" : "YEAR_ID",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "73",
+    "name" : "YEAR_IND",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "74",
+    "name" : "CAL_DT_MNS_1YEAR_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "75",
+    "name" : "CAL_DT_MNS_2YEAR_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "76",
+    "name" : "CAL_DT_MNS_1QTR_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "77",
+    "name" : "CAL_DT_MNS_2QTR_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "78",
+    "name" : "CAL_DT_MNS_1MONTH_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "79",
+    "name" : "CAL_DT_MNS_2MONTH_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "80",
+    "name" : "CAL_DT_MNS_1WEEK_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "81",
+    "name" : "CAL_DT_MNS_2WEEK_DT",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "82",
+    "name" : "CURR_CAL_DT_MNS_1YEAR_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "83",
+    "name" : "CURR_CAL_DT_MNS_2YEAR_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "84",
+    "name" : "CURR_CAL_DT_MNS_1QTR_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "85",
+    "name" : "CURR_CAL_DT_MNS_2QTR_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "86",
+    "name" : "CURR_CAL_DT_MNS_1MONTH_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "87",
+    "name" : "CURR_CAL_DT_MNS_2MONTH_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "88",
+    "name" : "CURR_CAL_DT_MNS_1WEEK_YN_IND",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "89",
+    "name" : "CURR_CAL_DT_MNS_2WEEK_YN_IND",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "90",
+    "name" : "RTL_MONTH_OF_RTL_YEAR_ID",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "91",
+    "name" : "RTL_QTR_OF_RTL_YEAR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "92",
+    "name" : "RTL_WEEK_OF_RTL_YEAR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "93",
+    "name" : "SEASON_OF_YEAR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "94",
+    "name" : "YTM_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "95",
+    "name" : "YTQ_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "96",
+    "name" : "YTW_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "97",
+    "name" : "CAL_DT_CRE_DATE",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "98",
+    "name" : "CAL_DT_CRE_USER",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "99",
+    "name" : "CAL_DT_UPD_DATE",
+    "datatype" : "varchar(256)"
+  }, {
+    "id" : "100",
+    "name" : "CAL_DT_UPD_USER",
+    "datatype" : "varchar(256)"
+  } ],
+  "database" : "edw",
+  "last_modified" : 0
+}
\ No newline at end of file
diff --git a/server-base/src/test/resources/update/table/TEST.COUNTRY.json b/server-base/src/test/resources/update/table/TEST.COUNTRY.json
new file mode 100644
index 0000000..b6c1ee0
--- /dev/null
+++ b/server-base/src/test/resources/update/table/TEST.COUNTRY.json
@@ -0,0 +1,26 @@
+{
+  "uuid" : "e286e39e-40d7-44c2-8fa2-41b365632882",
+ 
+  "name" : "COUNTRY",
+  "columns" : [ {
+    "id" : "1",
+    "name" : "COUNTRY",
+    "datatype" : "varchar(256)",
+    "index": "T"
+  }, {
+    "id" : "2",
+    "name" : "LATITUDE",
+    "datatype" : "double"
+  }, {
+    "id" : "3",
+    "name" : "LONGITUDE",
+    "datatype" : "double"
+  }, {
+    "id" : "4",
+    "name" : "NAME",
+    "datatype" : "varchar(256)",
+    "index": "T"
+  } ],
+  "database" : "TEST",
+  "last_modified" : 0
+}
\ No newline at end of file
diff --git a/server-base/src/test/resources/update/table/TEST.KYLIN_FACT.json b/server-base/src/test/resources/update/table/TEST.KYLIN_FACT.json
new file mode 100644
index 0000000..879f969
--- /dev/null
+++ b/server-base/src/test/resources/update/table/TEST.KYLIN_FACT.json
@@ -0,0 +1,73 @@
+{
+  "uuid" : "e286e39e-40d7-44c2-8fa2-41b365522771",
+  "name" : "KYLIN_FACT",
+  "data_gen" : "1",
+  "columns" : [ {
+    "id" : "1",
+    "name" : "TRANS_ID",
+    "datatype" : "bigint",
+    "data_gen" : "ID"
+  }, {
+    "id" : "2",
+    "name" : "ORDER_ID",
+    "datatype" : "bigint",
+    "index": "T"
+  }, {
+    "id" : "3",
+    "name" : "CAL_DT",
+    "datatype" : "date",
+    "data_gen" : "FK,order",
+    "index": "T"
+  }, {
+    "id" : "4",
+    "name" : "LSTG_FORMAT_NAME",
+    "datatype" : "varchar(256)",
+    "data_gen" : "FP-GTC|FP-non GTC|ABIN|Auction|Others",
+    "index": "T"
+  }, {
+    "id" : "5",
+    "name" : "LEAF_CATEG_ID",
+    "datatype" : "bigint",
+    "data_gen" : "FK,null,nullstr=0",
+    "index": "T"
+  }, {
+    "id" : "6",
+    "name" : "LSTG_SITE_ID",
+    "datatype" : "integer",
+    "index": "T"
+  }, {
+    "id" : "7",
+    "name" : "SLR_SEGMENT_CD",
+    "datatype" : "smallint",
+    "data_gen" : "FK,pk=EDW.TEST_SELLER_TYPE_DIM_TABLE.SELLER_TYPE_CD",
+    "index": "T"
+  }, {
+    "id" : "8",
+    "name" : "SELLER_ID",
+    "datatype" : "integer",
+    "data_gen" : "RAND||10000000|10001000",
+    "index": "T"
+  }, {
+    "id" : "9",
+    "name" : "PRICE",
+    "datatype" : "decimal(19,4)",
+    "data_gen" : "RAND|.##|-100|1000"
+  }, {
+    "id" : "10",
+    "name" : "ITEM_COUNT",
+    "datatype" : "integer",
+    "data_gen" : "RAND"
+  }, {
+    "id" : "11",
+    "name" : "TEST_COUNT_DISTINCT_BITMAP",
+    "datatype" : "varchar(256)",
+    "data_gen" : "RAND"
+  }, {
+    "id" : "12",
+    "name" : "TEST_COUNT_COLUMN",
+    "datatype" : "varchar(256)",
+    "data_gen" : "RAND,null=true,nullpct=0.2"
+  } ],
+  "database" : "TEST",
+  "last_modified" : 0
+}
diff --git a/server-base/src/test/resources/update/table/TEST.TEST_ACCOUNT.json b/server-base/src/test/resources/update/table/TEST.TEST_ACCOUNT.json
new file mode 100644
index 0000000..8facbea
--- /dev/null
+++ b/server-base/src/test/resources/update/table/TEST.TEST_ACCOUNT.json
@@ -0,0 +1,36 @@
+{
+  "uuid" : "f386e39e-40d7-44c2-9eb3-41b365632231",
+ 
+  "name" : "TEST_ACCOUNT",
+  "data_gen" : "2000",
+  
+  "columns" : [ {
+    "id" : "1",
+    "name" : "ACCOUNT_ID",
+    "datatype" : "bigint",
+    "data_gen" : "ID|10000000",
+    "index": "T"
+  }, {
+    "id" : "2",
+    "name" : "ACCOUNT_BUYER_LEVEL",
+    "datatype" : "integer",
+    "data_gen" : "RAND||0|5"
+  }, {
+    "id" : "3",
+    "name" : "ACCOUNT_SELLER_LEVEL",
+    "datatype" : "integer",
+    "data_gen" : "RAND||0|5"
+  }, {
+    "id" : "4",
+    "name" : "ACCOUNT_COUNTRY",
+    "datatype" : "varchar(256)",
+    "data_gen" : "CN|FR|GB|GE|JP|IT|RU|US",
+    "index": "T"
+  }, {
+    "id" : "5",
+    "name" : "ACCOUNT_CONTACT",
+    "datatype" : "varchar(256)"
+  } ],
+  "database" : "TEST",
+  "last_modified" : 0
+}
\ No newline at end of file
diff --git a/server-base/src/test/resources/update/table/TEST.TEST_CATEGORY_GROUPINGS.json b/server-base/src/test/resources/update/table/TEST.TEST_CATEGORY_GROUPINGS.json
new file mode 100644
index 0000000..bc7eebb
--- /dev/null
+++ b/server-base/src/test/resources/update/table/TEST.TEST_CATEGORY_GROUPINGS.json
@@ -0,0 +1,155 @@
+{
+ 
+  "uuid" : "952d11b5-69d9-45d1-92af-227489485e3f",
+  "name" : "TEST_CATEGORY_GROUPINGS",
+  "columns" : [ {
+    "id" : "1",
+    "name" : "LEAF_CATEG_ID",
+    "datatype" : "bigint",
+    "index": "T"
+  }, {
+    "id" : "2",
+    "name" : "LEAF_CATEG_NAME",
+    "datatype" : "string",
+    "index": "T"
+  }, {
+    "id" : "3",
+    "name" : "SITE_ID",
+    "datatype" : "int",
+    "index": "T"
+  }, {
+    "id" : "4",
+    "name" : "CATEG_BUSN_MGR",
+    "datatype" : "string"
+  }, {
+    "id" : "5",
+    "name" : "CATEG_BUSN_UNIT",
+    "datatype" : "string"
+  }, {
+    "id" : "6",
+    "name" : "REGN_CATEG",
+    "datatype" : "string"
+  }, {
+    "id" : "7",
+    "name" : "USER_DEFINED_FIELD1",
+    "datatype" : "string"
+  }, {
+    "id" : "8",
+    "name" : "USER_DEFINED_FIELD3",
+    "datatype" : "string"
+  }, {
+    "id" : "9",
+    "name" : "GROUPINGS_CRE_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "10",
+    "name" : "UPD_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "11",
+    "name" : "GROUPINGS_CRE_USER",
+    "datatype" : "string"
+  }, {
+    "id" : "12",
+    "name" : "UPD_USER",
+    "datatype" : "string"
+  }, {
+    "id" : "13",
+    "name" : "META_CATEG_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "14",
+    "name" : "META_CATEG_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "15",
+    "name" : "CATEG_LVL2_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "16",
+    "name" : "CATEG_LVL3_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "17",
+    "name" : "CATEG_LVL4_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "18",
+    "name" : "CATEG_LVL5_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "19",
+    "name" : "CATEG_LVL6_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "20",
+    "name" : "CATEG_LVL7_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "21",
+    "name" : "CATEG_LVL2_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "22",
+    "name" : "CATEG_LVL3_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "23",
+    "name" : "CATEG_LVL4_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "24",
+    "name" : "CATEG_LVL5_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "25",
+    "name" : "CATEG_LVL6_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "26",
+    "name" : "CATEG_LVL7_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "27",
+    "name" : "CATEG_FLAGS",
+    "datatype" : "decimal"
+  }, {
+    "id" : "28",
+    "name" : "ADULT_CATEG_YN",
+    "datatype" : "string"
+  }, {
+    "id" : "29",
+    "name" : "DOMAIN_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "30",
+    "name" : "USER_DEFINED_FIELD5",
+    "datatype" : "string"
+  }, {
+    "id" : "31",
+    "name" : "VCS_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "32",
+    "name" : "GCS_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "33",
+    "name" : "MOVE_TO",
+    "datatype" : "decimal"
+  }, {
+    "id" : "34",
+    "name" : "SAP_CATEGORY_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "35",
+    "name" : "SRC_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "36",
+    "name" : "BSNS_VRTCL_NAME",
+    "datatype" : "string"
+  } ],
+  "database" : "TEST",
+  "last_modified" : 0
+}
\ No newline at end of file


[kylin] 08/15: KYLIN-4487 Backend support for auto-migration to allow user to do cube migration by self service

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit a0ef1895232ab36c3155041dc65a53bd218df603
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Mon May 11 15:56:11 2020 +0800

    KYLIN-4487 Backend support for auto-migration to allow user to do cube migration by self service
---
 .../org/apache/kylin/common/KylinConfigBase.java   |  39 ++
 .../apache/kylin/common/restclient/RestClient.java |  34 +-
 .../java/org/apache/kylin/job/JoinedFlatTable.java |   3 +-
 cube-migration/pom.xml                             | 169 ++++++++
 .../kylin/rest/controller/MigrationController.java | 167 ++++++++
 .../kylin/rest/exception/ConflictException.java    |  44 ++
 .../rest/exception/RuleValidationException.java    |  47 +++
 .../kylin/rest/request/MigrationRequest.java       |  49 +++
 .../kylin/rest/service/MigrationRuleSet.java       | 469 +++++++++++++++++++++
 .../kylin/rest/service/MigrationService.java       | 225 ++++++++++
 .../kylin/rest/util/MailNotificationUtil.java      |  33 ++
 .../mail_templates/MIGRATION_APPROVED.ftl          | 189 +++++++++
 .../mail_templates/MIGRATION_COMPLETED.ftl         | 192 +++++++++
 .../resources/mail_templates/MIGRATION_FAILED.ftl  | 220 ++++++++++
 .../mail_templates/MIGRATION_REJECTED.ftl          | 196 +++++++++
 .../resources/mail_templates/MIGRATION_REQUEST.ftl | 192 +++++++++
 pom.xml                                            |   6 +
 .../rest/service/ModelSchemaUpdateChecker.java     |   8 +-
 .../apache/kylin/rest/service/ModelService.java    |  21 +-
 .../rest/service/TableSchemaUpdateChecker.java     |   4 +-
 .../apache/kylin/rest/service/TableService.java    |   9 +
 server/pom.xml                                     |  14 +
 server/src/main/resources/kylinSecurity.xml        |   3 +
 .../tool/migration/CompatibilityCheckRequest.java  |  51 +++
 .../kylin/tool/query/ProbabilityGenerator.java     |  88 ++++
 .../kylin/tool/query/ProbabilityGeneratorCLI.java  |  99 +++++
 .../apache/kylin/tool/query/QueryGenerator.java    | 189 +++++++++
 .../apache/kylin/tool/query/QueryGeneratorCLI.java | 138 ++++++
 .../tool/query/ProbabilityGeneratorCLITest.java    |  50 +++
 .../kylin/tool/query/QueryGeneratorCLITest.java    |  54 +++
 30 files changed, 2991 insertions(+), 11 deletions(-)

diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 13e73d9..ff6138a 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -2361,6 +2361,45 @@ public abstract class KylinConfigBase implements Serializable {
     }
 
     // ============================================================================
+    // CUBE MIGRATION
+    // ============================================================================
+    public int getMigrationRuleExpansionRateThreshold() {
+        return Integer.parseInt(getOptional("kylin.cube.migration.expansion-rate", "5"));
+    }
+
+    public int getMigrationRuleQueryGeneratorMaxDimensions() {
+        return Integer.parseInt(getOptional("kylin.cube.migration.query-generator-max-dimension-number", "3"));
+    }
+
+    public int getMigrationRuleQueryLatency() {
+        return 1000 * Integer.parseInt(getOptional("kylin.cube.migration.query-latency-seconds", "2"));
+    }
+
+    public int getMigrationRuleQueryLatencyMaxThreads() {
+        return Integer.parseInt(getOptional("kylin.cube.migration.query-latency-max-threads", "5"));
+    }
+
+    public int getMigrationRuleQueryEvaluationIteration() {
+        return Integer.parseInt(getOptional("kylin.cube.migration.query-latency-iteration", "5"));
+    }
+
+    public String getMigrationLocalAddress() {
+        return getOptional("kylin.cube.migration.source-address", "localhost:80");
+    }
+
+    public String getMigrationTargetAddress() {
+        return getOptional("kylin.cube.migration.target-address", "sandbox:80");
+    }
+
+    public String getMigrationEmailSuffix() {
+        return getOptional("kylin.cube.migration.mail-suffix", "@mail.com");
+    }
+
+    public boolean isMigrationApplyQueryLatencyRule() {
+        return Boolean.parseBoolean(getOptional("kylin.cube.migration.rule-query-latency-enabled", "true"));
+    }
+
+    // ============================================================================
     // tool
     // ============================================================================
     public boolean isAllowAutoMigrateCube() {
diff --git a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
index 955b0ff..a9971dd 100644
--- a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
+++ b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
@@ -30,6 +30,7 @@ import java.util.regex.Pattern;
 
 import javax.xml.bind.DatatypeConverter;
 
+import com.google.common.base.Strings;
 import org.apache.http.HttpResponse;
 import org.apache.http.auth.AuthScope;
 import org.apache.http.auth.UsernamePasswordCredentials;
@@ -207,12 +208,13 @@ public class RestClient {
         try {
             response = client.execute(request);
             String msg = EntityUtils.toString(response.getEntity());
-            Map<String, String> map = JsonUtil.readValueAsMap(msg);
-            msg = map.get("config");
 
             if (response.getStatusLine().getStatusCode() != 200)
                 throw new IOException(INVALID_RESPONSE + response.getStatusLine().getStatusCode()
                         + " with cache wipe url " + url + "\n" + msg);
+
+            Map<String, String> map = JsonUtil.readValueAsMap(msg);
+            msg = map.get("config");
             return msg;
         } finally {
             cleanup(request, response);
@@ -350,6 +352,26 @@ public class RestClient {
         return content;
     }
 
+    public void checkCompatibility(String jsonRequest) throws IOException {
+        checkCompatibility(jsonRequest, baseUrl + "/cubes/checkCompatibility");
+    }
+
+    private void checkCompatibility(String jsonRequest, String url) throws IOException {
+        HttpPost post = newPost(url);
+        try {
+            post.setEntity(new StringEntity(jsonRequest, "UTF-8"));
+            HttpResponse response = client.execute(post);
+            if (response.getStatusLine().getStatusCode() != 200) {
+                String msg = getContent(response);
+                Map<String, String> kvMap = JsonUtil.readValueAsMap(msg);
+                String exception = kvMap.containsKey("exception") ? kvMap.get("exception") : "unknown";
+                throw new IOException(exception);
+            }
+        } finally {
+            post.releaseConnection();
+        }
+    }
+    
     private HashMap dealResponse(HttpResponse response) throws IOException {
         if (response.getStatusLine().getStatusCode() != 200) {
             throw new IOException(INVALID_RESPONSE + response.getStatusLine().getStatusCode());
@@ -362,9 +384,11 @@ public class RestClient {
     private void addHttpHeaders(HttpRequestBase method) {
         method.addHeader("Accept", "application/json, text/plain, */*");
         method.addHeader("Content-Type", APPLICATION_JSON);
-        String basicAuth = DatatypeConverter
-                .printBase64Binary((this.userName + ":" + this.password).getBytes(StandardCharsets.UTF_8));
-        method.addHeader("Authorization", "Basic " + basicAuth);
+        if (!Strings.isNullOrEmpty(this.userName) && !Strings.isNullOrEmpty(this.password)) {
+            String basicAuth = DatatypeConverter
+                    .printBase64Binary((this.userName + ":" + this.password).getBytes(StandardCharsets.UTF_8));
+            method.addHeader("Authorization", "Basic " + basicAuth);
+        }
     }
 
     private HttpPost newPost(String url) {
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 2c4bc6a..27e25ba 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
@@ -161,7 +161,8 @@ public class JoinedFlatTable {
         return sql.toString();
     }
 
-    static void appendJoinStatement(IJoinedFlatTableDesc flatDesc, StringBuilder sql, boolean singleLine, SqlDialect sqlDialect) {
+    public static void appendJoinStatement(IJoinedFlatTableDesc flatDesc, StringBuilder sql, boolean singleLine,
+            SqlDialect sqlDialect) {
         final String sep = singleLine ? " " : "\n";
         Set<TableRef> dimTableCache = new HashSet<>();
 
diff --git a/cube-migration/pom.xml b/cube-migration/pom.xml
new file mode 100755
index 0000000..4f75de5
--- /dev/null
+++ b/cube-migration/pom.xml
@@ -0,0 +1,169 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>kylin-cube-migration</artifactId>
+    <packaging>jar</packaging>
+    <name>Apache Kylin - Cube Migration Service</name>
+    <description>Apache Kylin - Cube Migration Service</description>
+
+    <parent>
+        <groupId>org.apache.kylin</groupId>
+        <artifactId>kylin</artifactId>
+        <version>3.1.0-SNAPSHOT</version>
+    </parent>
+
+    <dependencies>
+    
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-server-base</artifactId>
+        </dependency>
+        
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-metadata</artifactId>
+        </dependency>
+        
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-tool</artifactId>
+        </dependency>
+        
+        <!-- Spring Core -->
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-webmvc</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-jdbc</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-aop</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-context-support</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-test</artifactId>
+        </dependency>
+
+        <!-- Spring Security -->
+        <dependency>
+            <groupId>org.springframework.security</groupId>
+            <artifactId>spring-security-acl</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework.security</groupId>
+            <artifactId>spring-security-ldap</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework.security.extensions</groupId>
+            <artifactId>spring-security-saml2-core</artifactId>
+        </dependency>
+
+        <!-- spring aop -->
+        <dependency>
+            <groupId>org.aspectj</groupId>
+            <artifactId>aspectjrt</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.aspectj</groupId>
+            <artifactId>aspectjweaver</artifactId>
+        </dependency>
+
+        <!-- Test & Env -->
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-common</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+            <!--MRUnit relies on older version of mockito, so cannot manage it globally-->
+            <version>${mockito.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-catalina</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-webapp</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <repositories>
+        <repository>
+            <id>spring-snapshots</id>
+            <url>https://repo.spring.io/libs-snapshot</url>
+            <snapshots>
+                <enabled>true</enabled>
+            </snapshots>
+        </repository>
+    </repositories>
+    <pluginRepositories>
+        <pluginRepository>
+            <id>spring-snapshots</id>
+            <url>https://repo.spring.io/libs-snapshot</url>
+            <snapshots>
+                <enabled>true</enabled>
+            </snapshots>
+        </pluginRepository>
+    </pluginRepositories>
+</project>
diff --git a/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java b/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java
new file mode 100644
index 0000000..9588e51
--- /dev/null
+++ b/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java
@@ -0,0 +1,167 @@
+/*
+ * 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.rest.controller;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.JsonUtil;
+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.TableDesc;
+import org.apache.kylin.rest.exception.BadRequestException;
+import org.apache.kylin.rest.exception.ConflictException;
+import org.apache.kylin.rest.exception.InternalErrorException;
+import org.apache.kylin.rest.request.MigrationRequest;
+import org.apache.kylin.rest.service.MigrationRuleSet;
+import org.apache.kylin.rest.service.MigrationService;
+import org.apache.kylin.rest.service.ModelService;
+import org.apache.kylin.rest.service.QueryService;
+import org.apache.kylin.rest.service.TableService;
+import org.apache.kylin.tool.migration.CompatibilityCheckRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestBody;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+
+/**
+ * Restful api for cube migration.
+ */
+@Controller
+@RequestMapping(value = "/cubes")
+public class MigrationController extends BasicController {
+    private static final Logger logger = LoggerFactory.getLogger(MigrationController.class);
+
+    @Autowired
+    private MigrationService migrationService;
+
+    @Autowired
+    private QueryService queryService;
+
+    @Autowired
+    private ModelService modelService;
+
+    @Autowired
+    private TableService tableService;
+
+    private final String targetHost = KylinConfig.getInstanceFromEnv().getMigrationTargetAddress();
+
+    private CubeInstance getCubeInstance(String cubeName) {
+        CubeInstance cube = queryService.getCubeManager().getCube(cubeName);
+        if (cube == null) {
+            throw new BadRequestException("Cannot find cube " + cubeName);
+        }
+        return cube;
+    }
+
+    @RequestMapping(value = "/{cubeName}/migrateRequest", method = { RequestMethod.PUT })
+    @ResponseBody
+    public String requestMigration(@PathVariable String cubeName, @RequestBody MigrationRequest request) {
+        CubeInstance cube = getCubeInstance(cubeName);
+        try {
+            MigrationRuleSet.Context ctx = new MigrationRuleSet.Context(queryService, cube,
+                    getTargetHost(request.getTargetHost()), request.getProjectName());
+            migrationService.requestMigration(cube, ctx);
+        } catch (Exception e) {
+            logger.error("Request migration failed.", e);
+            throw new BadRequestException(e.getMessage());
+        }
+        return "ok";
+    }
+
+    @RequestMapping(value = "/{cubeName}/migrateReject", method = { RequestMethod.PUT })
+    @ResponseBody
+    public void reject(@PathVariable String cubeName, @RequestBody MigrationRequest request) {
+        boolean reject = migrationService.reject(cubeName, request.getProjectName(), request.getReason());
+        if (!reject) {
+            throw new InternalErrorException("Email send out failed. See logs.");
+        }
+    }
+
+    @RequestMapping(value = "/{cubeName}/migrateApprove", method = { RequestMethod.PUT })
+    @ResponseBody
+    public String approve(@PathVariable String cubeName, @RequestBody MigrationRequest request) {
+        CubeInstance cube = getCubeInstance(cubeName);
+        try {
+            MigrationRuleSet.Context ctx = new MigrationRuleSet.Context(queryService, cube,
+                    getTargetHost(request.getTargetHost()), request.getProjectName());
+            migrationService.approve(cube, ctx);
+        } catch (Exception e) {
+            throw new BadRequestException(e.getMessage());
+        }
+        return "Cube " + cubeName + " migrated.";
+    }
+
+    private String getTargetHost(String targetHost) {
+        return Strings.isNullOrEmpty(targetHost) ? this.targetHost : targetHost;
+    }
+
+    /**
+     * Check the schema compatibility for table, model desc
+     */
+    @RequestMapping(value = "/checkCompatibility", method = { RequestMethod.POST })
+    @ResponseBody
+    public void checkCompatibility(@RequestBody CompatibilityCheckRequest request) {
+        try {
+            List<TableDesc> tableDescList = deserializeTableDescList(request);
+            for (TableDesc tableDesc : tableDescList) {
+                logger.info("Schema compatibility check for table {}", tableDesc.getName());
+                tableService.checkTableCompatibility(request.getProjectName(), tableDesc);
+                logger.info("Pass schema compatibility check for table {}", tableDesc.getName());
+            }
+            DataModelDesc dataModelDesc = JsonUtil.readValue(request.getModelDescData(), DataModelDesc.class);
+            logger.info("Schema compatibility check for model {}", dataModelDesc.getName());
+            modelService.checkModelCompatibility(request.getProjectName(), dataModelDesc, tableDescList);
+            logger.info("Pass schema compatibility check for model {}", dataModelDesc.getName());
+        } catch (Exception e) {
+            logger.error(e.getMessage(), e);
+            throw new ConflictException(e.getMessage(), e);
+        }
+    }
+
+    private List<TableDesc> deserializeTableDescList(CompatibilityCheckRequest request) {
+        List<TableDesc> result = Lists.newArrayList();
+        try {
+            for (String tableDescData : request.getTableDescDataList()) {
+                TableDesc tableDesc = JsonUtil.readValue(tableDescData, TableDesc.class);
+                for (ColumnDesc columnDesc : tableDesc.getColumns()) {
+                    columnDesc.init(tableDesc);
+                }
+                result.add(tableDesc);
+            }
+        } catch (JsonParseException | JsonMappingException e) {
+            throw new BadRequestException("Fail to parse table description: " + e);
+        } catch (IOException e) {
+            throw new InternalErrorException("Failed to deal with the request:" + e.getMessage(), e);
+        }
+        return result;
+    }
+}
diff --git a/cube-migration/src/main/java/org/apache/kylin/rest/exception/ConflictException.java b/cube-migration/src/main/java/org/apache/kylin/rest/exception/ConflictException.java
new file mode 100644
index 0000000..c6a8f0a
--- /dev/null
+++ b/cube-migration/src/main/java/org/apache/kylin/rest/exception/ConflictException.java
@@ -0,0 +1,44 @@
+/*
+ * 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.rest.exception;
+
+import org.springframework.http.HttpStatus;
+import org.springframework.web.bind.annotation.ResponseStatus;
+
+@ResponseStatus(value = HttpStatus.CONFLICT)
+public class ConflictException extends RuntimeException {
+    private static final long serialVersionUID = 1L;
+
+    public ConflictException() {
+        super();
+    }
+
+    public ConflictException(String message) {
+        super(message);
+    }
+
+    public ConflictException(Throwable cause) {
+        super(cause);
+    }
+
+    public ConflictException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}
diff --git a/cube-migration/src/main/java/org/apache/kylin/rest/exception/RuleValidationException.java b/cube-migration/src/main/java/org/apache/kylin/rest/exception/RuleValidationException.java
new file mode 100644
index 0000000..5eff71d
--- /dev/null
+++ b/cube-migration/src/main/java/org/apache/kylin/rest/exception/RuleValidationException.java
@@ -0,0 +1,47 @@
+/*
+ * 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.rest.exception;
+
+import org.springframework.http.HttpStatus;
+import org.springframework.web.bind.annotation.ResponseStatus;
+
+/**
+ * Throw RuleValidationException if cube breaks the rules during the migration.
+ */
+@ResponseStatus(value = HttpStatus.BAD_REQUEST)
+public class RuleValidationException extends RuntimeException {
+
+    private static final long serialVersionUID = 1L;
+
+    public RuleValidationException() {
+        super();
+    }
+
+    public RuleValidationException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public RuleValidationException(String message) {
+        super(message);
+    }
+
+    public RuleValidationException(Throwable cause) {
+        super(cause);
+    }
+}
diff --git a/cube-migration/src/main/java/org/apache/kylin/rest/request/MigrationRequest.java b/cube-migration/src/main/java/org/apache/kylin/rest/request/MigrationRequest.java
new file mode 100644
index 0000000..4e8c257
--- /dev/null
+++ b/cube-migration/src/main/java/org/apache/kylin/rest/request/MigrationRequest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.rest.request;
+
+public class MigrationRequest {
+    private String targetHost;
+    private String projectName;
+    private String reason; // reject reason
+
+    public String getTargetHost() {
+        return targetHost;
+    }
+
+    public void setTargetHost(String targetHost) {
+        this.targetHost = targetHost;
+    }
+
+    public String getProjectName() {
+        return projectName;
+    }
+
+    public void setProjectName(String projectName) {
+        this.projectName = projectName;
+    }
+
+    public String getReason() {
+        return reason;
+    }
+
+    public void setReason(String reason) {
+        this.reason = reason;
+    }
+}
diff --git a/cube-migration/src/main/java/org/apache/kylin/rest/service/MigrationRuleSet.java b/cube-migration/src/main/java/org/apache/kylin/rest/service/MigrationRuleSet.java
new file mode 100644
index 0000000..de158bd
--- /dev/null
+++ b/cube-migration/src/main/java/org/apache/kylin/rest/service/MigrationRuleSet.java
@@ -0,0 +1,469 @@
+/*
+ * 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.rest.service;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.restclient.RestClient;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.common.util.Pair;
+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.TableMetadataManager;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.DataModelManager;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TableExtDesc;
+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.realization.RealizationStatusEnum;
+import org.apache.kylin.rest.exception.InternalErrorException;
+import org.apache.kylin.rest.exception.RuleValidationException;
+import org.apache.kylin.rest.request.SQLRequest;
+import org.apache.kylin.rest.response.SQLResponse;
+import org.apache.kylin.source.ISourceMetadataExplorer;
+import org.apache.kylin.source.SourceManager;
+import org.apache.kylin.tool.migration.CompatibilityCheckRequest;
+import org.apache.kylin.tool.query.QueryGenerator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.context.SecurityContextHolder;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.LinkedHashMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.SetMultimap;
+import com.google.common.collect.Sets;
+
+/**
+ * Check the pre-defined rules. If not pass, we will throw
+ * {@link RuleValidationException}.
+ */
+public class MigrationRuleSet {
+
+    private static final Logger logger = LoggerFactory.getLogger(MigrationRuleSet.class);
+
+    public static final Rule DEFAULT_HIVE_TABLE_CONSISTENCY_RULE = new HiveTableConsistencyRule();
+    public static final Rule DEFAULT_CUBE_STATUS_RULE = new CubeStatusRule();
+    public static final Rule DEFAULT_PROJECT_EXIST_RULE = new ProjectExistenceRule();
+    public static final Rule DEFAULT_AUTO_MERGE_RULE = new AutoMergePolicyRule();
+    public static final Rule DEFAULT_EXPANSION_RULE = new ExpansionRateRule();
+    public static final Rule DEFAULT_EMAIL_NOTIFY_RULE = new NotificationEmailRule();
+    public static final Rule DEFAULT_COMPATIBLE_RULE = new CompatibleRule();
+    public static final Rule DEFAULT_SEGMENT_RULE = new SegmentRule();
+    public static final Rule DEFAULT_CUBE_OVERWRITE_RULE = new CubeOverwriteRule();
+    public static final Rule DEFAULT_QUERY_LATENCY_RULE = new QueryLatencyRule();
+
+    private static List<Rule> MUSTTOPASS_RULES = Lists.newLinkedList();
+
+    private static List<Rule> NICETOPASS_RULES = Lists.newLinkedList();
+
+    /**
+     * Register mandatory rules.
+     * @param rules
+     */
+    public static synchronized void register(Rule... rules) {
+        register(true, rules);
+    }
+
+    public static synchronized void register(boolean mandatory, Rule... rules) {
+        if (mandatory) {
+            for (Rule rule : rules) {
+                MUSTTOPASS_RULES.add(rule);
+            }
+        } else {
+            for (Rule rule : rules) {
+                NICETOPASS_RULES.add(rule);
+            }
+        }
+    }
+
+    // initialize default rules
+    static {
+        register(DEFAULT_HIVE_TABLE_CONSISTENCY_RULE, DEFAULT_CUBE_STATUS_RULE, DEFAULT_PROJECT_EXIST_RULE,
+                DEFAULT_EMAIL_NOTIFY_RULE, DEFAULT_SEGMENT_RULE, DEFAULT_CUBE_OVERWRITE_RULE, DEFAULT_COMPATIBLE_RULE);
+        register(false, DEFAULT_AUTO_MERGE_RULE, DEFAULT_EXPANSION_RULE, DEFAULT_QUERY_LATENCY_RULE);
+    }
+
+    /**
+     * @param ctx
+     * @return warn message if fail to pass some nice to have rules
+     * @throws RuleValidationException
+     */
+    public static String apply(Context ctx) throws RuleValidationException {
+        for (Rule rule : MUSTTOPASS_RULES) {
+            rule.apply(ctx);
+        }
+        StringBuilder sb = new StringBuilder();
+        for (Rule rule : NICETOPASS_RULES) {
+            try {
+                rule.apply(ctx);
+            } catch (RuleValidationException e) {
+                sb.append(e.getMessage());
+                sb.append("\n");
+            }
+        }
+        return sb.toString();
+    }
+
+    public interface Rule {
+        /**
+         * Apply the rule, success if no exception is thrown.
+         *
+         * @param ctx
+         * @throws RuleValidationException
+         *             if broke this rule
+         */
+        public void apply(Context ctx) throws RuleValidationException;
+    }
+
+    private static class ProjectExistenceRule implements Rule {
+
+        @Override
+        public void apply(Context ctx) throws RuleValidationException {
+            // code from CubeCopyCLI.java
+            ResourceStore dstStore = ctx.getTargetResourceStore();
+            String projectResPath = ProjectInstance.concatResourcePath(ctx.getTgtProjectName());
+            try {
+                if (!dstStore.exists(projectResPath)) {
+                    throw new RuleValidationException("The target project " + ctx.getTgtProjectName()
+                            + " does not exist on " + ctx.getTargetAddress());
+                }
+            } catch (RuleValidationException e) {
+                throw e;
+            } catch (IOException e) {
+                throw new RuleValidationException("Internal error: " + e.getMessage(), e);
+            }
+        }
+    }
+
+    private static class AutoMergePolicyRule implements Rule {
+
+        @Override
+        public void apply(Context ctx) throws RuleValidationException {
+            CubeDesc cubeDesc = ctx.getCubeInstance().getDescriptor();
+            long[] timeRanges = cubeDesc.getAutoMergeTimeRanges();
+            if (timeRanges == null || timeRanges.length == 0) {
+                throw new RuleValidationException(String.format(Locale.ROOT,
+                        "Auto merge time range for cube %s is not set.", cubeDesc.getName()));
+            }
+        }
+    }
+
+    private static class ExpansionRateRule implements Rule {
+
+        @Override
+        public void apply(Context ctx) throws RuleValidationException {
+            int expansionRateThr = KylinConfig.getInstanceFromEnv().getMigrationRuleExpansionRateThreshold();
+
+            CubeInstance cube = ctx.getCubeInstance();
+            if (cube.getInputRecordSizeBytes() == 0 || cube.getSizeKB() == 0) {
+                logger.warn("cube {} has zero input record size.", cube.getName());
+                throw new RuleValidationException(String.format(Locale.ROOT, "Cube %s is not built.", cube.getName()));
+            }
+            double expansionRate = cube.getSizeKB() * 1024.0 / cube.getInputRecordSizeBytes();
+            if (expansionRate > expansionRateThr) {
+                logger.info(
+                        "cube {}, size_kb {}, cube record size {}, cube expansion rate {} larger than threshold {}.",
+                        cube.getName(), cube.getSizeKB(), cube.getInputRecordSizeBytes(), expansionRate,
+                        expansionRateThr);
+                throw new RuleValidationException(
+                        "ExpansionRateRule: failed on expansion rate check with exceeding " + expansionRateThr);
+            }
+        }
+    }
+
+    private static class NotificationEmailRule implements Rule {
+
+        @Override
+        public void apply(Context ctx) throws RuleValidationException {
+            CubeDesc cubeDesc = ctx.getCubeInstance().getDescriptor();
+            List<String> notifyList = cubeDesc.getNotifyList();
+            if (notifyList == null || notifyList.size() == 0) {
+                throw new RuleValidationException("Cube email notification list is not set or empty.");
+            }
+        }
+    }
+
+    private static class CompatibleRule implements Rule {
+
+        @Override
+        public void apply(Context ctx) throws RuleValidationException {
+            try {
+                checkSchema(ctx);
+            } catch (Exception e) {
+                throw new RuleValidationException(e.getMessage(), e);
+            }
+        }
+
+        public void checkSchema(Context ctx) throws IOException {
+            Set<TableDesc> tableSet = Sets.newHashSet();
+            for (TableRef tableRef : ctx.getCubeInstance().getModel().getAllTables()) {
+                tableSet.add(tableRef.getTableDesc());
+            }
+
+            List<String> tableDataList = Lists.newArrayList();
+            for (TableDesc table : tableSet) {
+                tableDataList.add(JsonUtil.writeValueAsIndentString(table));
+            }
+
+            DataModelDesc model = ctx.getCubeInstance().getModel();
+            String modelDescData = JsonUtil.writeValueAsIndentString(model);
+
+            CompatibilityCheckRequest request = new CompatibilityCheckRequest();
+            request.setProjectName(ctx.getTgtProjectName());
+            request.setTableDescDataList(tableDataList);
+            request.setModelDescData(modelDescData);
+
+            String jsonRequest = JsonUtil.writeValueAsIndentString(request);
+            RestClient client = new RestClient(ctx.getTargetAddress());
+            client.checkCompatibility(jsonRequest);
+        }
+    }
+
+    private static class SegmentRule implements Rule {
+
+        @Override
+        public void apply(Context ctx) throws RuleValidationException {
+            List<CubeSegment> segments = ctx.getCubeInstance().getSegments(SegmentStatusEnum.READY);
+            if (segments == null || segments.size() == 0) {
+                throw new RuleValidationException("No built segment found.");
+            }
+        }
+    }
+
+    private static class CubeOverwriteRule implements Rule {
+
+        @Override
+        public void apply(Context ctx) throws RuleValidationException {
+            ResourceStore dstStore = ctx.getTargetResourceStore();
+            CubeInstance cube = ctx.getCubeInstance();
+            try {
+                if (dstStore.exists(cube.getResourcePath()))
+                    throw new RuleValidationException("The cube named " + cube.getName()
+                            + " already exists on target metadata store. Please delete it firstly and try again");
+            } catch (IOException e) {
+                logger.error(e.getMessage(), e);
+                throw new RuleValidationException(e.getMessage(), e);
+            }
+        }
+    }
+
+    private static class CubeStatusRule implements Rule {
+
+        @Override
+        public void apply(Context ctx) throws RuleValidationException {
+            CubeInstance cube = ctx.getCubeInstance();
+            RealizationStatusEnum status = cube.getStatus();
+            if (status != RealizationStatusEnum.READY) {
+                throw new RuleValidationException("The cube named " + cube.getName() + " is not in READY state.");
+            }
+        }
+    }
+
+    private static class QueryLatencyRule implements Rule {
+
+        @Override
+        public void apply(MigrationRuleSet.Context ctx) throws RuleValidationException {
+            logger.info("QueryLatencyRule started.");
+            CubeInstance cube = ctx.getCubeInstance();
+
+            int latency = KylinConfig.getInstanceFromEnv().getMigrationRuleQueryLatency();
+            int iteration = KylinConfig.getInstanceFromEnv().getMigrationRuleQueryEvaluationIteration();
+            int maxDimension = cube.getConfig().getMigrationRuleQueryGeneratorMaxDimensions();
+
+            try {
+                List<String> queries = QueryGenerator.generateQueryList(cube.getDescriptor(), iteration, maxDimension);
+                assert queries.size() > 0;
+                long avg = executeQueries(queries, ctx);
+                logger.info("QueryLatencyRule ended: average time cost " + avg + "ms.");
+                if (avg > latency) {
+                    throw new RuleValidationException(
+                            "Failed on query latency check with average cost " + avg + " exceeding " + latency + "ms.");
+                }
+            } catch (Exception e) {
+                logger.error(e.getMessage(), e);
+                if (e instanceof RuleValidationException) {
+                    throw (RuleValidationException) e;
+                } else {
+                    throw new RuleValidationException(e.getMessage(), e);
+                }
+            }
+        }
+
+        private long executeQueries(final List<String> queries, final Context ctx) throws Exception {
+            int maxThreads = KylinConfig.getInstanceFromEnv().getMigrationRuleQueryLatencyMaxThreads();
+            int threadNum = Math.min(maxThreads, queries.size());
+            ExecutorService threadPool = Executors.newFixedThreadPool(threadNum);
+            CompletionService<Long> completionService = new ExecutorCompletionService<Long>(threadPool);
+            final Authentication auth = SecurityContextHolder.getContext().getAuthentication();
+            long start = System.currentTimeMillis();
+            for (final String query : queries) {
+                completionService.submit(new Callable<Long>() {
+                    @Override
+                    public Long call() throws Exception {
+                        SecurityContextHolder.getContext().setAuthentication(auth);
+                        SQLRequest sqlRequest = new SQLRequest();
+                        sqlRequest.setProject(ctx.getSrcProjectName());
+                        sqlRequest.setSql(query);
+                        SQLResponse sqlResponse = ctx.getQueryService().doQueryWithCache(sqlRequest, false);
+                        if (sqlResponse.getIsException()) {
+                            throw new RuleValidationException(sqlResponse.getExceptionMessage());
+                        }
+                        return sqlResponse.getDuration();
+                    }
+
+                });
+            }
+            long timeCostSum = 0L;
+            for (int i = 0; i < queries.size(); ++i) {
+                try {
+                    timeCostSum += completionService.take().get();
+                } catch (InterruptedException | ExecutionException e) {
+                    threadPool.shutdownNow();
+                    throw e;
+                }
+            }
+            long end = System.currentTimeMillis();
+            logger.info("Execute" + queries.size() + " queries took " + (end - start) + " ms, query time cost sum "
+                    + timeCostSum + " ms.");
+            return timeCostSum / queries.size();
+        }
+    }
+
+    // check if table schema on Kylin is updated to date with external Hive table
+    private static class HiveTableConsistencyRule implements Rule {
+
+        @Override
+        public void apply(Context ctx) throws RuleValidationException {
+            // de-dup
+            SetMultimap<String, String> db2tables = LinkedHashMultimap.create();
+            for (TableRef tableRef : ctx.getCubeInstance().getModel().getAllTables()) {
+                db2tables.put(tableRef.getTableDesc().getDatabase().toUpperCase(Locale.ROOT),
+                        tableRef.getTableDesc().getName().toUpperCase(Locale.ROOT));
+            }
+
+            // load all tables first
+            List<Pair<TableDesc, TableExtDesc>> allMeta = Lists.newArrayList();
+            ISourceMetadataExplorer explr = SourceManager.getDefaultSource().getSourceMetadataExplorer();
+            try {
+                for (Map.Entry<String, String> entry : db2tables.entries()) {
+                    Pair<TableDesc, TableExtDesc> pair = explr.loadTableMetadata(entry.getKey(), entry.getValue(),
+                            ctx.getSrcProjectName());
+                    TableDesc tableDesc = pair.getFirst();
+                    Preconditions.checkState(tableDesc.getDatabase().equals(entry.getKey()));
+                    Preconditions.checkState(tableDesc.getName().equals(entry.getValue()));
+                    Preconditions.checkState(tableDesc.getIdentity().equals(entry.getKey() + "." + entry.getValue()));
+                    TableExtDesc extDesc = pair.getSecond();
+                    Preconditions.checkState(tableDesc.getIdentity().equals(extDesc.getIdentity()));
+                    allMeta.add(pair);
+                }
+            } catch (Exception e) {
+                logger.error(e.getMessage(), e);
+                throw new RuleValidationException(
+                        "Internal error when checking HiveTableConsistencyRule: " + e.getMessage());
+            }
+
+            // do schema check
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+            TableSchemaUpdateChecker checker = new TableSchemaUpdateChecker(TableMetadataManager.getInstance(config),
+                    CubeManager.getInstance(config), DataModelManager.getInstance(config));
+            for (Pair<TableDesc, TableExtDesc> pair : allMeta) {
+                try {
+                    TableSchemaUpdateChecker.CheckResult result = checker.allowReload(pair.getFirst(),
+                            ctx.getSrcProjectName());
+                    result.raiseExceptionWhenInvalid();
+                } catch (Exception e) {
+                    logger.error(e.getMessage(), e);
+                    throw new RuleValidationException("Table " + pair.getFirst().getIdentity()
+                            + " has incompatible changes on Hive, please reload the hive table and update your model/cube if needed.");
+                }
+            }
+            logger.info("Cube " + ctx.getCubeInstance().getName() + " Hive table consistency check passed.");
+        }
+
+    }
+
+    public static class Context {
+        private final QueryService queryService;
+        private final CubeInstance cubeInstance;
+        private final String targetAddress; // the target kylin host with port
+        private final ResourceStore targetResourceStore;
+        private final String tgtProjectName; // the target project name
+        private final String srcProjectName; // the source project name
+
+        public Context(QueryService queryService, CubeInstance cubeInstance, String targetHost, String tgtProjectName) {
+            this.queryService = queryService;
+            this.cubeInstance = cubeInstance;
+            this.targetAddress = targetHost;
+            KylinConfig targetConfig = KylinConfig.createInstanceFromUri(targetHost);
+            this.targetResourceStore = ResourceStore.getStore(targetConfig);
+            this.tgtProjectName = tgtProjectName;
+
+            List<ProjectInstance> projList = ProjectManager.getInstance(KylinConfig.getInstanceFromEnv())
+                    .findProjects(cubeInstance.getType(), cubeInstance.getName());
+            if (projList.size() != 1) {
+                throw new InternalErrorException("Cube " + cubeInstance.getName()
+                        + " should belong to only one project. However, it's belong to " + projList);
+            }
+            this.srcProjectName = projList.get(0).getName();
+        }
+
+        public QueryService getQueryService() {
+            return queryService;
+        }
+
+        public CubeInstance getCubeInstance() {
+            return cubeInstance;
+        }
+
+        public String getTargetAddress() {
+            return targetAddress;
+        }
+
+        public ResourceStore getTargetResourceStore() {
+            return targetResourceStore;
+        }
+
+        public String getTgtProjectName() {
+            return tgtProjectName;
+        }
+
+        public String getSrcProjectName() {
+            return srcProjectName;
+        }
+    }
+}
\ No newline at end of file
diff --git a/cube-migration/src/main/java/org/apache/kylin/rest/service/MigrationService.java b/cube-migration/src/main/java/org/apache/kylin/rest/service/MigrationService.java
new file mode 100644
index 0000000..ca1e71b
--- /dev/null
+++ b/cube-migration/src/main/java/org/apache/kylin/rest/service/MigrationService.java
@@ -0,0 +1,225 @@
+/*
+ * 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.rest.service;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.AclEntity;
+import org.apache.kylin.common.util.MailService;
+import org.apache.kylin.common.util.MailTemplateProvider;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.metadata.project.ProjectInstance;
+import org.apache.kylin.rest.constant.Constant;
+import org.apache.kylin.rest.exception.BadRequestException;
+import org.apache.kylin.rest.exception.RuleValidationException;
+import org.apache.kylin.rest.util.MailNotificationUtil;
+import org.apache.kylin.tool.CubeMigrationCLI;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.security.acls.domain.PrincipalSid;
+import org.springframework.security.acls.model.AccessControlEntry;
+import org.springframework.security.acls.model.Acl;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.stereotype.Component;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * Provide migration logic implementation.
+ */
+@Component("migrationService")
+public class MigrationService extends BasicService {
+
+    private static final Logger logger = LoggerFactory.getLogger(MigrationService.class);
+
+    @Autowired
+    private AccessService accessService;
+
+    @Autowired
+    private CubeService cubeService;
+
+    private final String localHost = KylinConfig.getInstanceFromEnv().getMigrationLocalAddress();
+    private final String envName = KylinConfig.getInstanceFromEnv().getDeployEnv();
+
+    public String checkRule(MigrationRuleSet.Context context) throws RuleValidationException {
+        return MigrationRuleSet.apply(context);
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION')")
+    public void requestMigration(CubeInstance cube, MigrationRuleSet.Context ctx) throws Exception {
+        Map<String, String> root = Maps.newHashMap();
+        root.put("projectname", ctx.getTgtProjectName());
+        root.put("cubename", ctx.getCubeInstance().getName());
+        root.put("status", "NEED APPROVE");
+        root.put("envname", envName);
+        sendMigrationMail(MailNotificationUtil.MIGRATION_REQUEST, getEmailRecipients(cube), root);
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
+    public boolean reject(String cubeName, String projectName, String reason) {
+        try {
+            Map<String, String> root = Maps.newHashMap();
+            root.put("cubename", cubeName);
+            root.put("rejectedReason", reason);
+            root.put("status", "REJECTED");
+            root.put("envname", envName);
+
+            sendMigrationMail(MailNotificationUtil.MIGRATION_REJECTED, getEmailRecipients(cubeName), root);
+        } catch (Exception e) {
+            logger.error(e.getMessage(), e);
+            return false;
+        }
+        return true;
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
+    public void approve(CubeInstance cube, MigrationRuleSet.Context ctx) throws Exception {
+        checkRule(ctx);
+
+        String cubeName = cube.getName();
+        String projectName = ctx.getTgtProjectName();
+        try {
+            sendApprovedMailQuietly(cubeName, projectName);
+
+            // do cube migration
+            new CubeMigrationCLI().moveCube(localHost, ctx.getTargetAddress(), cubeName, projectName, "true", "false",
+                    "true", "true", "false");
+
+            sendCompletedMailQuietly(cubeName, projectName);
+        } catch (Exception e) {
+            logger.error(e.getMessage(), e);
+            sendMigrationFailedMailQuietly(cubeName, projectName, e.getMessage());
+            throw e;
+        }
+    }
+
+    private boolean sendApprovedMailQuietly(String cubeName, String projectName) {
+        try {
+            Map<String, String> root = Maps.newHashMap();
+            root.put("projectname", projectName);
+            root.put("cubename", cubeName);
+            root.put("status", "APPROVED");
+            root.put("envname", envName);
+
+            sendMigrationMail(MailNotificationUtil.MIGRATION_APPROVED, getEmailRecipients(cubeName), root);
+        } catch (Exception e) {
+            logger.error(e.getMessage(), e);
+            return false;
+        }
+        return true;
+    }
+
+    private boolean sendCompletedMailQuietly(String cubeName, String projectName) {
+        try {
+            Map<String, String> root = Maps.newHashMap();
+            root.put("projectname", projectName);
+            root.put("cubename", cubeName);
+            root.put("status", "COMPLETED");
+            root.put("envname", envName);
+
+            sendMigrationMail(MailNotificationUtil.MIGRATION_COMPLETED, getEmailRecipients(cubeName), root);
+        } catch (Exception e) {
+            logger.error(e.getMessage(), e);
+            return false;
+        }
+        return true;
+    }
+
+    private boolean sendMigrationFailedMailQuietly(String cubeName, String projectName, String reason) {
+        try {
+            Map<String, String> root = Maps.newHashMap();
+            root.put("projectname", projectName);
+            root.put("cubename", cubeName);
+            root.put("status", "FAILED");
+            root.put("failedReason", reason);
+            root.put("envname", envName);
+
+            sendMigrationMail(MailNotificationUtil.MIGRATION_FAILED, getEmailRecipients(cubeName), root);
+        } catch (Exception e) {
+            logger.error(e.getMessage(), e);
+            return false;
+        }
+        return true;
+    }
+
+    public List<String> getCubeAdmins(CubeInstance cubeInstance) {
+        ProjectInstance prjInstance = cubeInstance.getProjectInstance();
+        AclEntity ae = accessService.getAclEntity("ProjectInstance", prjInstance.getUuid());
+        logger.info("ProjectUUID : " + prjInstance.getUuid());
+        Acl acl = accessService.getAcl(ae);
+
+        String mailSuffix = KylinConfig.getInstanceFromEnv().getMigrationEmailSuffix();
+        List<String> cubeAdmins = Lists.newArrayList();
+        if (acl != null) {
+            for (AccessControlEntry ace : acl.getEntries()) {
+                if (ace.getPermission().getMask() == 16) {
+                    PrincipalSid ps = (PrincipalSid) ace.getSid();
+                    cubeAdmins.add(ps.getPrincipal() + mailSuffix);
+                }
+            }
+        }
+
+        if (cubeAdmins.isEmpty()) {
+            throw new BadRequestException("Cube access list is null, please add at least one role in it.");
+        }
+        return cubeAdmins;
+    }
+
+    public List<String> getEmailRecipients(String cubeName) throws Exception {
+        CubeInstance cubeInstance = cubeService.getCubeManager().getCube(cubeName);
+        return getEmailRecipients(cubeInstance);
+    }
+
+    public List<String> getEmailRecipients(CubeInstance cubeInstance) throws Exception {
+        List<String> recipients = Lists.newArrayList();
+        recipients.addAll(getCubeAdmins(cubeInstance));
+        recipients.addAll(cubeInstance.getDescriptor().getNotifyList());
+        String[] adminDls = KylinConfig.getInstanceFromEnv().getAdminDls();
+        if (adminDls != null) {
+            recipients.addAll(Lists.newArrayList(adminDls));
+        }
+        return recipients;
+    }
+
+    public void sendMigrationMail(String state, List<String> recipients, Map<String, String> root) {
+        String submitter = SecurityContextHolder.getContext().getAuthentication().getName();
+        root.put("requester", submitter);
+
+        String title;
+
+        // No project name for rejected title
+        if (state == MailNotificationUtil.MIGRATION_REJECTED) {
+            title = MailNotificationUtil.getMailTitle("MIGRATION", root.get("status"), root.get("envname"),
+                    root.get("cubename"));
+        } else {
+            title = MailNotificationUtil.getMailTitle("MIGRATION", root.get("status"), root.get("envname"),
+                    root.get("projectname"), root.get("cubename"));
+        }
+
+        String content = MailTemplateProvider.getInstance().buildMailContent(state,
+                Maps.<String, Object> newHashMap(root));
+
+        new MailService(KylinConfig.getInstanceFromEnv()).sendMail(recipients, title, content);
+    }
+}
diff --git a/cube-migration/src/main/java/org/apache/kylin/rest/util/MailNotificationUtil.java b/cube-migration/src/main/java/org/apache/kylin/rest/util/MailNotificationUtil.java
new file mode 100644
index 0000000..6ef4e28
--- /dev/null
+++ b/cube-migration/src/main/java/org/apache/kylin/rest/util/MailNotificationUtil.java
@@ -0,0 +1,33 @@
+/*
+ * 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.rest.util;
+
+import com.google.common.base.Joiner;
+
+public class MailNotificationUtil {
+    public static final String MIGRATION_REQUEST = "MIGRATION_REQUEST";
+    public static final String MIGRATION_REJECTED = "MIGRATION_REJECTED";
+    public static final String MIGRATION_APPROVED = "MIGRATION_APPROVED";
+    public static final String MIGRATION_COMPLETED = "MIGRATION_COMPLETED";
+    public static final String MIGRATION_FAILED = "MIGRATION_FAILED";
+
+    public static String getMailTitle(String... titleParts) {
+        return "[" + Joiner.on("]-[").join(titleParts) + "]";
+    }
+}
diff --git a/cube-migration/src/main/resources/mail_templates/MIGRATION_APPROVED.ftl b/cube-migration/src/main/resources/mail_templates/MIGRATION_APPROVED.ftl
new file mode 100644
index 0000000..03c76bf
--- /dev/null
+++ b/cube-migration/src/main/resources/mail_templates/MIGRATION_APPROVED.ftl
@@ -0,0 +1,189 @@
+<!--
+* 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.
+-->
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN"
+        "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
+
+<head>
+    <meta http-equiv="Content-Type" content="Multipart/Alternative; charset=UTF-8"/>
+    <meta name="viewport" content="width=device-width, initial-scale=1.0"/>
+</head>
+
+<style>
+    html {
+        font-size: 10px;
+    }
+
+    * {
+        box-sizing: border-box;
+    }
+
+    a:hover,
+    a:focus {
+        color: #23527c;
+        text-decoration: underline;
+    }
+
+    a:focus {
+        outline: 5px auto -webkit-focus-ring-color;
+        outline-offset: -2px;
+    }
+</style>
+
+<body>
+<div style="font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+<span style="
+line-height: 1;font-size: 16px;">
+    <p style="text-align:left;">Dear ${requester},</p>
+    <p>Your cube migration request has been approved. Please wait for migration completed. </p>
+</span>
+    <hr style="margin-top: 10px;
+margin-bottom: 10px;
+height:0px;
+border-top: 1px solid #eee;
+border-right:0px;
+border-bottom:0px;
+border-left:0px;">
+    <span style="display: inline;
+                background-color: #337ab7;
+                color: #fff;
+                line-height: 1;
+                font-weight: 700;
+                font-size:36px;
+                text-align: center;">&nbsp;Approved&nbsp;</span>
+    <hr style="margin-top: 10px;
+margin-bottom: 10px;
+height:0px;
+border-top: 1px solid #eee;
+border-right:0px;
+border-bottom:0px;
+border-left:0px;">
+    <table cellpadding="0" cellspacing="0" width="100%" style="border-collapse: collapse;border:1px solid #bce8f1;">
+
+        <tr>
+
+            <td style="padding: 10px 15px;
+            border: 1px solid #bce8f1;
+            background-color: #d9edf7;">
+                <h4 style="margin-top: 0;
+                margin-bottom: 0;
+                font-size: 14px;
+                color: #31708f;
+                font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+                    Request detail
+                </h4>
+            </td>
+        </tr>
+        <tr>
+
+            <td style="padding: 15px;">
+                <table cellpadding="0" cellspacing="0" width="100%"
+                       style="margin-bottom: 20px;border:1 solid #ddd;border-collapse: collapse;font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                    padding: 8px;">
+                            <h4 style="
+                                        margin-top: 0;
+                                        margin-bottom: 0;
+                                        line-height: 1.5;
+                                        text-align: left;
+                                        font-size: 14px;
+                                        font-style: normal;">Requester</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                padding: 8px;">
+                            <h4 style="margin-top: 0;
+                        margin-bottom: 0;
+                        line-height: 1.5;
+                        text-align: left;
+                        font-size: 14px;
+                        font-style: normal;
+                        font-weight: 300;">
+                            ${requester}</h4>
+                        </td>
+                    </tr>
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                    padding: 8px;">
+                            <h4 style="
+                                        margin-top: 0;
+                                        margin-bottom: 0;
+                                        line-height: 1.5;
+                                        text-align: left;
+                                        font-size: 14px;
+                                        font-style: normal;">Project Name</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                padding: 8px;">
+                            <h4 style="margin-top: 0;
+                        margin-bottom: 0;
+                        line-height: 1.5;
+                        text-align: left;
+                        font-size: 14px;
+                        font-style: normal;
+                        font-weight: 300;">
+                            ${projectname}</h4>
+                        </td>
+                    </tr>
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                    padding: 8px;">
+                            <h4 style="
+                                        margin-top: 0;
+                                        margin-bottom: 0;
+                                        line-height: 1.5;
+                                        text-align: left;
+                                        font-size: 14px;
+                                        font-style: normal;">Cube Name</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                padding: 8px;">
+                            <h4 style="margin-top: 0;
+                        margin-bottom: 0;
+                        line-height: 1.5;
+                        text-align: left;
+                        font-size: 14px;
+                        font-style: normal;
+                        font-weight: 300;">
+                            ${cubename}</h4>
+                        </td>
+                    </tr>
+                </table>
+            </td>
+        </tr>
+
+    </table>
+    <hr style="margin-top: 20px;
+margin-bottom: 20px;
+height:0px;
+border-top: 1px solid #eee;
+border-right:0px;
+border-bottom:0px;
+border-left:0px;">
+    <h4 style="font-weight: 500;
+line-height: 1;font-size: 16px;">
+
+        <p>For any question, please contact support team <a href="mailto:DL-eBay-Kylin-Core@ebay.com "
+                                                            style="color: #337ab7;text-decoration: none;"><b>DL-eBay-Kylin-DISupport</b></a>
+            or Kylin <a href="https://ebay-eng.slack.com/messages/C1ZG2TN7J"
+                        style="color: #337ab7;text-decoration: none;"><b>Slack</b></a> channel.</p>
+    </h4>
+</div>
+</body>
+
+</html>
\ No newline at end of file
diff --git a/cube-migration/src/main/resources/mail_templates/MIGRATION_COMPLETED.ftl b/cube-migration/src/main/resources/mail_templates/MIGRATION_COMPLETED.ftl
new file mode 100644
index 0000000..58db588
--- /dev/null
+++ b/cube-migration/src/main/resources/mail_templates/MIGRATION_COMPLETED.ftl
@@ -0,0 +1,192 @@
+<!--
+* 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.
+-->
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN"
+        "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
+
+<head>
+    <meta http-equiv="Content-Type" content="Multipart/Alternative; charset=UTF-8"/>
+    <meta name="viewport" content="width=device-width, initial-scale=1.0"/>
+</head>
+
+<style>
+    html {
+        font-size: 10px;
+    }
+
+    * {
+        box-sizing: border-box;
+    }
+
+    a:hover,
+    a:focus {
+        color: #23527c;
+        text-decoration: underline;
+    }
+
+    a:focus {
+        outline: 5px auto -webkit-focus-ring-color;
+        outline-offset: -2px;
+    }
+</style>
+
+<body>
+<div style="font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+        <span style="
+line-height: 1;font-size: 16px;">
+            <p style="text-align:left;">Dear ${requester},</p>
+            <p>Your cube migration request has completed. Please check the cube in Kylin production.</p>
+        </span>
+    <hr style="margin-top: 10px;
+margin-bottom: 10px;
+height:0px;
+border-top: 1px solid #eee;
+border-right:0px;
+border-bottom:0px;
+border-left:0px;">
+    <span style="display: inline;
+                background-color: #5cb85c;
+                color: #fff;
+                line-height: 1;
+                font-weight: 700;
+                font-size:36px;
+                text-align: center;">&nbsp;Completed&nbsp;</span>
+    <hr style="margin-top: 10px;
+margin-bottom: 10px;
+height:0px;
+border-top: 1px solid #eee;
+border-right:0px;
+border-bottom:0px;
+border-left:0px;">
+
+    <table cellpadding="0" cellspacing="0" width="100%" style="border-collapse: collapse;border:1px solid #d6e9c6;">
+
+        <tr>
+
+            <td style="padding: 10px 15px;
+            background-color: #dff0d8;
+            border:1px solid #d6e9c6;">
+                <h4 style="margin-top: 0;
+                margin-bottom: 0;
+                font-size: 14px;
+                color: #3c763d;
+                font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+                    Request detail
+                </h4>
+            </td>
+        </tr>
+        <tr>
+
+            <td style="padding: 15px;">
+                <table cellpadding="0" cellspacing="0" width="100%"
+                       style="margin-bottom: 20px;border:1 solid #ddd;border-collapse: collapse;font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                                padding: 8px;">
+                            <h4 style="
+                                        margin-top: 0;
+                                        margin-bottom: 0;
+                                        line-height: 1.5;
+                                        text-align: left;
+                                        font-size: 14px;
+                                        font-style: normal;">Requester</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                            padding: 8px;">
+                            <h4 style="margin-top: 0;
+                                    margin-bottom: 0;
+                                    line-height: 1.5;
+                                    text-align: left;
+                                    font-size: 14px;
+                                    font-style: normal;
+                                    font-weight: 300;">
+                            ${requester}</h4>
+                        </td>
+                    </tr>
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                                padding: 8px;">
+                            <h4 style="
+                                                    margin-top: 0;
+                                                    margin-bottom: 0;
+                                                    line-height: 1.5;
+                                                    text-align: left;
+                                                    font-size: 14px;
+                                                    font-style: normal;">Project Name</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                            padding: 8px;">
+                            <h4 style="margin-top: 0;
+                                    margin-bottom: 0;
+                                    line-height: 1.5;
+                                    text-align: left;
+                                    font-size: 14px;
+                                    font-style: normal;
+                                    font-weight: 300;">${projectname}</h4>
+                        </td>
+                    </tr>
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                                padding: 8px;">
+                            <h4 style="
+                                                    margin-top: 0;
+                                                    margin-bottom: 0;
+                                                    line-height: 1.5;
+                                                    text-align: left;
+                                                    font-size: 14px;
+                                                    font-style: normal;">Cube Name</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                            padding: 8px;">
+                            <h4 style="margin-top: 0;
+                                    margin-bottom: 0;
+                                    line-height: 1.5;
+                                    text-align: left;
+                                    font-size: 14px;
+                                    font-style: normal;
+                                    font-weight: 300;">${cubename}</h4>
+                        </td>
+                    </tr>
+                </table>
+            </td>
+        </tr>
+
+    </table>
+    <hr style="margin-top: 20px;
+        margin-bottom: 20px;
+        height:0px;
+        border-top: 1px solid #eee;
+        border-right:0px;
+        border-bottom:0px;
+        border-left:0px;">
+    <h4 style="font-weight: 500;
+line-height: 1;font-size: 16px;">
+
+        <p>For any question, please contact support team
+            <a href="mailto:DL-eBay-Kylin-Core@ebay.com " style="color: #337ab7; text-decoration: none;">
+                <b>DL-eBay-Kylin-DISupport</b>
+            </a>
+            or Kylin
+            <a href="https://ebay-eng.slack.com/messages/C1ZG2TN7J" style="color: #337ab7; text-decoration: none;">
+                <b>Slack</b>
+            </a> channel.</p>
+    </h4>
+</div>
+</body>
+
+</html>
\ No newline at end of file
diff --git a/cube-migration/src/main/resources/mail_templates/MIGRATION_FAILED.ftl b/cube-migration/src/main/resources/mail_templates/MIGRATION_FAILED.ftl
new file mode 100644
index 0000000..2a02f92
--- /dev/null
+++ b/cube-migration/src/main/resources/mail_templates/MIGRATION_FAILED.ftl
@@ -0,0 +1,220 @@
+<!--
+* 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.
+-->
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN"
+        "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
+
+<head>
+    <meta http-equiv="Content-Type" content="Multipart/Alternative; charset=UTF-8" />
+    <meta name="viewport" content="width=device-width, initial-scale=1.0" />
+</head>
+
+<style>
+    html {
+        font-size: 10px;
+    }
+
+    * {
+        box-sizing: border-box;
+    }
+
+    a:hover,
+    a:focus {
+        color: #23527c;
+        text-decoration: underline;
+    }
+
+    a:focus {
+        outline: 5px auto -webkit-focus-ring-color;
+        outline-offset: -2px;
+    }
+</style>
+
+<body>
+<div style="font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+        <span style="
+        line-height: 1;font-size: 16px;">
+            <p style="text-align:left;">Dear ${requester},</p>
+            <p>Your cube migration request has failed. Please engage support team to check the reason. You can file a support
+                ticket through
+                <a href="http://go/dis/submit" style="color: #337ab7;text-decoration: none;">
+                    <b>JIRA</b>
+                </a>.</p>
+        </span>
+    <hr style="margin-top: 10px;
+margin-bottom: 10px;
+height:0px;
+border-top: 1px solid #eee;
+border-right:0px;
+border-bottom:0px;
+border-left:0px;">
+    <span style="display: inline;
+                background-color: #d9534f;
+                color: #fff;
+                line-height: 1;
+                font-weight: 700;
+                font-size:36px;
+                text-align: center;">&nbsp;Failed&nbsp;</span>
+    <hr style="margin-top: 10px;
+                margin-bottom: 10px;
+                height:0px;
+                border-top: 1px solid #eee;
+                border-right:0px;
+                border-bottom:0px;
+                border-left:0px;">
+
+    <table cellpadding="0" cellspacing="0" width="100%" style="border-collapse: collapse;border:1px solid #ebccd1;">
+
+        <tr>
+
+            <td style="padding: 10px 15px;
+            border:1px solid #ebccd1;
+            background-color: #f2dede;">
+                <h4 style="margin-top: 0;
+                margin-bottom: 0;
+                font-size: 14px;
+                color: #a94442;
+                font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+                    Request detail
+                </h4>
+            </td>
+        </tr>
+        <tr>
+
+            <td style="padding: 15px;">
+                <table cellpadding="0" cellspacing="0" width="100%" style="margin-bottom: 20px;border:1 solid #ddd;border-collapse: collapse;font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                                padding: 8px;">
+                            <h4 style="
+                                                    margin-top: 0;
+                                                    margin-bottom: 0;
+                                                    line-height: 1.5;
+                                                    text-align: left;
+                                                    font-size: 14px;
+                                                    font-style: normal;">Requester</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                            padding: 8px;">
+                            <h4 style="margin-top: 0;
+                                    margin-bottom: 0;
+                                    line-height: 1.5;
+                                    text-align: left;
+                                    font-size: 14px;
+                                    font-style: normal;
+                                    font-weight: 300;">
+                            ${requester}</h4>
+                        </td>
+                    </tr>
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                                padding: 8px;">
+                            <h4 style="
+                                                    margin-top: 0;
+                                                    margin-bottom: 0;
+                                                    line-height: 1.5;
+                                                    text-align: left;
+                                                    font-size: 14px;
+                                                    font-style: normal;">Project Name</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                            padding: 8px;">
+                            <h4 style="margin-top: 0;
+                                    margin-bottom: 0;
+                                    line-height: 1.5;
+                                    text-align: left;
+                                    font-size: 14px;
+                                    font-style: normal;
+                                    font-weight: 300;">
+                            ${projectname}</h4>
+                        </td>
+                    </tr>
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                                padding: 8px;">
+                            <h4 style="
+                                                    margin-top: 0;
+                                                    margin-bottom: 0;
+                                                    line-height: 1.5;
+                                                    text-align: left;
+                                                    font-size: 14px;
+                                                    font-style: normal;">Cube Name</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                            padding: 8px;">
+                            <h4 style="margin-top: 0;
+                                    margin-bottom: 0;
+                                    line-height: 1.5;
+                                    text-align: left;
+                                    font-size: 14px;
+                                    font-style: normal;
+                                    font-weight: 300;">
+                            ${cubename}</h4>
+                        </td>
+                    </tr>
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                                padding: 8px;">
+                            <h4 style="
+                                                    margin-top: 0;
+                                                    margin-bottom: 0;
+                                                    line-height: 1.5;
+                                                    text-align: left;
+                                                    font-size: 14px;
+                                                    font-style: normal;">Failed Reason</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                            padding: 8px;">
+                            <h4 style="margin-top: 0;
+                                    margin-bottom: 0;
+                                    line-height: 1.5;
+                                    text-align: left;
+                                    font-size: 14px;
+                                    font-style: normal;
+                                    font-weight: 300;">
+                            ${failedReason}</h4>
+                        </td>
+                    </tr>
+                </table>
+            </td>
+        </tr>
+
+    </table>
+    <hr style="margin-top: 20px;
+        margin-bottom: 20px;
+        height:0px;
+        border-top: 1px solid #eee;
+        border-right:0px;
+        border-bottom:0px;
+        border-left:0px;">
+    <h4 style="font-weight: 500;
+    line-height: 1;font-size:16px;">
+
+        <p>For any question, please contact support team
+            <a href="mailto:DL-eBay-Kylin-Core@ebay.com " style="color: #337ab7;text-decoration: none;">
+                <b>DL-eBay-Kylin-DISupport</b>
+            </a>
+            or Kylin
+            <a href="https://ebay-eng.slack.com/messages/C1ZG2TN7J" style="color: #337ab7;text-decoration: none;">
+                <b>Slack</b>
+            </a> channel.</p>
+    </h4>
+</div>
+</body>
+
+</html>
\ No newline at end of file
diff --git a/cube-migration/src/main/resources/mail_templates/MIGRATION_REJECTED.ftl b/cube-migration/src/main/resources/mail_templates/MIGRATION_REJECTED.ftl
new file mode 100644
index 0000000..ea9eff6
--- /dev/null
+++ b/cube-migration/src/main/resources/mail_templates/MIGRATION_REJECTED.ftl
@@ -0,0 +1,196 @@
+<!--
+* 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.
+-->
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN"
+        "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
+
+<head>
+    <meta http-equiv="Content-Type" content="Multipart/Alternative; charset=UTF-8"/>
+    <meta name="viewport" content="width=device-width, initial-scale=1.0"/>
+</head>
+
+<style>
+    html {
+        font-size: 10px;
+    }
+
+    * {
+        box-sizing: border-box;
+    }
+
+    a:hover,
+    a:focus {
+        color: #23527c;
+        text-decoration: underline;
+    }
+
+    a:focus {
+        outline: 5px auto -webkit-focus-ring-color;
+        outline-offset: -2px;
+    }
+</style>
+
+<body>
+<div style="font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+        <span style="line-height: 1;font-size: 16px;">
+            <p style="text-align:left;">Dear ${requester},</p>
+            <p>Your cube migration request has been rejected. Please check the reason below and re-submit the request after
+                making related changes.</p>
+        </span>
+    <hr style="margin-top: 10px;
+margin-bottom: 10px;
+height:0px;
+border-top: 1px solid #eee;
+border-right:0px;
+border-bottom:0px;
+border-left:0px;">
+    <span style="display: inline;
+        background-color: #d9534f;
+        color: #fff;
+        line-height: 1;
+        font-weight: 700;
+        font-size:36px;
+        text-align: center;">&nbsp;Rejected&nbsp;</span>
+    <hr style="margin-top: 10px;
+margin-bottom: 10px;
+height:0px;
+border-top: 1px solid #eee;
+border-right:0px;
+border-bottom:0px;
+border-left:0px;">
+
+    <table cellpadding="0" cellspacing="0" width="100%"
+           style="border-collapse: collapse;border:1px solid #ebccd1;table-layout:fixed">
+
+        <tr>
+
+            <td style="padding: 10px 15px;
+            background-color: #f2dede;
+            border:1px solid #ebccd1;">
+                <h4 style="margin-top: 0;
+                margin-bottom: 0;
+                font-size: 14px;
+                color: #a94442;
+                font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+                    Request detail
+                </h4>
+            </td>
+        </tr>
+        <tr>
+
+            <td style="padding: 15px;">
+                <table cellpadding="0" cellspacing="0" width="100%"
+                       style="margin-bottom: 20px;border:1 solid #ddd;border-collapse: collapse;font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                    padding: 8px;">
+                            <h4 style="
+                                        margin-top: 0;
+                                        margin-bottom: 0;
+                                        line-height: 1.5;
+                                        text-align: left;
+                                        font-size: 14px;
+                                        font-style: normal;">Requester</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                padding: 8px;">
+                            <h4 style="margin-top: 0;
+                        margin-bottom: 0;
+                        line-height: 1.5;
+                        text-align: left;
+                        font-size: 14px;
+                        font-style: normal;
+                        font-weight: 300;">
+                            ${requester}</h4>
+                        </td>
+                    </tr>
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                    padding: 8px;">
+                            <h4 style="
+                                        margin-top: 0;
+                                        margin-bottom: 0;
+                                        line-height: 1.5;
+                                        text-align: left;
+                                        font-size: 14px;
+                                        font-style: normal;">Cube Name</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                padding: 8px;">
+                            <h4 style="margin-top: 0;
+                        margin-bottom: 0;
+                        line-height: 1.5;
+                        text-align: left;
+                        font-size: 14px;
+                        font-style: normal;
+                        font-weight: 300;">
+                            ${cubename}</h4>
+                        </td>
+                    </tr>
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                    padding: 8px;">
+                            <h4 style="
+                                        margin-top: 0;
+                                        margin-bottom: 0;
+                                        line-height: 1.5;
+                                        text-align: left;
+                                        font-size: 14px;
+                                        font-style: normal;">Rejected Reason</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                padding: 8px;">
+                            <h4 style="margin-top: 0;
+                        margin-bottom: 0;
+                        line-height: 1.5;
+                        text-align: left;
+                        font-size: 14px;
+                        font-style: normal;
+                        font-weight: 300;">
+                                <pre style="white-space: pre-wrap;">${rejectedReason}</pre>
+                            </h4>
+                        </td>
+                    </tr>
+                </table>
+            </td>
+        </tr>
+
+    </table>
+    <hr style="margin-top: 20px;
+margin-bottom: 20px;
+height:0px;
+border-top: 1px solid #eee;
+border-right:0px;
+border-bottom:0px;
+border-left:0px;">
+    <h4 style="font-weight: 500;
+    line-height: 1;font-size:16px;">
+
+        <p>For any question, please contact support team
+            <a href="mailto:DL-eBay-Kylin-Core@ebay.com " style="color: #337ab7;text-decoration: none;">
+                <b>DL-eBay-Kylin-DISupport</b>
+            </a>
+            or Kylin
+            <a href="https://ebay-eng.slack.com/messages/C1ZG2TN7J" style="color: #337ab7;text-decoration: none;">
+                <b>Slack</b>
+            </a> channel.</p>
+    </h4>
+</div>
+</body>
+
+</html>
\ No newline at end of file
diff --git a/cube-migration/src/main/resources/mail_templates/MIGRATION_REQUEST.ftl b/cube-migration/src/main/resources/mail_templates/MIGRATION_REQUEST.ftl
new file mode 100644
index 0000000..6c50f9c
--- /dev/null
+++ b/cube-migration/src/main/resources/mail_templates/MIGRATION_REQUEST.ftl
@@ -0,0 +1,192 @@
+<!--
+* 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.
+-->
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN"
+        "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
+
+<head>
+    <meta http-equiv="Content-Type" content="Multipart/Alternative; charset=UTF-8"/>
+    <meta name="viewport" content="width=device-width, initial-scale=1.0"/>
+</head>
+
+<style>
+    html {
+        font-size: 10px;
+    }
+
+    * {
+        box-sizing: border-box;
+    }
+
+    a:hover,
+    a:focus {
+        color: #23527c;
+        text-decoration: underline;
+    }
+
+    a:focus {
+        outline: 5px auto -webkit-focus-ring-color;
+        outline-offset: -2px;
+    }
+</style>
+
+<body>
+<div style="ont-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+<span style="
+    line-height: 1;font-size: 16px;">
+    <p style="text-align:left;">Dear ${requester},</p>
+    <p>Your cube migration request has been submitted. It has been sent to Kylin PM for approval. Please stay tuned.</p>
+</span>
+    <hr style="margin-top: 10px;
+margin-bottom: 10px;
+height:0px;
+border-top: 1px solid #eee;
+border-right:0px;
+border-bottom:0px;
+border-left:0px;">
+    <span style="display: inline;
+                background-color: #337ab7;
+                color: #fff;
+                line-height: 1;
+                font-weight: 700;
+                font-size:36px;
+                text-align: center;">&nbsp;Waiting for approval&nbsp;</span>
+    <hr style="margin-top: 10px;
+margin-bottom: 10px;
+height:0px;
+border-top: 1px solid #eee;
+border-right:0px;
+border-bottom:0px;
+border-left:0px;">
+
+    <table cellpadding="0" cellspacing="0" width="100%" style="border-collapse: collapse;border:1px solid #bce8f1;">
+
+        <tr>
+
+            <td style="padding: 10px 15px;
+            border-bottom: 1px solid transparent;
+            background-color: #d9edf7;
+            border-color: #bce8f1;">
+                <h4 style="margin-top: 0;
+                margin-bottom: 0;
+                font-size: 14px;
+                color: #31708f;
+                font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+                    Request detail
+                </h4>
+            </td>
+        </tr>
+        <tr>
+
+            <td style="padding: 15px;">
+                <table cellpadding="0" cellspacing="0" width="100%"
+                       style="margin-bottom: 20px;border:1 solid #ddd;border-collapse: collapse;font-family: 'Trebuchet MS ', Arial, Helvetica, sans-serif;">
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                    padding: 8px;">
+                            <h4 style="
+                                        margin-top: 0;
+                                        margin-bottom: 0;
+                                        line-height: 1.5;
+                                        text-align: left;
+                                        font-size: 14px;
+                                        font-style: normal;">Requester</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                padding: 8px;">
+                            <h4 style="margin-top: 0;
+                        margin-bottom: 0;
+                        line-height: 1.5;
+                        text-align: left;
+                        font-size: 14px;
+                        font-style: normal;
+                        font-weight: 300;">
+                            ${requester}</h4>
+                        </td>
+                    </tr>
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                    padding: 8px;">
+                            <h4 style="
+                                        margin-top: 0;
+                                        margin-bottom: 0;
+                                        line-height: 1.5;
+                                        text-align: left;
+                                        font-size: 14px;
+                                        font-style: normal;">Project Name</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                padding: 8px;">
+                            <h4 style="margin-top: 0;
+                        margin-bottom: 0;
+                        line-height: 1.5;
+                        text-align: left;
+                        font-size: 14px;
+                        font-style: normal;
+                        font-weight: 300;">
+                            ${projectname}</h4>
+                        </td>
+                    </tr>
+                    <tr>
+                        <th width="30%" style="border: 1px solid #ddd;
+                    padding: 8px;">
+                            <h4 style="
+                                        margin-top: 0;
+                                        margin-bottom: 0;
+                                        line-height: 1.5;
+                                        text-align: left;
+                                        font-size: 14px;
+                                        font-style: normal;">Cube Name</h4>
+                        </th>
+                        <td style="border: 1px solid #ddd;
+                padding: 8px;">
+                            <h4 style="margin-top: 0;
+                        margin-bottom: 0;
+                        line-height: 1.5;
+                        text-align: left;
+                        font-size: 14px;
+                        font-style: normal;
+                        font-weight: 300;">
+                            ${cubename}</h4>
+                        </td>
+                    </tr>
+
+                </table>
+            </td>
+        </tr>
+
+    </table>
+    <hr style="margin-top: 20px;
+margin-bottom: 20px;
+height:0px;
+border-top: 1px solid #eee;
+border-right:0px;
+border-bottom:0px;
+border-left:0px;">
+    <h4 style="font-weight: 500;
+    line-height: 1;font-size:16px;">
+
+        <p>For any question, please contact support team <a href="mailto:DL-eBay-Kylin-Core@ebay.com "
+                                                            style="color: #337ab7;text-decoration: none;"><b>DL-eBay-Kylin-DISupport</b></a>
+            or Kylin <a href="https://ebay-eng.slack.com/messages/C1ZG2TN7J"
+                        style="color: #337ab7;text-decoration: none;"><b>Slack</b></a> channel.</p>
+    </h4>
+</div>
+</body>
+
+</html>
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index 20481e6..c73bf28 100644
--- a/pom.xml
+++ b/pom.xml
@@ -363,6 +363,11 @@
       </dependency>
       <dependency>
         <groupId>org.apache.kylin</groupId>
+        <artifactId>kylin-cube-migration</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.kylin</groupId>
         <artifactId>kylin-tool</artifactId>
         <version>${project.version}</version>
       </dependency>
@@ -1374,6 +1379,7 @@
     <module>metrics-reporter-hive</module>
     <module>metrics-reporter-kafka</module>
     <module>cache</module>
+    <module>cube-migration</module>
     <module>datasource-sdk</module>
     <module>storage-stream</module>
     <module>stream-receiver</module>
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/ModelSchemaUpdateChecker.java b/server-base/src/main/java/org/apache/kylin/rest/service/ModelSchemaUpdateChecker.java
index e8c04fd..63d2e78 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/ModelSchemaUpdateChecker.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/ModelSchemaUpdateChecker.java
@@ -169,6 +169,10 @@ public class ModelSchemaUpdateChecker {
     }
 
     public CheckResult allowEdit(DataModelDesc modelDesc, String prj) {
+        return allowEdit(modelDesc, prj, !modelDesc.isDraft());
+    }
+
+    public CheckResult allowEdit(DataModelDesc modelDesc, String prj, boolean needInit) {
 
         final String modelName = modelDesc.getName();
         // No model
@@ -176,7 +180,9 @@ public class ModelSchemaUpdateChecker {
         if (existing == null) {
             return CheckResult.validOnFirstCreate(modelName);
         }
-        modelDesc.init(metadataManager.getConfig(), metadataManager.getAllTablesMap(prj));
+        if (needInit) {
+            modelDesc.init(metadataManager.getConfig(), metadataManager.getAllTablesMap(prj));
+        }
 
         // No cube
         List<CubeInstance> cubes = findCubeByModel(modelName);
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
index d1c583f..5a3a1ca 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
@@ -148,12 +148,12 @@ public class ModelService extends BasicService {
     public DataModelDesc updateModelAndDesc(String project, DataModelDesc desc) throws IOException {
         aclEvaluate.checkProjectWritePermission(project);
         validateModel(project, desc);
-        checkModelCompatible(project, desc);
+        checkModelCompatibility(project, desc);
         getDataModelManager().updateDataModelDesc(desc);
         return desc;
     }
 
-    public void checkModelCompatible(String project, DataModelDesc dataModalDesc) {
+    public void checkModelCompatibility(String project, DataModelDesc dataModalDesc) {
         ProjectInstance prjInstance = getProjectManager().getProject(project);
         if (prjInstance == null) {
             throw new BadRequestException("Project " + project + " does not exist");
@@ -168,6 +168,23 @@ public class ModelService extends BasicService {
         result.raiseExceptionWhenInvalid();
     }
 
+    public void checkModelCompatibility(String project, DataModelDesc dataModalDesc, List<TableDesc> tableDescList) {
+        ProjectInstance prjInstance = getProjectManager().getProject(project);
+        if (prjInstance == null) {
+            throw new BadRequestException("Project " + project + " does not exist");
+        }
+        ModelSchemaUpdateChecker checker = new ModelSchemaUpdateChecker(getTableManager(), getCubeManager(),
+                getDataModelManager());
+
+        Map<String, TableDesc> tableDescMap = Maps.newHashMapWithExpectedSize(tableDescList.size());
+        for (TableDesc tableDesc : tableDescList) {
+            tableDescMap.put(tableDesc.getIdentity(), tableDesc);
+        }
+        dataModalDesc.init(getConfig(), tableDescMap);
+        ModelSchemaUpdateChecker.CheckResult result = checker.allowEdit(dataModalDesc, project, false);
+        result.raiseExceptionWhenInvalid();
+    }
+
     public void validateModel(String project, DataModelDesc desc) throws IllegalArgumentException {
         String factTableName = desc.getRootFactTableName();
         TableDesc tableDesc = getTableManager().getTableDesc(factTableName, project);
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java b/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java
index 1b205be..f03acb8 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/TableSchemaUpdateChecker.java
@@ -50,7 +50,7 @@ public class TableSchemaUpdateChecker {
     private final CubeManager cubeManager;
     private final DataModelManager dataModelManager;
 
-    static class CheckResult {
+    public static class CheckResult {
         private final boolean valid;
         private final String reason;
 
@@ -59,7 +59,7 @@ public class TableSchemaUpdateChecker {
             this.reason = reason;
         }
 
-        void raiseExceptionWhenInvalid() {
+        public void raiseExceptionWhenInvalid() {
             if (!valid) {
                 throw new RuntimeException(reason);
             }
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java b/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
index cd18d2b..764a32a 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
@@ -117,6 +117,15 @@ public class TableService extends BasicService {
     @Autowired
     private AclEvaluate aclEvaluate;
 
+    public TableSchemaUpdateChecker getSchemaUpdateChecker() {
+        return new TableSchemaUpdateChecker(getTableManager(), getCubeManager(), getDataModelManager());
+    }
+
+    public void checkTableCompatibility(String prj, TableDesc tableDesc) {
+        TableSchemaUpdateChecker.CheckResult result = getSchemaUpdateChecker().allowReload(tableDesc, prj);
+        result.raiseExceptionWhenInvalid();
+    }
+    
     public List<TableDesc> getTableDescByProject(String project, boolean withExt) throws IOException {
         aclEvaluate.checkProjectReadPermission(project);
         List<TableDesc> tables = getProjectManager().listDefinedTables(project);
diff --git a/server/pom.xml b/server/pom.xml
index e0ec203..517c1cd 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -54,6 +54,20 @@
         </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-cube-migration</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-shaded-guava</artifactId>
             <scope>compile</scope>
         </dependency>
diff --git a/server/src/main/resources/kylinSecurity.xml b/server/src/main/resources/kylinSecurity.xml
index 0003a5f..baf7172 100644
--- a/server/src/main/resources/kylinSecurity.xml
+++ b/server/src/main/resources/kylinSecurity.xml
@@ -236,6 +236,7 @@
             <scr:http-basic entry-point-ref="unauthorisedEntryPoint"/>
 
             <scr:intercept-url pattern="/api/user/authentication*/**" access="permitAll"/>
+            <scr:intercept-url pattern="/api/cubes/checkCompatibility" access="permitAll"/>
             <scr:intercept-url pattern="/api/query/runningQueries" access="hasRole('ROLE_ADMIN')"/>
             <scr:intercept-url pattern="/api/query/*/stop" access="hasRole('ROLE_ADMIN')"/>
             <scr:intercept-url pattern="/api/query*/**" access="isAuthenticated()"/>
@@ -251,6 +252,7 @@
             <scr:intercept-url pattern="/api/streaming*/**" access="isAuthenticated()"/>
             <scr:intercept-url pattern="/api/job*/**" access="isAuthenticated()"/>
             <scr:intercept-url pattern="/api/admin/public_config" access="permitAll"/>
+            <scr:intercept-url pattern="/api/admin/config" access="permitAll"/>
             <scr:intercept-url pattern="/api/admin/version" access="permitAll"/>
             <scr:intercept-url pattern="/api/projects" access="permitAll"/>
             <scr:intercept-url pattern="/api/admin*/**" access="hasRole('ROLE_ADMIN')"/>
@@ -287,6 +289,7 @@
             <scr:http-basic entry-point-ref="unauthorisedEntryPoint"/>
 
             <scr:intercept-url pattern="/api/user/authentication*/**" access="permitAll"/>
+            <scr:intercept-url pattern="/api/cubes/checkCompatibility" access="permitAll"/>
             <scr:intercept-url pattern="/api/query/runningQueries" access="hasRole('ROLE_ADMIN')"/>
             <scr:intercept-url pattern="/api/query/*/stop" access="hasRole('ROLE_ADMIN')"/>
             <scr:intercept-url pattern="/api/query*/**" access="isAuthenticated()"/>
diff --git a/tool/src/main/java/org/apache/kylin/tool/migration/CompatibilityCheckRequest.java b/tool/src/main/java/org/apache/kylin/tool/migration/CompatibilityCheckRequest.java
new file mode 100644
index 0000000..7144f7d
--- /dev/null
+++ b/tool/src/main/java/org/apache/kylin/tool/migration/CompatibilityCheckRequest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.tool.migration;
+
+import java.util.List;
+
+public class CompatibilityCheckRequest {
+    private List<String> tableDescDataList;
+    private String modelDescData;
+    private String projectName;
+
+    public List<String> getTableDescDataList() {
+        return tableDescDataList;
+    }
+
+    public void setTableDescDataList(List<String> tableDescDataList) {
+        this.tableDescDataList = tableDescDataList;
+    }
+
+    public String getModelDescData() {
+        return modelDescData;
+    }
+
+    public void setModelDescData(String modelDescData) {
+        this.modelDescData = modelDescData;
+    }
+
+    public String getProjectName() {
+        return projectName;
+    }
+
+    public void setProjectName(String projectName) {
+        this.projectName = projectName;
+    }
+}
diff --git a/tool/src/main/java/org/apache/kylin/tool/query/ProbabilityGenerator.java b/tool/src/main/java/org/apache/kylin/tool/query/ProbabilityGenerator.java
new file mode 100644
index 0000000..9f32de1
--- /dev/null
+++ b/tool/src/main/java/org/apache/kylin/tool/query/ProbabilityGenerator.java
@@ -0,0 +1,88 @@
+/*
+ * 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.tool.query;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ProbabilityGenerator {
+
+    private static final Logger logger = LoggerFactory.getLogger(ProbabilityGenerator.class);
+
+    public static double[] generate(int size) {
+        double[] probArray = generateProbabilityList(size);
+        return generateProbabilityCumulative(probArray);
+    }
+
+    public static int searchIndex(double p, double[] pCumArray) {
+        return binarySearchIndex(p, pCumArray, 0, pCumArray.length - 1);
+    }
+
+    private static int binarySearchIndex(double key, double[] array, int from, int to) {
+        if (from < 0 || to < 0) {
+            throw new IllegalArgumentException("params from & length must larger than 0 .");
+        }
+        if (key < array[from]) {
+            return from - 1;
+        } else if (key >= array[to]) {
+            return to;
+        }
+
+        int middle = (from >>> 1) + (to >>> 1);
+        double temp = array[middle];
+        if (temp > key) {
+            to = middle - 1;
+        } else if (temp < key) {
+            from = middle + 1;
+        } else {
+            return middle;
+        }
+        return binarySearchIndex(key, array, from, to);
+    }
+
+    public static double[] generateProbabilityCumulative(double[] pQueryArray) {
+        double[] pCumArray = new double[pQueryArray.length];
+        pCumArray[0] = 0;
+        for (int i = 0; i < pQueryArray.length - 1; i++) {
+            pCumArray[i + 1] = pCumArray[i] + pQueryArray[i];
+        }
+        return pCumArray;
+    }
+
+    public static double[] generateProbabilityList(int nOfEle) {
+        Integer[] nHitArray = new Integer[nOfEle];
+        double[] pQueryArray = new double[nOfEle];
+
+        int sumHit = generateHitNumberList(nHitArray);
+        for (int i = 0; i < nOfEle; i++) {
+            pQueryArray[i] = nHitArray[i] * 1.0 / sumHit;
+        }
+        return pQueryArray;
+    }
+
+    public static int generateHitNumberList(Integer[] nHitArray) {
+        int sumHit = 0;
+        for (int i = 0; i < nHitArray.length; i++) {
+            int randomNum = 1 + (int) (Math.random() * nHitArray.length);
+            nHitArray[i] = randomNum * randomNum;
+            sumHit += nHitArray[i];
+        }
+        return sumHit;
+    }
+}
diff --git a/tool/src/main/java/org/apache/kylin/tool/query/ProbabilityGeneratorCLI.java b/tool/src/main/java/org/apache/kylin/tool/query/ProbabilityGeneratorCLI.java
new file mode 100644
index 0000000..c248ad9
--- /dev/null
+++ b/tool/src/main/java/org/apache/kylin/tool/query/ProbabilityGeneratorCLI.java
@@ -0,0 +1,99 @@
+/*
+ * 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.tool.query;
+
+import java.io.BufferedWriter;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.kylin.common.util.AbstractApplication;
+import org.apache.kylin.common.util.OptionsHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ProbabilityGeneratorCLI extends AbstractApplication {
+
+    private static final Logger logger = LoggerFactory.getLogger(ProbabilityGeneratorCLI.class);
+
+    private static final Option OPTION_SIZE = OptionBuilder.withArgName("size").hasArg().isRequired(true)
+            .withDescription("Specify the size of query set to be generated").create("size");
+    private static final Option OPTION_OUTPUT = OptionBuilder.withArgName("output").hasArg().isRequired(false)
+            .withDescription("Specify the output path for generated probability set").create("output");
+
+    protected final Options options;
+    private int size;
+    private String outputPath;
+
+    public ProbabilityGeneratorCLI() {
+        options = new Options();
+        options.addOption(OPTION_SIZE);
+        options.addOption(OPTION_OUTPUT);
+    }
+
+    protected Options getOptions() {
+        return options;
+    }
+
+    protected void execute(OptionsHelper optionsHelper) throws Exception {
+        this.size = Integer.parseInt(optionsHelper.getOptionValue(OPTION_SIZE));
+        this.outputPath = optionsHelper.getOptionValue(OPTION_OUTPUT);
+
+        run();
+    }
+
+    public double[] execute(int sizeOfQueryList, String outputPath) throws Exception {
+        this.size = sizeOfQueryList;
+        this.outputPath = outputPath;
+
+        return run();
+    }
+
+    private double[] run() throws Exception {
+        double[] probArray = ProbabilityGenerator.generateProbabilityList(this.size);
+        storeProbability(probArray, outputPath);
+        return ProbabilityGenerator.generateProbabilityCumulative(probArray);
+    }
+
+    public double[] execute(int size) throws Exception {
+        this.size = size;
+        double[] pQueryArray = ProbabilityGenerator.generateProbabilityList(this.size);
+        return ProbabilityGenerator.generateProbabilityCumulative(pQueryArray);
+    }
+
+    public static void storeProbability(double[] probArray, String outputPath) throws IOException {
+        try (BufferedWriter bufferedWriter = new BufferedWriter(
+                new OutputStreamWriter(new FileOutputStream(outputPath + ".prob"), StandardCharsets.UTF_8))) {
+            for (double elem : probArray) {
+                bufferedWriter.append(String.valueOf(elem));
+                bufferedWriter.append("\n");
+                logger.info(String.valueOf(elem));
+            }
+        }
+    }
+
+    public static void main(String[] args) {
+        ProbabilityGeneratorCLI probabilityGeneratorCLI = new ProbabilityGeneratorCLI();
+        probabilityGeneratorCLI.execute(args);
+    }
+}
diff --git a/tool/src/main/java/org/apache/kylin/tool/query/QueryGenerator.java b/tool/src/main/java/org/apache/kylin/tool/query/QueryGenerator.java
new file mode 100644
index 0000000..83fca53
--- /dev/null
+++ b/tool/src/main/java/org/apache/kylin/tool/query/QueryGenerator.java
@@ -0,0 +1,189 @@
+/*
+ * 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.tool.query;
+
+import java.math.BigInteger;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
+import org.apache.kylin.cube.model.DimensionDesc;
+import org.apache.kylin.job.JoinedFlatTable;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+public class QueryGenerator {
+
+    private static final Logger logger = LoggerFactory.getLogger(QueryGenerator.class);
+
+    public static List<String> generateQueryList(CubeDesc cubeDesc, int nOfQuery, int maxNumOfDimension) {
+        int nDimension = cubeDesc.getDimensions().size();
+        if (maxNumOfDimension > nDimension) {
+            maxNumOfDimension = nDimension;
+        } else if (maxNumOfDimension < 1) {
+            maxNumOfDimension = 1;
+        }
+
+        int queryMaxSize = getQueryMaxSize(maxNumOfDimension, nDimension);
+        queryMaxSize = (int) Math.ceil(queryMaxSize * 0.5);
+        if (nOfQuery > queryMaxSize) {
+            nOfQuery = queryMaxSize;
+        }
+
+        logger.info("Will generate {} queries", nOfQuery);
+
+        List<String> sqlList = Lists.newArrayListWithExpectedSize(nOfQuery);
+        Set<BitSet> selected = Sets.newHashSetWithExpectedSize(nOfQuery);
+        while (sqlList.size() < nOfQuery) {
+            sqlList.add(generateQuery(cubeDesc, selected, maxNumOfDimension));
+        }
+
+        return sqlList;
+    }
+
+    public static int getQueryMaxSize(int m, int nDimension) {
+        int a = nDimension - m >= m ? nDimension - m : m;
+        int b = nDimension - a;
+
+        BigInteger result = new BigInteger(String.valueOf(1));
+        for (int i = a + 1; i <= nDimension; i++) {
+            result = result.multiply(new BigInteger(String.valueOf(i)));
+        }
+        for (int i = 2; i <= b; i++) {
+            result = result.divide(new BigInteger(String.valueOf(i)));
+        }
+        return result.intValue();
+    }
+
+    public static String generateQuery(CubeDesc cubeDesc, Set<BitSet> selected, int maxNumOfDimension) {
+        IJoinedFlatTableDesc flatDesc = new CubeJoinedFlatTableDesc(cubeDesc);
+
+        String dimensionStatement = createDimensionStatement(cubeDesc.getDimensions(), selected, maxNumOfDimension);
+        String measureStatement = createMeasureStatement(cubeDesc.getMeasures());
+
+        StringBuilder sql = new StringBuilder();
+        sql.append("SELECT" + "\n");
+        sql.append(dimensionStatement);
+        sql.append(measureStatement);
+
+        StringBuilder joinPart = new StringBuilder();
+        JoinedFlatTable.appendJoinStatement(flatDesc, joinPart, false, null);
+        sql.append(joinPart.toString().replaceAll("DEFAULT\\.", ""));
+
+        sql.append("GROUP BY" + "\n");
+        sql.append(dimensionStatement);
+        String ret = sql.toString();
+        ret = ret.replaceAll("`", "\"");
+        return ret;
+    }
+
+    public static String createMeasureStatement(List<MeasureDesc> measureList) {
+        StringBuilder sql = new StringBuilder();
+
+        for (MeasureDesc measureDesc : measureList) {
+            FunctionDesc functionDesc = measureDesc.getFunction();
+            if (functionDesc.isSum() || functionDesc.isMax() || functionDesc.isMin()) {
+                sql.append("," + functionDesc.getExpression() + "(" + functionDesc.getParameter().getValue() + ")\n");
+                break;
+            } else if (functionDesc.isCountDistinct()) {
+                sql.append(",COUNT" + "(DISTINCT " + functionDesc.getParameter().getValue() + ")\n");
+                break;
+            }
+        }
+
+        return sql.toString();
+    }
+
+    public static String createDimensionStatement(List<DimensionDesc> dimensionList, Set<BitSet> selected,
+            final int maxNumOfDimension) {
+        StringBuilder sql = new StringBuilder();
+
+        BitSet bitSet;
+
+        do {
+            bitSet = generateIfSelectList(dimensionList.size(),
+                    Math.ceil(maxNumOfDimension * Math.random()) / dimensionList.size());
+        } while (bitSet.cardinality() > maxNumOfDimension || bitSet.cardinality() <= 0 || selected.contains(bitSet));
+        selected.add(bitSet);
+
+        List<String> selectedCols = Lists.newArrayList();
+        int j = 0;
+        for (int i = 0; i < dimensionList.size(); i++) {
+            if (bitSet.get(i)) {
+                DimensionDesc dimensionDesc = dimensionList.get(i);
+                String tableName = getTableName(dimensionDesc.getTable());
+                String columnName = dimensionDesc.getColumn();
+                if (Strings.isNullOrEmpty(columnName) || columnName.equals("{FK}")) {
+                    String[] derivedCols = dimensionDesc.getDerived();
+                    BitSet subBitSet;
+                    do {
+                        subBitSet = generateIfSelectList(derivedCols.length, 0.5);
+                    } while (subBitSet.cardinality() <= 0);
+
+                    for (int k = 0; k < derivedCols.length; k++) {
+                        if (subBitSet.get(k)) {
+                            if (j > 0) {
+                                sql.append(",");
+                            }
+                            sql.append(tableName + ".\"" + derivedCols[k] + "\"\n");
+                            selectedCols.add(derivedCols[k]);
+                            j++;
+                        }
+                    }
+                } else {
+                    if (j > 0) {
+                        sql.append(",");
+                    }
+                    sql.append(tableName + ".\"" + columnName + "\"\n");
+                    selectedCols.add(columnName);
+                    j++;
+                }
+            }
+        }
+
+        return sql.toString();
+    }
+
+    public static BitSet generateIfSelectList(int n, double threshold) {
+        BitSet bitSet = new BitSet(n);
+        for (int i = 0; i < n; i++) {
+            if (Math.random() < threshold) {
+                bitSet.set(i);
+            }
+        }
+        return bitSet;
+    }
+
+    public static String getTableName(String name) {
+        int lastIndexOfDot = name.lastIndexOf(".");
+        if (lastIndexOfDot >= 0) {
+            name = name.substring(lastIndexOfDot + 1);
+        }
+        return name;
+    }
+}
diff --git a/tool/src/main/java/org/apache/kylin/tool/query/QueryGeneratorCLI.java b/tool/src/main/java/org/apache/kylin/tool/query/QueryGeneratorCLI.java
new file mode 100644
index 0000000..d1b7f98
--- /dev/null
+++ b/tool/src/main/java/org/apache/kylin/tool/query/QueryGeneratorCLI.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.tool.query;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.AbstractApplication;
+import org.apache.kylin.common.util.OptionsHelper;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.cube.CubeDescManager;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Strings;
+
+public class QueryGeneratorCLI extends AbstractApplication {
+
+    private static final Logger logger = LoggerFactory.getLogger(QueryGeneratorCLI.class);
+
+    private static final Option OPTION_MAX_DIM_NUM = OptionBuilder.withArgName("maxDimNum").hasArg().isRequired(false)
+            .withDescription("Specify the maximum number of dimensions for generating a query").create("maxDimNum");
+    private static final Option OPTION_CUBE = OptionBuilder.withArgName("cube").hasArg().isRequired(true)
+            .withDescription("Specify for which cube to generate query").create("cube");
+    private static final Option OPTION_SIZE = OptionBuilder.withArgName("size").hasArg().isRequired(true)
+            .withDescription("Specify the size of query set to be generated").create("size");
+    private static final Option OPTION_OUTPUT = OptionBuilder.withArgName("output").hasArg().isRequired(true)
+            .withDescription("Specify the output path for generated query set").create("output");
+
+    public static final String SQL_SEPARATOR = "#############";
+
+    protected final Options options;
+
+    private int sizeOfQueryList;
+    private String outputPath;
+    private int maxNumOfDim = 3;
+
+    public QueryGeneratorCLI() {
+        options = new Options();
+        options.addOption(OPTION_MAX_DIM_NUM);
+        options.addOption(OPTION_CUBE);
+        options.addOption(OPTION_SIZE);
+        options.addOption(OPTION_OUTPUT);
+    }
+
+    protected Options getOptions() {
+        return options;
+    }
+
+    protected void execute(OptionsHelper optionsHelper) throws Exception {
+        String temp = optionsHelper.getOptionValue(OPTION_MAX_DIM_NUM);
+        if (!Strings.isNullOrEmpty(temp)) {
+            this.maxNumOfDim = Integer.parseInt(temp);
+        }
+
+        this.outputPath = optionsHelper.getOptionValue(OPTION_OUTPUT);
+        this.sizeOfQueryList = Integer.parseInt(optionsHelper.getOptionValue(OPTION_SIZE));
+
+        String cubeName = optionsHelper.getOptionValue(OPTION_CUBE);
+        run(cubeName, true);
+    }
+
+    public Pair<List<String>, double[]> execute(String cubeName, int sizeOfQueryList, String outputPath)
+            throws Exception {
+        this.outputPath = outputPath;
+        this.sizeOfQueryList = sizeOfQueryList;
+
+        return run(cubeName, true);
+    }
+
+    public Pair<List<String>, double[]> execute(String cubeName, int sizeOfQueryList) throws Exception {
+        this.sizeOfQueryList = sizeOfQueryList;
+        return run(cubeName, false);
+    }
+
+    private Pair<List<String>, double[]> run(String cubeName, boolean needToStore) throws Exception {
+        CubeDesc cubeDesc = CubeDescManager.getInstance(KylinConfig.getInstanceFromEnv()).getCubeDesc(cubeName);
+
+        //Generate query list
+        List<String> queryList = QueryGenerator.generateQueryList(cubeDesc, sizeOfQueryList, maxNumOfDim);
+        ProbabilityGeneratorCLI probabilityGeneratorCLI = new ProbabilityGeneratorCLI();
+        double[] pCumArray;
+        if (needToStore) {
+            storeQuery(queryList, outputPath + "/" + cubeName);
+            pCumArray = probabilityGeneratorCLI.execute(queryList.size(), outputPath + "/" + cubeName);
+        } else {
+            pCumArray = probabilityGeneratorCLI.execute(queryList.size());
+        }
+        return new Pair<>(queryList, pCumArray);
+    }
+
+    public static void storeQuery(List<String> querySet, String outputPath) throws IOException {
+        String fileName = outputPath + ".sql";
+        File parentFile = new File(fileName).getParentFile();
+        if (!parentFile.exists()) {
+            parentFile.mkdirs();
+        }
+        try (BufferedWriter bufferedWriter = new BufferedWriter(
+                new OutputStreamWriter(new FileOutputStream(fileName), StandardCharsets.UTF_8))) {
+            for (String query : querySet) {
+                bufferedWriter.append(query);
+                bufferedWriter.append(SQL_SEPARATOR + "\n");
+                logger.info(query);
+            }
+        }
+    }
+
+    public static void main(String[] args) {
+        QueryGeneratorCLI queryGeneratorCLI = new QueryGeneratorCLI();
+        queryGeneratorCLI.execute(args);
+    }
+}
diff --git a/tool/src/test/java/org/apache/kylin/tool/query/ProbabilityGeneratorCLITest.java b/tool/src/test/java/org/apache/kylin/tool/query/ProbabilityGeneratorCLITest.java
new file mode 100755
index 0000000..a9a79af
--- /dev/null
+++ b/tool/src/test/java/org/apache/kylin/tool/query/ProbabilityGeneratorCLITest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.tool.query;
+
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ProbabilityGeneratorCLITest extends LocalFileMetadataTestCase {
+
+    public final String cubeName = "test_kylin_cube_with_slr_desc";
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testExecute() throws Exception {
+        ProbabilityGeneratorCLI probabilityGeneratorCLI = new ProbabilityGeneratorCLI();
+        double[] pCumArray = probabilityGeneratorCLI.execute(100, cubeName);
+        for (double pCum : pCumArray) {
+            System.out.print(pCum + " ");
+        }
+        int pIdx = ProbabilityGenerator.searchIndex(0.5, pCumArray);
+        System.out.print("\n" + pIdx);
+    }
+}
diff --git a/tool/src/test/java/org/apache/kylin/tool/query/QueryGeneratorCLITest.java b/tool/src/test/java/org/apache/kylin/tool/query/QueryGeneratorCLITest.java
new file mode 100755
index 0000000..ff71ac0
--- /dev/null
+++ b/tool/src/test/java/org/apache/kylin/tool/query/QueryGeneratorCLITest.java
@@ -0,0 +1,54 @@
+/*
+ * 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.tool.query;
+
+import java.util.List;
+
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.common.util.Pair;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class QueryGeneratorCLITest extends LocalFileMetadataTestCase {
+
+    public final String cubeName = "test_kylin_cube_with_slr_desc";
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testExecute() throws Exception {
+        QueryGeneratorCLI queryGeneratorCLI = new QueryGeneratorCLI();
+        Pair<List<String>, double[]> result = queryGeneratorCLI.execute(cubeName, 10);
+        List<String> sqls = result.getFirst();
+        double[] probs = result.getSecond();
+        for (int i = 0; i < sqls.size(); i++) {
+            System.out.println("Accumulate Probability: " + probs[i]);
+            System.out.println("SQL: " + sqls.get(i));
+        }
+    }
+}


[kylin] 14/15: KYLIN-4485 Minor fix

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit 36fa31b6b983edaa5f50879ae3c31fe6e353530e
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Tue Jun 9 10:18:52 2020 +0800

    KYLIN-4485 Minor fix
---
 .../java/org/apache/kylin/common/KConstants.java   | 23 -------------------
 .../org/apache/kylin/common/KylinConfigBase.java   | 26 ++++++++++++++++++----
 .../src/main/resources/kylin-defaults.properties   |  3 +++
 .../kylin/metadata/TableMetadataManager.java       |  2 +-
 .../kylin/rest/service/MigrationService.java       |  2 +-
 .../apache/kylin/rest/service/TableService.java    |  2 +-
 .../kylin/rest/service/AdminServiceTest.java       |  1 +
 .../org/apache/kylin/tool/CubeMigrationCLI.java    |  4 ++--
 webapp/app/js/services/kylinProperties.js          |  8 +++++++
 webapp/app/partials/cubes/cubes.html               |  6 ++---
 10 files changed, 42 insertions(+), 35 deletions(-)

diff --git a/core-common/src/main/java/org/apache/kylin/common/KConstants.java b/core-common/src/main/java/org/apache/kylin/common/KConstants.java
deleted file mode 100644
index 5e1723c..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/KConstants.java
+++ /dev/null
@@ -1,23 +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.common;
-
-public class KConstants {
-    public static final int DEFAULT_SERVICE_PORT = 7070;
-}
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 7db46a6..041f545 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -204,6 +204,15 @@ public abstract class KylinConfigBase implements Serializable {
         }
     }
 
+    final protected String[] getRawStringArray(String prop, String[] dft) {
+        final String property = properties.getProperty(prop);
+        if (!StringUtils.isBlank(property)) {
+            return property.split("\\s*,\\s*");
+        } else {
+            return dft;
+        }
+    }
+
     final protected int[] getOptionalIntArray(String prop, String[] dft) {
         String[] strArray = getOptionalStringArray(prop, dft);
         int[] intArray = new int[strArray.length];
@@ -711,7 +720,7 @@ public abstract class KylinConfigBase implements Serializable {
     }
 
     public boolean isRowKeyEncodingAutoConvert() {
-        return Boolean.parseBoolean(getOptional("kylin.cube.kylin.cube.rowkey-encoding-auto-convert", "true"));
+        return Boolean.parseBoolean(getOptional("kylin.cube.rowkey-encoding-auto-convert", "true"));
     }
     
     public String getSegmentAdvisor() {
@@ -2136,6 +2145,10 @@ public abstract class KylinConfigBase implements Serializable {
         return getOptionalStringArray("kylin.server.cluster-servers", new String[0]);
     }
 
+    public String[] getRawRestServers() {
+        return getRawStringArray("kylin.server.cluster-servers", new String[0]);
+    }
+
     public String getServerRestAddress() {
         return getOptional("kylin.server.host-address", "localhost:7070");
     }
@@ -2250,6 +2263,7 @@ public abstract class KylinConfigBase implements Serializable {
 
     public String getPropertiesWhiteList() {
         return getOptional("kylin.web.properties.whitelist", "kylin.web.timezone,kylin.query.cache-enabled,kylin.env,"
+                + "kylin.cube.migration.enabled,"
                 + "kylin.web.hive-limit,kylin.storage.default,"
                 + "kylin.engine.default,kylin.web.link-hadoop,kylin.web.link-diagnostic,"
                 + "kylin.web.contact-mail,kylin.web.help.length,kylin.web.help.0,kylin.web.help.1,kylin.web.help.2,"
@@ -2367,6 +2381,10 @@ public abstract class KylinConfigBase implements Serializable {
     // ============================================================================
     // CUBE MIGRATION
     // ============================================================================
+    public boolean isMigrationCubeEnabled() {
+        return Boolean.parseBoolean(getOptional("kylin.cube.migration.enabled", FALSE));
+    }
+
     public int getMigrationRuleExpansionRateThreshold() {
         return Integer.parseInt(getOptional("kylin.cube.migration.expansion-rate", "5"));
     }
@@ -2395,12 +2413,12 @@ public abstract class KylinConfigBase implements Serializable {
         return getOptional("kylin.cube.migration.target-address", "sandbox:80");
     }
 
-    public String getMigrationEmailSuffix() {
-        return getOptional("kylin.cube.migration.mail-suffix", "@mail.com");
+    public String getNotificationMailSuffix() {
+        return getOptional("kylin.cube.notification-mail-suffix", "@mail.com");
     }
 
     public boolean isMigrationApplyQueryLatencyRule() {
-        return Boolean.parseBoolean(getOptional("kylin.cube.migration.rule-query-latency-enabled", "true"));
+        return Boolean.parseBoolean(getOptional("kylin.cube.migration.rule-query-latency-enabled", TRUE));
     }
 
     // ============================================================================
diff --git a/core-common/src/main/resources/kylin-defaults.properties b/core-common/src/main/resources/kylin-defaults.properties
index 1dee90f..c16419a 100644
--- a/core-common/src/main/resources/kylin-defaults.properties
+++ b/core-common/src/main/resources/kylin-defaults.properties
@@ -77,6 +77,9 @@ kylin.web.link-diagnostic=
 kylin.web.contact-mail=
 kylin.server.external-acl-provider=
 
+### CUBE MIGRATION
+kylin.cube.migration.enabled=false
+
 # Default time filter for job list, 0->current day, 1->last one day, 2->last one week, 3->last one year, 4->all
 kylin.web.default-time-filter=1
 
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/TableMetadataManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/TableMetadataManager.java
index ec903d4..0d84c34 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/TableMetadataManager.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/TableMetadataManager.java
@@ -327,7 +327,7 @@ public class TableMetadataManager {
 
     public void reloadTableExtQuietly(String table, String project) {
         try (AutoLock lock = srcExtMapLock.lockForWrite()) {
-            srcExtCrud.reloadQuietly(TableExtDesc.concatResourcePath(table, project));
+            srcExtCrud.reloadQuietly(TableDesc.makeResourceName(table, project));
         }
     }
 
diff --git a/cube-migration/src/main/java/org/apache/kylin/rest/service/MigrationService.java b/cube-migration/src/main/java/org/apache/kylin/rest/service/MigrationService.java
index ca1e71b..67fce7e 100644
--- a/cube-migration/src/main/java/org/apache/kylin/rest/service/MigrationService.java
+++ b/cube-migration/src/main/java/org/apache/kylin/rest/service/MigrationService.java
@@ -169,7 +169,7 @@ public class MigrationService extends BasicService {
         logger.info("ProjectUUID : " + prjInstance.getUuid());
         Acl acl = accessService.getAcl(ae);
 
-        String mailSuffix = KylinConfig.getInstanceFromEnv().getMigrationEmailSuffix();
+        String mailSuffix = KylinConfig.getInstanceFromEnv().getNotificationMailSuffix();
         List<String> cubeAdmins = Lists.newArrayList();
         if (acl != null) {
             for (AccessControlEntry ace : acl.getEntries()) {
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java b/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
index 0420ad8..0d3f9f0 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
@@ -141,7 +141,7 @@ public class TableService extends BasicService {
             hiveTableDesc = pair.getFirst();
         } catch (Exception e) {
             logger.error("Fail to get metadata for hive table {} due to ", tableDesc.getIdentity(), e);
-            throw new RuntimeException("Fail to get metadata for hive table" + tableDesc.getIdentity());
+            throw new RuntimeException("Fail to get metadata for hive table " + tableDesc.getIdentity());
         }
 
         TableSchemaUpdateChecker.CheckResult result = getSchemaUpdateChecker().allowMigrate(tableDesc, hiveTableDesc);
diff --git a/server/src/test/java/org/apache/kylin/rest/service/AdminServiceTest.java b/server/src/test/java/org/apache/kylin/rest/service/AdminServiceTest.java
index dd098de..e732580 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/AdminServiceTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/AdminServiceTest.java
@@ -63,6 +63,7 @@ public class AdminServiceTest extends ServiceTestBase {
                     "kylin.cube.cubeplanner.enabled=true\n" +
                     "kylin.web.help=\n" +
                     "kylin.web.export-allow-other=true\n" +
+                    "kylin.cube.migration.enabled=false\n" +
                     "kylin.web.link-hadoop=\n" +
                     "kylin.web.hide-measures=RAW\n" +
                     "kylin.htrace.show-gui-trace-toggle=false\n" +
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 9212d08..e8123dd 100644
--- a/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
+++ b/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
@@ -645,10 +645,10 @@ public class CubeMigrationCLI extends AbstractApplication {
     }
 
     private void updateMeta(KylinConfig config, String projectName, String cubeName, DataModelDesc model) {
-        String[] nodes = config.getRestServers();
+        String[] nodes = config.getRawRestServers();
         Map<String, String> tableToProjects = new HashMap<>();
         for (TableRef tableRef : model.getAllTables()) {
-            tableToProjects.put(tableRef.getTableIdentity(), tableRef.getTableDesc().getProject());
+            tableToProjects.put(tableRef.getTableIdentity(), projectName);
         }
 
         for (String node : nodes) {
diff --git a/webapp/app/js/services/kylinProperties.js b/webapp/app/js/services/kylinProperties.js
index 05fce98..4762753 100644
--- a/webapp/app/js/services/kylinProperties.js
+++ b/webapp/app/js/services/kylinProperties.js
@@ -66,6 +66,14 @@ KylinApp.service('kylinConfig', function (AdminService, $log) {
     return false;
   }
 
+  this.isCubeMigrationEnabled = function(){
+    var status = this.getProperty("kylin.cube.migration.enabled").trim();
+    if(status!=='false'){
+      return true;
+    }
+    return false;
+  }
+
   //deprecated
   this.getDeployEnv = function () {
     this.deployEnv = this.getProperty("kylin.env");
diff --git a/webapp/app/partials/cubes/cubes.html b/webapp/app/partials/cubes/cubes.html
index dba3c8b..605f4e5 100644
--- a/webapp/app/partials/cubes/cubes.html
+++ b/webapp/app/partials/cubes/cubes.html
@@ -102,7 +102,7 @@
                         <li ng-if="cube.status=='DISABLED' && (userService.hasRole('ROLE_ADMIN') || hasPermission('cube',cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask))"><a ng-click="startDeleteSegment(cube)">Delete Segment</a></li>
                         <li ng-if="cube.status=='DISABLED' && (userService.hasRole('ROLE_ADMIN') || hasPermission('cube',cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask))"><a ng-click="purge(cube)">Purge</a></li>
                         <li ng-if="cube.status!='DESCBROKEN' && (userService.hasRole('ROLE_ADMIN') || hasPermission('cube',cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask))"><a ng-click="cloneCube(cube)">Clone</a></li>
-                        <li ng-if="cube.status=='READY' && (userService.hasRole('ROLE_ADMIN') || hasPermission(cube, permissions.ADMINISTRATION.mask) || hasPermission(cubeProjectEntity, permissions.ADMINISTRATION.mask))"><a ng-click="startMigrateCube(cube, 0);">Migrate</a></li>
+                        <li ng-if="cube.status=='READY' && kylinConfig.isCubeMigrationEnabled() && (userService.hasRole('ROLE_ADMIN') || hasPermission(cube, permissions.ADMINISTRATION.mask) || hasPermission(cubeProjectEntity, permissions.ADMINISTRATION.mask))"><a ng-click="startMigrateCube(cube, 0);">Migrate</a></li>
 
                     </ul>
                     <ul ng-if="(userService.hasRole('ROLE_ADMIN') || hasPermission('cube', cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask, permissions.OPERATION.mask)) && cube.streamingV2 && actionLoaded" class="dropdown-menu" role="menu" style="right:0;left:auto;">
@@ -132,8 +132,8 @@
                     <ul class="dropdown-menu" role="menu" style="right:0;left:auto;">
                         <li ng-if="cube.status!='READY'"><a href="cubes/edit/{{cube.name}}/descriptionjson">Edit CubeDesc</a></li>
                         <li><a href="cubes/view/{{cube.name}}/instancejson">View Cube</a></li>
-                        <li ng-if="cube.status=='READY' && kylinConfig.getDeployEnv().indexOf('QA') > -1"><a ng-click="startMigrateCube(cube, 1)">Approve Migration</a></li>
-                        <li ng-if="cube.status=='READY' && kylinConfig.getDeployEnv().indexOf('QA') > -1"><a ng-click="startMigrateCube(cube, -1)">Reject Migration</a></li>
+                        <li ng-if="cube.status=='READY' && kylinConfig.isCubeMigrationEnabled()"><a ng-click="startMigrateCube(cube, 1)">Approve Migration</a></li>
+                        <li ng-if="cube.status=='READY' && kylinConfig.isCubeMigrationEnabled()"><a ng-click="startMigrateCube(cube, -1)">Reject Migration</a></li>
                     </ul>
                 </div>
             </td>


[kylin] 04/15: KYLIN-4421 Add a rest API for updating table & database name

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit 70521fd7937d752085af24e55d39509b8bbd40c2
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Mon Apr 13 15:15:46 2020 +0800

    KYLIN-4421 Add a rest API for updating table & database name
---
 .../kylin/metadata/TableMetadataManager.java       |  27 ++++-
 .../kylin/metadata/project/ProjectManager.java     |  16 +++
 .../kylin/rest/controller/TableController.java     |  17 +++
 .../kylin/rest/request/TableUpdateRequest.java     |  44 ++++++++
 .../apache/kylin/rest/service/TableService.java    | 115 +++++++++++++++++++++
 .../service/update/TableSchemaUpdateMapping.java   |   3 +-
 6 files changed, 217 insertions(+), 5 deletions(-)

diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/TableMetadataManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/TableMetadataManager.java
index 932f631..ec903d4 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/TableMetadataManager.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/TableMetadataManager.java
@@ -185,7 +185,7 @@ public class TableMetadataManager {
             Map<String, TableDesc> ret = new LinkedHashMap<>();
             for (String tableName : prjTableNames) {
                 String tableIdentity = getTableIdentity(tableName);
-                ret.put(tableIdentity, getProjectSpecificTableDesc(tableIdentity, prj));
+                ret.put(tableIdentity, getProjectSpecificTableDesc(tableIdentity, prj, true));
             }
             return ret;
         }
@@ -195,8 +195,12 @@ public class TableMetadataManager {
      * Get TableDesc by name
      */
     public TableDesc getTableDesc(String tableName, String prj) {
+        return getTableDesc(tableName, prj, true);
+    }
+
+    public TableDesc getTableDesc(String tableName, String prj, boolean ifUseGlobal) {
         try (AutoLock lock = srcTableMapLock.lockForWrite()) {
-            return getProjectSpecificTableDesc(getTableIdentity(tableName), prj);
+            return getProjectSpecificTableDesc(getTableIdentity(tableName), prj, ifUseGlobal);
         }
     }
 
@@ -205,11 +209,11 @@ public class TableMetadataManager {
      * 
      * All locks on srcTableMapLock are WRITE LOCKS because of this method!!
      */
-    private TableDesc getProjectSpecificTableDesc(String fullTableName, String prj) {
+    private TableDesc getProjectSpecificTableDesc(String fullTableName, String prj, boolean ifUseGlobal) {
         String key = mapKey(fullTableName, prj);
         TableDesc result = srcTableMap.get(key);
 
-        if (result == null) {
+        if (result == null && ifUseGlobal) {
             try (AutoLock lock = srcTableMapLock.lockForWrite()) {
                 result = srcTableMap.get(mapKey(fullTableName, null));
                 if (result != null) {
@@ -402,6 +406,21 @@ public class TableMetadataManager {
         }
     }
 
+    public void saveNewTableExtFromOld(String oldTableId, String prj, String newTableId) throws IOException {
+        try (AutoLock lock = srcExtMapLock.lockForWrite()) {
+            String path = TableExtDesc.concatResourcePath(oldTableId, prj);
+            ResourceStore store = getStore();
+            TableExtDesc newTableExt = store.getResource(path, TABLE_EXT_SERIALIZER);
+            if (newTableExt != null) {
+                newTableExt.setIdentity(newTableId);
+                newTableExt.setLastModified(0L);
+
+                newTableExt.init(prj);
+                srcExtCrud.save(newTableExt);
+            }
+        }
+    }
+
     private TableExtDesc convertOldTableExtToNewer(String resourceName) {
         ResourceStore store = getStore();
         Map<String, String> attrs = Maps.newHashMap();
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 c4ff1e0..15b6a2d 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
@@ -333,6 +333,22 @@ public class ProjectManager {
         }
     }
 
+    public void removeTableDescFromProject(String[] tableIdentities, String projectName) throws IOException {
+        try (AutoLock lock = prjMapLock.lockForWrite()) {
+            TableMetadataManager metaMgr = getTableManager();
+            ProjectInstance projectInstance = getProject(projectName);
+            for (String tableId : tableIdentities) {
+                TableDesc table = metaMgr.getTableDesc(tableId, projectName);
+                if (table == null) {
+                    throw new IllegalStateException("Cannot find table '" + tableId + "' in metadata manager");
+                }
+                projectInstance.removeTable(table.getIdentity());
+            }
+
+            save(projectInstance);
+        }
+    }
+
     public ProjectInstance addExtFilterToProject(String[] filters, String projectName) throws IOException {
         try (AutoLock lock = prjMapLock.lockForWrite()) {
             TableMetadataManager metaMgr = getTableManager();
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
index 2f9c07c..6c529b9 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
@@ -29,10 +29,12 @@ import java.util.Set;
 import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.rest.exception.BadRequestException;
 import org.apache.kylin.rest.exception.InternalErrorException;
 import org.apache.kylin.rest.exception.NotFoundException;
 import org.apache.kylin.rest.request.CardinalityRequest;
 import org.apache.kylin.rest.request.HiveTableRequest;
+import org.apache.kylin.rest.request.TableUpdateRequest;
 import org.apache.kylin.rest.response.TableSnapshotResponse;
 import org.apache.kylin.rest.service.TableACLService;
 import org.apache.kylin.rest.service.TableService;
@@ -133,6 +135,21 @@ public class TableController extends BasicController {
         return result;
     }
 
+    @RequestMapping(value = "/{project}/update", method = { RequestMethod.POST }, produces = { "application/json" })
+    @ResponseBody
+    public void updateHiveTables(@PathVariable String project, @RequestBody TableUpdateRequest request)
+            throws IOException {
+        try {
+            tableService.updateHiveTable(project, request.getMapping(), request.isUseExisting());
+        } catch (BadRequestException e) {
+            logger.error("Failed to update Hive Table", e);
+            throw e;
+        } catch (Throwable e) {
+            logger.error("Failed to update Hive Table", e);
+            throw new InternalErrorException(e.getLocalizedMessage(), e);
+        }
+    }
+
     @RequestMapping(value = "/{tables}/{project}", method = { RequestMethod.DELETE }, produces = { "application/json" })
     @ResponseBody
     public Map<String, String[]> unLoadHiveTables(@PathVariable String tables, @PathVariable String project) {
diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/TableUpdateRequest.java b/server-base/src/main/java/org/apache/kylin/rest/request/TableUpdateRequest.java
new file mode 100644
index 0000000..f645b52
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/request/TableUpdateRequest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.rest.request;
+
+import java.io.Serializable;
+import java.util.Map;
+
+import org.apache.kylin.rest.service.update.TableSchemaUpdateMapping;
+
+public class TableUpdateRequest implements Serializable {
+    private Map<String, TableSchemaUpdateMapping> mapping;
+    private boolean isUseExisting;
+
+    public Map<String, TableSchemaUpdateMapping> getMapping() {
+        return mapping;
+    }
+
+    public void setMapping(Map<String, TableSchemaUpdateMapping> mapping) {
+        this.mapping = mapping;
+    }
+
+    public boolean isUseExisting() {
+        return isUseExisting;
+    }
+
+    public void setIsUseExisting(boolean useExisting) {
+        isUseExisting = useExisting;
+    }
+}
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java b/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
index ea0b8b3..cd18d2b 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
@@ -27,7 +27,9 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
+import java.util.stream.Collectors;
 
 import javax.annotation.Nullable;
 
@@ -39,6 +41,8 @@ import org.apache.kylin.common.util.RandomUtil;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.CubeUpdate;
+import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.dict.lookup.ExtTableSnapshotInfo;
 import org.apache.kylin.dict.lookup.ExtTableSnapshotInfoManager;
 import org.apache.kylin.dict.lookup.IExtLookupTableCache.CacheState;
@@ -54,6 +58,7 @@ import org.apache.kylin.job.execution.ExecutableManager;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.metadata.TableMetadataManager;
 import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.ISourceAware;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TableExtDesc;
@@ -63,6 +68,8 @@ import org.apache.kylin.rest.msg.Message;
 import org.apache.kylin.rest.msg.MsgPicker;
 import org.apache.kylin.rest.response.TableDescResponse;
 import org.apache.kylin.rest.response.TableSnapshotResponse;
+import org.apache.kylin.rest.service.update.TableSchemaUpdateMapping;
+import org.apache.kylin.rest.service.update.TableSchemaUpdater;
 import org.apache.kylin.rest.util.AclEvaluate;
 import org.apache.kylin.source.IReadableTable;
 import org.apache.kylin.source.IReadableTable.TableSignature;
@@ -84,6 +91,7 @@ import org.apache.kylin.shaded.com.google.common.collect.LinkedHashMultimap;
 import org.apache.kylin.shaded.com.google.common.collect.Lists;
 import org.apache.kylin.shaded.com.google.common.collect.Maps;
 import org.apache.kylin.shaded.com.google.common.collect.SetMultimap;
+import org.apache.kylin.shaded.com.google.common.collect.Sets;
 
 @Component("tableService")
 public class TableService extends BasicService {
@@ -91,6 +99,10 @@ public class TableService extends BasicService {
     private static final Logger logger = LoggerFactory.getLogger(TableService.class);
 
     @Autowired
+    @Qualifier("cubeMgmtService")
+    private CubeService cubeService;
+
+    @Autowired
     @Qualifier("modelMgmtService")
     private ModelService modelService;
 
@@ -531,4 +543,107 @@ public class TableService extends BasicService {
         String[] dbTableName = HadoopUtil.parseHiveTableName(tableName);
         return (dbTableName[0] + "." + dbTableName[1]).toUpperCase(Locale.ROOT);
     }
+
+    /**
+     * 1. Check whether it's able to do the change
+     *      - related cube instance should be disabled
+     * 2. Get all influenced metadata
+     *      - table
+     *      - project
+     *      - model
+     *      - cube desc
+     *      - cube instance
+     * 3. Update the metadata
+     * 4. Save the updated metadata
+     *      - table
+     *      - project
+     *      - model
+     *      - cube desc
+     *      - cube instance
+     * 5. Delete dirty table metadata
+     */
+    public void updateHiveTable(String projectName, Map<String, TableSchemaUpdateMapping> mapping, boolean isUseExisting) throws IOException {
+        final ProjectInstance prjInstance = getProjectManager().getProject(projectName);
+        if (prjInstance == null) {
+            throw new BadRequestException("Project " + projectName + " does not exist");
+        }
+        // To deal with case sensitive issue for table resource path
+        final String project = prjInstance.getName();
+        aclEvaluate.checkProjectWritePermission(project);
+
+        // Check whether it's able to do the change
+        Set<CubeInstance> infCubes = cubeService.listAllCubes(project).stream()
+                .filter(cube -> isTablesUsed(cube.getModel(), mapping.keySet())).collect(Collectors.toSet());
+        Set<CubeInstance> readyCubeSet = infCubes.stream().filter(cube -> cube.isReady()).collect(Collectors.toSet());
+        if (!readyCubeSet.isEmpty()) {
+            throw new BadRequestException("Influenced cubes " + readyCubeSet + " should be disabled");
+        }
+
+        // Get influenced metadata and update the metadata
+        Map<String, TableDesc> newTables = mapping.keySet().stream()
+                .map(t -> getTableManager().getTableDesc(t, project)).collect(Collectors.toMap(t -> t.getIdentity(),
+                        t -> TableSchemaUpdater.dealWithMappingForTable(t, mapping)));
+        Map<String, String> existingTables = newTables.entrySet().stream()
+                .filter(t -> getTableManager().getTableDesc(t.getValue().getIdentity(), project, false) != null)
+                .collect(Collectors.toMap(t -> t.getKey(), t -> t.getValue().getIdentity()));
+        if (!existingTables.isEmpty()) {
+            if (isUseExisting) {
+                logger.info("Will use existing tables {}", existingTables.values());
+            } else {
+                throw new BadRequestException("Tables " + existingTables.values() + " already exist");
+            }
+        }
+        Map<String, DataModelDesc> newModels = prjInstance.getModels().stream()
+                .map(m -> getDataModelManager().getDataModelDesc(m)).filter(m -> isTablesUsed(m, mapping.keySet()))
+                .map(m -> TableSchemaUpdater.dealWithMappingForModel(m, mapping))
+                .collect(Collectors.toMap(m -> m.getName(), m -> m));
+
+        Map<String, CubeDesc> newCubeDescs = infCubes.stream()
+                .map(cube -> TableSchemaUpdater.dealWithMappingForCubeDesc(cube.getDescriptor(), mapping))
+                .collect(Collectors.toMap(cube -> cube.getName(), cube -> cube));
+        Map<String, CubeInstance> newCubes = infCubes.stream()
+                .map(cube -> TableSchemaUpdater.dealWithMappingForCube(cube, mapping))
+                .collect(Collectors.toMap(cube -> cube.getName(), cube -> cube));
+
+        // Save the updated metadata
+        // -- 1. table & table_ext
+        for (Map.Entry<String, TableDesc> entry : newTables.entrySet()) {
+            if (existingTables.containsKey(entry.getKey())) {
+                continue;
+            }
+            getTableManager().saveNewTableExtFromOld(entry.getKey(), project, entry.getValue().getIdentity());
+            getTableManager().saveSourceTable(entry.getValue(), project);
+        }
+        // -- 2. project
+        Set<String> newTableNames = newTables.values().stream().map(t -> t.getIdentity()).collect(Collectors.toSet());
+        getProjectManager().addTableDescToProject(newTableNames.toArray(new String[0]), project);
+        // -- 3. model
+        for (Map.Entry<String, DataModelDesc> entry : newModels.entrySet()) {
+            getDataModelManager().updateDataModelDesc(entry.getValue());
+        }
+        // -- 4. cube_desc & cube instance
+        for (Map.Entry<String, CubeDesc> entry : newCubeDescs.entrySet()) {
+            getCubeDescManager().updateCubeDesc(entry.getValue());
+        }
+        for (Map.Entry<String, CubeInstance> entry : newCubes.entrySet()) {
+            CubeUpdate update = new CubeUpdate(entry.getValue());
+            getCubeManager().updateCube(update);
+        }
+
+        // Delete dirty table metadata
+        Set<String> oldTables = Sets.newHashSet(newTables.keySet());
+        oldTables.removeAll(existingTables.values());
+        getProjectManager().removeTableDescFromProject(oldTables.toArray(new String[0]), project);
+        for (String entry : newTables.keySet()) {
+            getTableManager().removeTableExt(entry, project);
+            getTableManager().removeSourceTable(entry, project);
+        }
+    }
+
+    private static boolean isTablesUsed(DataModelDesc model, Set<String> tables) {
+        Set<String> usingTables = model.getAllTables().stream().map(t -> t.getTableIdentity())
+                .collect(Collectors.toSet());
+        usingTables.retainAll(tables);
+        return !usingTables.isEmpty();
+    }
 }
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdateMapping.java b/server-base/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdateMapping.java
index 62b4a35..7b7f878 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdateMapping.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdateMapping.java
@@ -18,12 +18,13 @@
 
 package org.apache.kylin.rest.service.update;
 
+import java.io.Serializable;
 import java.util.Locale;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 
-public class TableSchemaUpdateMapping {
+public class TableSchemaUpdateMapping implements Serializable {
 
     private String database;
 


[kylin] 15/15: KYLIN-4429 Auto convert filter date values to date format string when the related column type is string

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit b5ef6dd7ddd587e4eccdebd2ab3efb74e2745571
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Fri Jun 12 13:40:04 2020 +0800

    KYLIN-4429 Auto convert filter date values to date format string when the related column type is string
---
 .../storage/translate/DerivedFilterTranslator.java | 27 +++++++++++++++++++++-
 .../query/relnode/visitor/TupleFilterVisitor.java  | 22 ++++++++++++++++++
 2 files changed, 48 insertions(+), 1 deletion(-)

diff --git a/core-storage/src/main/java/org/apache/kylin/storage/translate/DerivedFilterTranslator.java b/core-storage/src/main/java/org/apache/kylin/storage/translate/DerivedFilterTranslator.java
index 3aeabba..9bfdd76 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/translate/DerivedFilterTranslator.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/translate/DerivedFilterTranslator.java
@@ -23,10 +23,12 @@ import java.util.Set;
 
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Array;
+import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
 import org.apache.kylin.cube.model.CubeDesc.DeriveType;
 import org.apache.kylin.dict.lookup.ILookupTable;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.datatype.DataTypeOrder;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
@@ -60,7 +62,20 @@ public class DerivedFilterTranslator {
             assert hostCols.length == 1;
             CompareTupleFilter newComp = new CompareTupleFilter(compf.getOperator());
             newComp.addChild(new ColumnTupleFilter(hostCols[0]));
-            newComp.addChild(new ConstantTupleFilter(compf.getValues()));
+            Set<?> values = compf.getValues();
+            DataType pkDataType = compf.getColumn().getType();
+            if (pkDataType.isDateTimeFamily() && hostCols[0].getType().isStringFamily()) {
+                Set<String> newValues = Sets.newHashSetWithExpectedSize(values.size());
+                for (Object entry : values) {
+                    long ts = DateFormat.stringToMillis((String) entry);
+                    String newEntry = pkDataType.isDate() ? DateFormat.formatToDateStr(ts)
+                            : DateFormat.formatToTimeWithoutMilliStr(ts);
+                    newValues.add(newEntry);
+                }
+                newComp.addChild(new ConstantTupleFilter(newValues));
+            } else {
+                newComp.addChild(new ConstantTupleFilter(values));
+            }
             return new Pair<TupleFilter, Boolean>(newComp, false);
         }
 
@@ -83,6 +98,16 @@ public class DerivedFilterTranslator {
             }
         }
 
+        for (Array<String> entry : satisfyingHostRecords) {
+            for (int i = 0; i < pkCols.length; i++) {
+                if (pkCols[i].getType().isDateTimeFamily() && hostCols[i].getType().isStringFamily()) {
+                    long ts = DateFormat.stringToMillis(entry.data[i]);
+                    entry.data[i] = pkCols[i].getType().isDate() ? DateFormat.formatToDateStr(ts)
+                            : DateFormat.formatToTimeWithoutMilliStr(ts);
+                }
+            }
+        }
+        
         TupleFilter translated;
         boolean loosened;
         if (satisfyingHostRecords.size() > KylinConfig.getInstanceFromEnv().getDerivedInThreshold()) {
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/visitor/TupleFilterVisitor.java b/query/src/main/java/org/apache/kylin/query/relnode/visitor/TupleFilterVisitor.java
index e0792a0..ffff10b 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/visitor/TupleFilterVisitor.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/visitor/TupleFilterVisitor.java
@@ -149,6 +149,7 @@ public class TupleFilterVisitor extends RexVisitorImpl<TupleFilter> {
             filter = new UnsupportedTupleFilter(TupleFilter.FilterOperatorEnum.UNSUPPORTED);
         }
 
+        boolean isChildValueDateTimeType = false;
         for (RexNode operand : call.operands) {
             TupleFilter childFilter = operand.accept(this);
             if (filter == null) {
@@ -156,6 +157,27 @@ public class TupleFilterVisitor extends RexVisitorImpl<TupleFilter> {
             } else {
                 filter.addChild(childFilter);
             }
+            if (operand instanceof RexLiteral && ((RexLiteral) operand).getValue() instanceof GregorianCalendar) {
+                isChildValueDateTimeType = true;
+            }
+        }
+        if (filter instanceof CompareTupleFilter) {
+            CompareTupleFilter compFilter = (CompareTupleFilter) filter;
+            if (compFilter.getChildren().size() == 2 && compFilter.getChildren().get(0) instanceof ColumnTupleFilter
+                    && compFilter.getChildren().get(1) instanceof ConstantTupleFilter) {
+                ColumnTupleFilter colFilter = (ColumnTupleFilter) compFilter.getChildren().get(0);
+                ConstantTupleFilter constFilter = (ConstantTupleFilter) compFilter.getChildren().get(1);
+                if (isChildValueDateTimeType && colFilter.getColumn().getType().isStringFamily()) {
+                    Set<Object> newValues = Sets.newHashSet();
+                    for (Object v : constFilter.getValues()) {
+                        newValues.add(DateFormat.formatToDateStr(DateFormat.stringToMillis(v.toString())));
+                    }
+                    ConstantTupleFilter newConstFilter = new ConstantTupleFilter(newValues);
+                    filter = new CompareTupleFilter(filter.getOperator());
+                    filter.addChild(colFilter);
+                    filter.addChild(newConstFilter);
+                }
+            }
         }
 
         if (op.getKind() == SqlKind.OR) {


[kylin] 09/15: KYLIN-4499 Extract kylin server self discovery service from CuratorScheduler

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit c5d1a398e355631f3135d2a1a47661dfeb13144f
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Mon May 18 09:48:49 2020 +0800

    KYLIN-4499 Extract kylin server self discovery service from CuratorScheduler
---
 core-common/pom.xml                                |  11 ++
 .../java/org/apache/kylin/common/KConstants.java   |  23 +++
 .../java/org/apache/kylin/common/KylinConfig.java  |  14 +-
 .../org/apache/kylin/common/KylinConfigBase.java   |   4 +
 .../common/zookeeper/KylinServerDiscovery.java     | 164 +++++++++++++++++++++
 .../kylin/common/zookeeper}/ExampleServer.java     |  26 +---
 .../common/zookeeper/KylinServerDiscoveryTest.java |  46 +++---
 .../kylin/job/impl/curator/CuratorScheduler.java   | 100 +------------
 .../org/apache/kylin/rest/service/JobService.java  |   6 +
 9 files changed, 247 insertions(+), 147 deletions(-)

diff --git a/core-common/pom.xml b/core-common/pom.xml
index fe3af13..407ab9c 100644
--- a/core-common/pom.xml
+++ b/core-common/pom.xml
@@ -50,6 +50,12 @@
             <artifactId>jackson-databind</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-x-discovery</artifactId>
+            <!-- this jar was absent from hbase lib, so compile it  -->
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
             <groupId>org.freemarker</groupId>
             <artifactId>freemarker</artifactId>
         </dependency>
@@ -104,6 +110,11 @@
             <scope>test</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-test</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-shaded-guava</artifactId>
         </dependency>
diff --git a/core-common/src/main/java/org/apache/kylin/common/KConstants.java b/core-common/src/main/java/org/apache/kylin/common/KConstants.java
new file mode 100644
index 0000000..5e1723c
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/KConstants.java
@@ -0,0 +1,23 @@
+/*
+ * 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.common;
+
+public class KConstants {
+    public static final int DEFAULT_SERVICE_PORT = 7070;
+}
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
index 6297bd1..7b0888b 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -18,7 +18,6 @@
 
 package org.apache.kylin.common;
 
-import org.apache.kylin.shaded.com.google.common.base.Preconditions;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.restclient.RestClient;
@@ -48,6 +47,9 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.kylin.shaded.com.google.common.base.Strings;
+import org.apache.kylin.shaded.com.google.common.base.Preconditions;
+
 /**
  */
 public class KylinConfig extends KylinConfigBase {
@@ -525,7 +527,15 @@ public class KylinConfig extends KylinConfigBase {
             String value = entry.getValue().toString();
             orderedProperties.setProperty(key, value);
         }
-
+        // Reset some properties which might be overriden by system properties
+        String[] systemProps = { "kylin.server.cluster-servers", "kylin.server.cluster-servers-with-mode" };
+        for (String sysProp : systemProps) {
+            String sysPropValue = System.getProperty(sysProp);
+            if (!Strings.isNullOrEmpty(sysPropValue)) {
+                orderedProperties.setProperty(sysProp, sysPropValue);
+            }
+        }
+        
         final StringBuilder sb = new StringBuilder();
         for (Map.Entry<String, String> entry : orderedProperties.entrySet()) {
             sb.append(entry.getKey() + "=" + entry.getValue()).append('\n');
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index ff6138a..62021a9 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -2140,6 +2140,10 @@ public abstract class KylinConfigBase implements Serializable {
         return getOptional("kylin.server.host-address", "localhost:7070");
     }
 
+    public boolean getServerSelfDiscoveryEnabled() {
+        return Boolean.parseBoolean(getOptional("kylin.server.self-discovery-enabled", FALSE));
+    }
+
     public String getClusterName() {
         String key = "kylin.server.cluster-name";
         String clusterName = this.getOptional(key, getMetadataUrlPrefix());
diff --git a/core-common/src/main/java/org/apache/kylin/common/zookeeper/KylinServerDiscovery.java b/core-common/src/main/java/org/apache/kylin/common/zookeeper/KylinServerDiscovery.java
new file mode 100644
index 0000000..5b67d3e
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/zookeeper/KylinServerDiscovery.java
@@ -0,0 +1,164 @@
+/*
+ * 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.common.zookeeper;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.x.discovery.ServiceCache;
+import org.apache.curator.x.discovery.ServiceDiscovery;
+import org.apache.curator.x.discovery.ServiceDiscoveryBuilder;
+import org.apache.curator.x.discovery.ServiceInstance;
+import org.apache.curator.x.discovery.details.JsonInstanceSerializer;
+import org.apache.curator.x.discovery.details.ServiceCacheListener;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.ZKUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public class KylinServerDiscovery implements Closeable {
+
+    private static final Logger logger = LoggerFactory.getLogger(KylinServerDiscovery.class);
+
+    public static final String SERVICE_PATH = "/service";
+    public static final String SERVICE_NAME = "cluster_servers";
+    public static final String SERVICE_PAYLOAD_DESCRIPTION = "description";
+
+    private static class SingletonHolder {
+        private static final KylinServerDiscovery INSTANCE = new KylinServerDiscovery();
+    }
+
+    public static KylinServerDiscovery getInstance() {
+        return SingletonHolder.INSTANCE;
+    }
+
+    private final KylinConfig kylinConfig;
+    private final CuratorFramework curator;
+    private final ServiceDiscovery<LinkedHashMap> serviceDiscovery;
+    private final ServiceCache<LinkedHashMap> serviceCache;
+
+    private KylinServerDiscovery() {
+        this(KylinConfig.getInstanceFromEnv());
+    }
+
+    @VisibleForTesting
+    protected KylinServerDiscovery(KylinConfig kylinConfig) {
+        this.kylinConfig = kylinConfig;
+        this.curator = ZKUtil.getZookeeperClient(kylinConfig);
+        try {
+            final JsonInstanceSerializer<LinkedHashMap> serializer = new JsonInstanceSerializer<>(LinkedHashMap.class);
+            serviceDiscovery = ServiceDiscoveryBuilder.builder(LinkedHashMap.class).client(curator)
+                    .basePath(SERVICE_PATH).serializer(serializer).build();
+            serviceDiscovery.start();
+
+            serviceCache = serviceDiscovery.serviceCacheBuilder().name(SERVICE_NAME)
+                    .threadFactory(
+                            new ThreadFactoryBuilder().setDaemon(true).setNameFormat("KylinServerTracker-%d").build())
+                    .build();
+
+            final AtomicBoolean isFinishInit = new AtomicBoolean(false);
+            serviceCache.addListener(new ServiceCacheListener() {
+                @Override
+                public void stateChanged(CuratorFramework curatorFramework, ConnectionState connectionState) {
+                }
+
+                @Override
+                public void cacheChanged() {
+                    logger.info("Service discovery get cacheChanged notification");
+                    final List<ServiceInstance<LinkedHashMap>> instances = serviceCache.getInstances();
+                    Map<String, String> instanceNodes = Maps.newHashMapWithExpectedSize(instances.size());
+                    for (ServiceInstance<LinkedHashMap> entry : instances) {
+                        instanceNodes.put(entry.getAddress() + ":" + entry.getPort(),
+                                (String) entry.getPayload().get(SERVICE_PAYLOAD_DESCRIPTION));
+                    }
+
+                    logger.info("kylin.server.cluster-servers update to " + instanceNodes);
+                    // update cluster servers
+                    System.setProperty("kylin.server.cluster-servers", StringUtil.join(instanceNodes.keySet(), ","));
+
+                    // get servers and its mode(query, job, all)
+                    final String restServersInClusterWithMode = StringUtil.join(instanceNodes.entrySet().stream()
+                            .map(input -> input.getKey() + ":" + input.getValue()).collect(Collectors.toList()), ",");
+                    logger.info("kylin.server.cluster-servers-with-mode update to " + restServersInClusterWithMode);
+                    System.setProperty("kylin.server.cluster-servers-with-mode", restServersInClusterWithMode);
+                    isFinishInit.set(true);
+                }
+            });
+            serviceCache.start();
+
+            registerSelf();
+            while (!isFinishInit.get()) {
+                logger.info("Haven't registered, waiting ...");
+                Thread.sleep(100L);
+            }
+        } catch (Exception e) {
+            throw new RuntimeException("Fail to initialize due to ", e);
+        }
+    }
+
+    private void registerSelf() throws Exception {
+        String hostAddr = kylinConfig.getServerRestAddress();
+        String[] hostAddrInfo = hostAddr.split(":");
+        if (hostAddrInfo.length < 2) {
+            logger.error("kylin.server.host-address {} is not qualified ", hostAddr);
+            throw new RuntimeException("kylin.server.host-address " + hostAddr + " is not qualified");
+        }
+        String host = hostAddrInfo[0];
+        int port = Integer.parseInt(hostAddrInfo[1]);
+
+        String serverMode = kylinConfig.getServerMode();
+        registerServer(host, port, serverMode);
+    }
+
+    private void registerServer(String host, int port, String mode) throws Exception {
+        final LinkedHashMap<String, String> instanceDetail = new LinkedHashMap<>();
+        instanceDetail.put(SERVICE_PAYLOAD_DESCRIPTION, mode);
+
+        ServiceInstance<LinkedHashMap> thisInstance = ServiceInstance.<LinkedHashMap> builder().name(SERVICE_NAME)
+                .payload(instanceDetail).port(port).address(host).build();
+
+        for (ServiceInstance<LinkedHashMap> instance : serviceCache.getInstances()) {
+            // Check for registered instances to avoid being double registered
+            if (instance.getAddress().equals(thisInstance.getAddress())
+                    && instance.getPort().equals(thisInstance.getPort())) {
+                serviceDiscovery.unregisterService(instance);
+            }
+        }
+        serviceDiscovery.registerService(thisInstance);
+    }
+
+    @Override
+    public void close() throws IOException {
+        IOUtils.closeQuietly(serviceCache);
+        IOUtils.closeQuietly(serviceDiscovery);
+    }
+}
diff --git a/core-job/src/test/java/org/apache/kylin/job/impl/curator/ExampleServer.java b/core-common/src/test/java/org/apache/kylin/common/zookeeper/ExampleServer.java
similarity index 61%
rename from core-job/src/test/java/org/apache/kylin/job/impl/curator/ExampleServer.java
rename to core-common/src/test/java/org/apache/kylin/common/zookeeper/ExampleServer.java
index 66e3832..9e9fe95 100644
--- a/core-job/src/test/java/org/apache/kylin/job/impl/curator/ExampleServer.java
+++ b/core-common/src/test/java/org/apache/kylin/common/zookeeper/ExampleServer.java
@@ -16,38 +16,28 @@
  * limitations under the License.
  */
 
-package org.apache.kylin.job.impl.curator;
+package org.apache.kylin.common.zookeeper;
 
 import java.io.Closeable;
 import java.io.IOException;
 
-import org.apache.curator.framework.CuratorFramework;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.ZKUtil;
-import org.apache.kylin.job.engine.JobEngineConfig;
-import org.apache.kylin.job.exception.SchedulerException;
-import org.apache.kylin.job.lock.MockJobLock;
 
 /**
  */
 public class ExampleServer implements Closeable {
 
     private String address;
-    private CuratorScheduler scheduler;
+    private KylinServerDiscovery discovery;
 
-    public ExampleServer(String address) throws Exception {
+    public ExampleServer(String address) {
         this.address = address;
 
         KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
         KylinConfig kylinConfig1 = KylinConfig.createKylinConfig(kylinConfig);
         kylinConfig1.setProperty("kylin.server.host-address", address);
 
-        CuratorFramework client = ZKUtil.newZookeeperClient(kylinConfig1);
-        scheduler = new CuratorScheduler(client);
-        scheduler.init(new JobEngineConfig(kylinConfig1), new MockJobLock());
-        if (!scheduler.hasStarted()) {
-            throw new RuntimeException("scheduler has not been started");
-        }
+        discovery = new KylinServerDiscovery(kylinConfig1);
     }
 
     public String getAddress() {
@@ -56,13 +46,7 @@ public class ExampleServer implements Closeable {
 
     @Override
     public void close() throws IOException {
-
-        if (scheduler!= null)
-            try {
-                scheduler.shutdown();
-            } catch (SchedulerException e) {
-               //
-            }
+        discovery.close();
     }
 
 }
diff --git a/core-job/src/test/java/org/apache/kylin/job/impl/curator/CuratorSchedulerTest.java b/core-common/src/test/java/org/apache/kylin/common/zookeeper/KylinServerDiscoveryTest.java
similarity index 71%
rename from core-job/src/test/java/org/apache/kylin/job/impl/curator/CuratorSchedulerTest.java
rename to core-common/src/test/java/org/apache/kylin/common/zookeeper/KylinServerDiscoveryTest.java
index 4cf1410..efcb2ef 100644
--- a/core-job/src/test/java/org/apache/kylin/job/impl/curator/CuratorSchedulerTest.java
+++ b/core-common/src/test/java/org/apache/kylin/common/zookeeper/KylinServerDiscoveryTest.java
@@ -15,13 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.kylin.job.impl.curator;
+package org.apache.kylin.common.zookeeper;
 
 import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.List;
-
-import javax.annotation.Nullable;
+import java.util.stream.Collectors;
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.retry.ExponentialBackoffRetry;
@@ -30,9 +29,9 @@ import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.x.discovery.ServiceDiscovery;
 import org.apache.curator.x.discovery.ServiceDiscoveryBuilder;
 import org.apache.curator.x.discovery.ServiceInstance;
+import org.apache.curator.x.discovery.details.JsonInstanceSerializer;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.common.util.ZKUtil;
-import org.apache.kylin.job.execution.ExecutableManager;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -40,19 +39,16 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.kylin.shaded.com.google.common.base.Function;
 import org.apache.kylin.shaded.com.google.common.collect.Lists;
 
 /**
  */
-public class CuratorSchedulerTest extends LocalFileMetadataTestCase {
+public class KylinServerDiscoveryTest extends LocalFileMetadataTestCase {
 
-    private static final Logger logger = LoggerFactory.getLogger(CuratorSchedulerTest.class);
+    private static final Logger logger = LoggerFactory.getLogger(KylinServerDiscoveryTest.class);
 
     private TestingServer zkTestServer;
 
-    protected ExecutableManager jobService;
-
     @Before
     public void setup() throws Exception {
         zkTestServer = new TestingServer();
@@ -80,10 +76,8 @@ public class CuratorSchedulerTest extends LocalFileMetadataTestCase {
         ServiceDiscovery<LinkedHashMap> serviceDiscovery = null;
         CuratorFramework curatorClient = null;
         try {
-
-            final CuratorScheduler.JsonInstanceSerializer<LinkedHashMap> serializer = new CuratorScheduler.JsonInstanceSerializer<>(
-                    LinkedHashMap.class);
-            String servicePath = CuratorScheduler.KYLIN_SERVICE_PATH;
+            String servicePath = KylinServerDiscovery.SERVICE_PATH;
+            final JsonInstanceSerializer<LinkedHashMap> serializer = new JsonInstanceSerializer<>(LinkedHashMap.class);
             curatorClient = ZKUtil.newZookeeperClient(zkString, new ExponentialBackoffRetry(3000, 3));
             serviceDiscovery = ServiceDiscoveryBuilder.builder(LinkedHashMap.class).client(curatorClient)
                     .basePath(servicePath).serializer(serializer).build();
@@ -94,36 +88,32 @@ public class CuratorSchedulerTest extends LocalFileMetadataTestCase {
 
             Collection<String> serviceNames = serviceDiscovery.queryForNames();
             Assert.assertTrue(serviceNames.size() == 1);
-            Assert.assertTrue(CuratorScheduler.SERVICE_NAME.equals(serviceNames.iterator().next()));
+            Assert.assertTrue(KylinServerDiscovery.SERVICE_NAME.equals(serviceNames.iterator().next()));
             Collection<ServiceInstance<LinkedHashMap>> instances = serviceDiscovery
-                    .queryForInstances(CuratorScheduler.SERVICE_NAME);
+                    .queryForInstances(KylinServerDiscovery.SERVICE_NAME);
             Assert.assertTrue(instances.size() == 2);
             List<ServiceInstance<LinkedHashMap>> instancesList = Lists.newArrayList(instances);
 
-            final List<String> instanceNodes = Lists.transform(instancesList,
-                    new Function<ServiceInstance<LinkedHashMap>, String>() {
-
-                        @Nullable
-                        @Override
-                        public String apply(@Nullable ServiceInstance<LinkedHashMap> stringServiceInstance) {
-                            return (String) stringServiceInstance.getPayload()
-                                    .get(CuratorScheduler.SERVICE_PAYLOAD_DESCRIPTION);
-                        }
-                    });
+            final List<String> instanceNodes = instancesList.stream()
+                    .map(input -> input.getAddress() + ":" + input.getPort() + ":"
+                            + input.getPayload().get(KylinServerDiscovery.SERVICE_PAYLOAD_DESCRIPTION))
+                    .collect(Collectors.toList());
 
             Assert.assertTrue(instanceNodes.contains(server1.getAddress() + ":query"));
             Assert.assertTrue(instanceNodes.contains(server2.getAddress() + ":query"));
 
             // stop one server
             server1.close();
-            instances = serviceDiscovery.queryForInstances(CuratorScheduler.SERVICE_NAME);
+            instances = serviceDiscovery.queryForInstances(KylinServerDiscovery.SERVICE_NAME);
+            ServiceInstance<LinkedHashMap> existingInstance = instances.iterator().next();
             Assert.assertTrue(instances.size() == 1);
             Assert.assertEquals(server2.getAddress() + ":query",
-                    instances.iterator().next().getPayload().get(CuratorScheduler.SERVICE_PAYLOAD_DESCRIPTION));
+                    existingInstance.getAddress() + ":" + existingInstance.getPort() + ":"
+                            + existingInstance.getPayload().get(KylinServerDiscovery.SERVICE_PAYLOAD_DESCRIPTION));
 
             // all stop
             server2.close();
-            instances = serviceDiscovery.queryForInstances(CuratorScheduler.SERVICE_NAME);
+            instances = serviceDiscovery.queryForInstances(KylinServerDiscovery.SERVICE_NAME);
             Assert.assertTrue(instances.size() == 0);
 
         } finally {
diff --git a/core-job/src/main/java/org/apache/kylin/job/impl/curator/CuratorScheduler.java b/core-job/src/main/java/org/apache/kylin/job/impl/curator/CuratorScheduler.java
index faa7d71..7e829ef 100644
--- a/core-job/src/main/java/org/apache/kylin/job/impl/curator/CuratorScheduler.java
+++ b/core-job/src/main/java/org/apache/kylin/job/impl/curator/CuratorScheduler.java
@@ -21,27 +21,16 @@ package org.apache.kylin.job.impl.curator;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.net.InetAddress;
-import java.util.LinkedHashMap;
-import java.util.List;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.Collectors;
-
-import javax.annotation.Nullable;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.state.ConnectionState;
-import org.apache.curator.x.discovery.ServiceCache;
-import org.apache.curator.x.discovery.ServiceDiscovery;
-import org.apache.curator.x.discovery.ServiceDiscoveryBuilder;
 import org.apache.curator.x.discovery.ServiceInstance;
 import org.apache.curator.x.discovery.details.InstanceSerializer;
-import org.apache.curator.x.discovery.details.ServiceCacheListener;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.ServerMode;
-import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.common.util.ZKUtil;
 import org.apache.kylin.job.Scheduler;
 import org.apache.kylin.job.engine.JobEngineConfig;
@@ -54,9 +43,7 @@ import org.slf4j.LoggerFactory;
 import com.fasterxml.jackson.databind.DeserializationFeature;
 import com.fasterxml.jackson.databind.JavaType;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.kylin.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.kylin.shaded.com.google.common.base.Function;
-import org.apache.kylin.shaded.com.google.common.collect.Lists;
+import com.google.common.annotations.VisibleForTesting;
 
 public class CuratorScheduler implements Scheduler<AbstractExecutable> {
 
@@ -64,15 +51,10 @@ public class CuratorScheduler implements Scheduler<AbstractExecutable> {
     private boolean started = false;
     private CuratorFramework curatorClient = null;
     private static CuratorLeaderSelector jobClient = null;
-    private ServiceDiscovery<LinkedHashMap> serviceDiscovery = null;
-    private ServiceCache<LinkedHashMap> serviceCache = null;
     private KylinConfig kylinConfig;
     private AtomicInteger count = new AtomicInteger();
 
     static final String JOB_ENGINE_LEADER_PATH = "/job_engine/leader";
-    static final String KYLIN_SERVICE_PATH = "/service";
-    static final String SERVICE_NAME = "kylin";
-    static final String SERVICE_PAYLOAD_DESCRIPTION = "description";
 
     // the default constructor should exist for reflection initialization
     public CuratorScheduler() {
@@ -100,13 +82,7 @@ public class CuratorScheduler implements Scheduler<AbstractExecutable> {
                 curatorClient = ZKUtil.getZookeeperClient(kylinConfig);
             }
 
-            final String serverMode = jobEngineConfig.getConfig().getServerMode();
-            final String restAddress = kylinConfig.getServerRestAddress();
-            try {
-                registerInstance(restAddress, serverMode);
-            } catch (Exception e) {
-                throw new SchedulerException(e);
-            }
+            String restAddress = kylinConfig.getServerRestAddress();
 
             String jobEnginePath = JOB_ENGINE_LEADER_PATH;
 
@@ -120,79 +96,13 @@ public class CuratorScheduler implements Scheduler<AbstractExecutable> {
                     throw new SchedulerException(e);
                 }
             } else {
-                logger.info("server mode: " + jobEngineConfig.getConfig().getServerMode() + ", no need to run job scheduler");
+                logger.info("server mode: " + jobEngineConfig.getConfig().getServerMode()
+                        + ", no need to run job scheduler");
             }
             started = true;
         }
     }
 
-    private void registerInstance(String restAddress, String mode) throws Exception {
-        final String host = restAddress.substring(0, restAddress.indexOf(":"));
-        final String port = restAddress.substring(restAddress.indexOf(":") + 1);
-
-        final JsonInstanceSerializer<LinkedHashMap> serializer = new JsonInstanceSerializer<>(LinkedHashMap.class);
-        final String servicePath = KYLIN_SERVICE_PATH;
-        serviceDiscovery = ServiceDiscoveryBuilder.builder(LinkedHashMap.class).client(curatorClient)
-                .basePath(servicePath).serializer(serializer).build();
-        serviceDiscovery.start();
-
-        serviceCache = serviceDiscovery.serviceCacheBuilder().name(SERVICE_NAME)
-                .threadFactory(Executors.defaultThreadFactory()).build();
-
-        serviceCache.addListener(new ServiceCacheListener() {
-            @Override
-            public void stateChanged(CuratorFramework curatorFramework, ConnectionState connectionState) {
-            }
-
-            @Override
-            public void cacheChanged() {
-                logger.info("Service discovery get cacheChanged notification");
-                final List<ServiceInstance<LinkedHashMap>> instances = serviceCache.getInstances();
-                final List<String> instanceNodes = Lists.transform(instances,
-                        new Function<ServiceInstance<LinkedHashMap>, String>() {
-
-                            @Nullable
-                            @Override
-                            public String apply(@Nullable ServiceInstance<LinkedHashMap> stringServiceInstance) {
-                                return (String) stringServiceInstance.getPayload().get(SERVICE_PAYLOAD_DESCRIPTION);
-                            }
-                        });
-
-                final String restServersInCluster = //
-                        StringUtil.join(instanceNodes.stream().map(input -> { //
-                            String[] split = input.split(":"); //
-                            return split[0] + ":" + split[1]; //
-                        }).collect(Collectors.toList()), ","); //
-
-
-                logger.info("kylin.server.cluster-servers update to " + restServersInCluster);
-                // update cluster servers
-                System.setProperty("kylin.server.cluster-servers", restServersInCluster);
-
-                // get servers and its mode(query, job, all)
-                final String restServersInClusterWithMode = StringUtil.join(instanceNodes, ",");
-                logger.info("kylin.server.cluster-servers-with-mode update to " + restServersInClusterWithMode);
-                System.setProperty("kylin.server.cluster-servers-with-mode", restServersInClusterWithMode);
-            }
-        });
-        serviceCache.start();
-
-        final LinkedHashMap<String, String> instanceDetail = new LinkedHashMap<>();
-
-        instanceDetail.put(SERVICE_PAYLOAD_DESCRIPTION, restAddress + ":" + mode);
-        ServiceInstance<LinkedHashMap> thisInstance = ServiceInstance.<LinkedHashMap> builder().name(SERVICE_NAME)
-                .payload(instanceDetail).port(Integer.valueOf(port)).address(host).build();
-
-        for (ServiceInstance<LinkedHashMap> instance : serviceCache.getInstances()) {
-            // Check for registered instances to avoid being double registered
-            if (instance.getAddress().equals(thisInstance.getAddress())
-                    && instance.getPort().equals(thisInstance.getPort())) {
-                serviceDiscovery.unregisterService(instance);
-            }
-        }
-        serviceDiscovery.registerService(thisInstance);
-    }
-
     private void monitorJobEngine() {
         logger.info("Start collect monitor ZK Participants");
         Executors.newSingleThreadScheduledExecutor().scheduleAtFixedRate(new Runnable() {
@@ -220,8 +130,6 @@ public class CuratorScheduler implements Scheduler<AbstractExecutable> {
 
     @Override
     public void shutdown() throws SchedulerException {
-        IOUtils.closeQuietly(serviceCache);
-        IOUtils.closeQuietly(serviceDiscovery);
         IOUtils.closeQuietly(curatorClient);
         IOUtils.closeQuietly(jobClient);
         started = false;
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
index d1ec4cf..2f42410 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
@@ -37,6 +37,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.lock.DistributedLock;
 import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.common.zookeeper.KylinServerDiscovery;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
@@ -134,6 +135,11 @@ public class JobService extends BasicService implements InitializingBean {
         final Scheduler<AbstractExecutable> scheduler = (Scheduler<AbstractExecutable>) SchedulerFactory
                 .scheduler(kylinConfig.getSchedulerType());
 
+        if (kylinConfig.getServerSelfDiscoveryEnabled()) {
+            KylinServerDiscovery.getInstance();
+        }
+        logger.info("Cluster servers: {}", Lists.newArrayList(kylinConfig.getRestServers()));
+        
         scheduler.init(new JobEngineConfig(kylinConfig), new ZookeeperJobLock());
 
         Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {


[kylin] 07/15: KYLIN-4419 Refine IT sqls with adding order by

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit 67f347ba68093fb79b791b80ab959dd1921c8054
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Mon May 11 17:46:07 2020 +0800

    KYLIN-4419 Refine IT sqls with adding order by
---
 kylin-it/src/test/resources/query/sql_timestamp/query01.sql   | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query02.sql   | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query03.sql   | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query05.sql   | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query11.sql   | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query12.sql   | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query13.sql   | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query14.sql   | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query21.sql   | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query21_a.sql | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query22.sql   | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query22_a.sql | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query22_b.sql | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query22_c.sql | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query22_d.sql | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query22_e.sql | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query23.sql   | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query23_a.sql | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query25.sql   | 1 +
 kylin-it/src/test/resources/query/sql_timestamp/query25_a.sql | 1 +
 20 files changed, 20 insertions(+)

diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query01.sql b/kylin-it/src/test/resources/query/sql_timestamp/query01.sql
index 08e13a2..86a99dc 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query01.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query01.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query02.sql b/kylin-it/src/test/resources/query/sql_timestamp/query02.sql
index f372212..1b61f34 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query02.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query02.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query03.sql b/kylin-it/src/test/resources/query/sql_timestamp/query03.sql
index 0dc76bd..5db0fe5 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query03.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query03.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query05.sql b/kylin-it/src/test/resources/query/sql_timestamp/query05.sql
index 171f473..470e8f1 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query05.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query05.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query11.sql b/kylin-it/src/test/resources/query/sql_timestamp/query11.sql
index edb3d54..f24dfae 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query11.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query11.sql
@@ -19,3 +19,4 @@ SELECT sum(price)  as sum_price
  FROM TEST_KYLIN_FACT 
  WHERE CAL_DT > cast(TIMESTAMPADD(Day, -15000, CURRENT_DATE) as DATE)
 GROUP BY CAL_DT
+ORDER BY CAL_DT
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query12.sql b/kylin-it/src/test/resources/query/sql_timestamp/query12.sql
index 73b432f..dad4920 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query12.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query12.sql
@@ -19,3 +19,4 @@ SELECT sum(price)  as sum_price
  FROM TEST_KYLIN_FACT 
  WHERE CAL_DT > cast(TIMESTAMPADD(Day, -2000, CURRENT_DATE) as DATE)
 GROUP BY CAL_DT
+ORDER BY CAL_DT
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query13.sql b/kylin-it/src/test/resources/query/sql_timestamp/query13.sql
index fed0b73..98cb4d7 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query13.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query13.sql
@@ -19,3 +19,4 @@ SELECT sum(price)  as sum_price
  FROM TEST_KYLIN_FACT 
  WHERE CAL_DT > cast(TIMESTAMPADD(Day, -0, CURRENT_TIMESTAMP) as DATE)
 GROUP BY CAL_DT
+ORDER BY CAL_DT
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query14.sql b/kylin-it/src/test/resources/query/sql_timestamp/query14.sql
index 40378b9..89426b5 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query14.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query14.sql
@@ -19,3 +19,4 @@ SELECT sum(price)  as sum_price
  FROM TEST_KYLIN_FACT 
  WHERE CAL_DT > cast(TIMESTAMPADD(Day, -15000, CURRENT_TIMESTAMP ) as DATE)
 GROUP BY CAL_DT
+ORDER BY CAL_DT
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query21.sql b/kylin-it/src/test/resources/query/sql_timestamp/query21.sql
index d1b8945..e0e883c 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query21.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query21.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query21_a.sql b/kylin-it/src/test/resources/query/sql_timestamp/query21_a.sql
index 1d4d617..56324f6 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query21_a.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query21_a.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query22.sql b/kylin-it/src/test/resources/query/sql_timestamp/query22.sql
index 34524c2..eadcabc 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query22.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query22.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query22_a.sql b/kylin-it/src/test/resources/query/sql_timestamp/query22_a.sql
index d68a354..ff7c7a6 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query22_a.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query22_a.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query22_b.sql b/kylin-it/src/test/resources/query/sql_timestamp/query22_b.sql
index 4d4fa98..0c996a2 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query22_b.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query22_b.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query22_c.sql b/kylin-it/src/test/resources/query/sql_timestamp/query22_c.sql
index d8e8096..b491c4b 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query22_c.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query22_c.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query22_d.sql b/kylin-it/src/test/resources/query/sql_timestamp/query22_d.sql
index 5bf36dd..e213299 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query22_d.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query22_d.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query22_e.sql b/kylin-it/src/test/resources/query/sql_timestamp/query22_e.sql
index bb36d83..fc73a60 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query22_e.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query22_e.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query23.sql b/kylin-it/src/test/resources/query/sql_timestamp/query23.sql
index 9050894..1c4b16a 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query23.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query23.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query23_a.sql b/kylin-it/src/test/resources/query/sql_timestamp/query23_a.sql
index f3f7141..34cf1f3 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query23_a.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query23_a.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query25.sql b/kylin-it/src/test/resources/query/sql_timestamp/query25.sql
index 2fb8b23..8b239ea 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query25.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query25.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file
diff --git a/kylin-it/src/test/resources/query/sql_timestamp/query25_a.sql b/kylin-it/src/test/resources/query/sql_timestamp/query25_a.sql
index 2034f19..d379c58 100644
--- a/kylin-it/src/test/resources/query/sql_timestamp/query25_a.sql
+++ b/kylin-it/src/test/resources/query/sql_timestamp/query25_a.sql
@@ -26,3 +26,4 @@ inner JOIN edw.test_cal_dt as test_cal_dt
  inner JOIN edw.test_sites as test_sites
  ON test_kylin_fact.lstg_site_id = test_sites.site_id
  GROUP BY test_kylin_fact.cal_dt
+ ORDER BY test_kylin_fact.cal_dt
\ No newline at end of file


[kylin] 13/15: KYLIN-4489 Create a tool for migration cross clusters

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit 2e2a4dff9ca9c08351084f53d293e189083db140
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Tue May 19 11:14:42 2020 +0800

    KYLIN-4489 Create a tool for migration cross clusters
---
 .../java/org/apache/kylin/common/KylinConfig.java  |  20 +
 .../apache/kylin/common/restclient/RestClient.java |  40 +-
 .../kylin/metadata/model/DataModelManager.java     |   3 +
 .../kylin/metadata/project/ProjectManager.java     |   6 +-
 .../kylin/rest/controller/MigrationController.java |   2 +-
 pom.xml                                            |   5 +
 .../kylin/rest/controller/AdminController.java     |  22 +
 .../apache/kylin/rest/service/AdminService.java    |  14 +
 .../apache/kylin/rest/service/HBaseInfoUtil.java   |  17 +-
 .../apache/kylin/rest/service/ModelService.java    |   8 +-
 .../kylin/storage/hbase/HBaseResourceStore.java    |   7 +-
 .../storage/hbase/util/DeployCoprocessorCLI.java   |  21 +-
 tool/pom.xml                                       |  11 +
 .../apache/kylin/tool/migration/ClusterUtil.java   | 165 +++++
 .../migration/CubeMigrationCrossClusterCLI.java    | 757 +++++++++++++++++++++
 .../kylin/tool/migration/DstClusterUtil.java       | 371 ++++++++++
 .../kylin/tool/migration/SrcClusterUtil.java       | 148 ++++
 17 files changed, 1595 insertions(+), 22 deletions(-)

diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
index 7b0888b..c9001f0 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -20,6 +20,7 @@ package org.apache.kylin.common;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.kylin.common.restclient.RestClient;
 import org.apache.kylin.common.threadlocal.InternalThreadLocal;
 import org.apache.kylin.common.util.ClassUtil;
@@ -571,6 +572,25 @@ public class KylinConfig extends KylinConfigBase {
         reloadKylinConfig(buildSiteProperties());
     }
 
+    public static String getConfigAsString(Configuration conf) {
+        final StringBuilder sb = new StringBuilder();
+        for (Map.Entry<String, String> entry : conf) {
+            sb.append(entry.getKey() + "=" + entry.getValue()).append('\n');
+        }
+        return sb.toString();
+    }
+
+    public static Configuration getConfigFromString(String configInStr) throws IOException {
+        Properties props = new Properties();
+        props.load(new StringReader(configInStr));
+
+        Configuration config = new Configuration();
+        for (Map.Entry<Object, Object> entry : props.entrySet()) {
+            config.set((String) entry.getKey(), (String) entry.getValue());
+        }
+        return config;
+    }
+
     public KylinConfig base() {
         return this;
     }
diff --git a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
index d908f58..21b08f8 100644
--- a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
+++ b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
@@ -30,7 +30,6 @@ import java.util.regex.Pattern;
 
 import javax.xml.bind.DatatypeConverter;
 
-import com.google.common.base.Strings;
 import org.apache.http.HttpResponse;
 import org.apache.http.auth.AuthScope;
 import org.apache.http.auth.UsernamePasswordCredentials;
@@ -55,6 +54,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Strings;
 
 /**
  */
@@ -175,6 +175,25 @@ public class RestClient {
         }
     }
 
+    public void announceWipeCache(String entity, String event, String cacheKey) throws IOException {
+        String url = baseUrl + "/cache/announce/" + entity + "/" + cacheKey + "/" + event;
+        HttpPut request = new HttpPut(url);
+
+        try {
+            HttpResponse response = client.execute(request);
+
+            if (response.getStatusLine().getStatusCode() != 200) {
+                String msg = EntityUtils.toString(response.getEntity());
+                throw new IOException("Invalid response " + response.getStatusLine().getStatusCode()
+                        + " with announce cache wipe url " + url + "\n" + msg);
+            }
+        } catch (Exception ex) {
+            throw new IOException(ex);
+        } finally {
+            request.releaseConnection();
+        }
+    }
+    
     public void wipeCache(String entity, String event, String cacheKey) throws IOException {
         HttpPut request;
         String url;
@@ -202,8 +221,19 @@ public class RestClient {
     }
 
     public String getKylinProperties() throws IOException {
-        String url = baseUrl + "/admin/config";
-        HttpGet request = new HttpGet(url);
+        return getConfiguration(baseUrl + "/admin/config", false);
+    }
+
+    public String getHDFSConfiguration() throws IOException {
+        return getConfiguration(baseUrl + "/admin/config/hdfs", true);
+    }
+
+    public String getHBaseConfiguration() throws IOException {
+        return getConfiguration(baseUrl + "/admin/config/hbase", true);
+    }
+
+    private String getConfiguration(String url, boolean ifAuth) throws IOException {
+        HttpGet request = ifAuth ? newGet(url) : new HttpGet(url);
         HttpResponse response = null;
         try {
             response = client.execute(request);
@@ -372,7 +402,7 @@ public class RestClient {
                 String msg = getContent(response);
                 Map<String, String> kvMap = JsonUtil.readValueAsMap(msg);
                 String exception = kvMap.containsKey("exception") ? kvMap.get("exception") : "unknown";
-                throw new IOException(exception);
+                throw new IOException("Error code: " + response.getStatusLine().getStatusCode() + "\n" + exception);
             }
         } finally {
             post.releaseConnection();
@@ -411,7 +441,7 @@ public class RestClient {
     }
 
     private HttpGet newGet(String url) {
-        HttpGet get = new HttpGet();
+        HttpGet get = new HttpGet(url);
         addHttpHeaders(get);
         return get;
     }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelManager.java
index 47e2c3d..f483a2c 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelManager.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelManager.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
 import org.apache.kylin.common.persistence.WriteConflictException;
@@ -48,6 +49,8 @@ import org.apache.kylin.shaded.com.google.common.collect.Lists;
 public class DataModelManager {
 
     private static final Logger logger = LoggerFactory.getLogger(DataModelManager.class);
+    public static final Serializer<DataModelDesc> MODELDESC_SERIALIZER = new JsonSerializer<DataModelDesc>(
+            DataModelDesc.class);
 
     public static DataModelManager getInstance(KylinConfig config) {
         return config.getManager(DataModelManager.class);
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 15b6a2d..ebcd45c 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
@@ -30,7 +30,9 @@ import java.util.Set;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.persistence.Serializer;
 import org.apache.kylin.common.util.AutoReadWriteLock;
 import org.apache.kylin.common.util.AutoReadWriteLock.AutoLock;
 import org.apache.kylin.metadata.TableMetadataManager;
@@ -54,7 +56,9 @@ import org.apache.kylin.shaded.com.google.common.collect.Sets;
 
 public class ProjectManager {
     private static final Logger logger = LoggerFactory.getLogger(ProjectManager.class);
-
+    public static final Serializer<ProjectInstance> PROJECT_SERIALIZER = new JsonSerializer<ProjectInstance>(
+            ProjectInstance.class);
+    
     public static ProjectManager getInstance(KylinConfig config) {
         return config.getManager(ProjectManager.class);
     }
diff --git a/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java b/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java
index efef5cf..45f83ea 100644
--- a/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java
+++ b/cube-migration/src/main/java/org/apache/kylin/rest/controller/MigrationController.java
@@ -155,7 +155,7 @@ public class MigrationController extends BasicController {
             }
             DataModelDesc dataModelDesc = JsonUtil.readValue(request.getModelDescData(), DataModelDesc.class);
             logger.info("Schema compatibility check for model {}", dataModelDesc.getName());
-            modelService.checkModelCompatibility(request.getProjectName(), dataModelDesc, tableDescList);
+            modelService.checkModelCompatibility(dataModelDesc, tableDescList);
             logger.info("Pass schema compatibility check for model {}", dataModelDesc.getName());
         } catch (Exception e) {
             logger.error(e.getMessage(), e);
diff --git a/pom.xml b/pom.xml
index c73bf28..f60b7c8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -542,6 +542,11 @@
         <optional>true</optional>
       </dependency>
       <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-distcp</artifactId>
+        <version>${hadoop2.version}</version>
+      </dependency>
+      <dependency>
         <groupId>org.apache.zookeeper</groupId>
         <artifactId>zookeeper</artifactId>
         <version>${zookeeper.version}</version>
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/AdminController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/AdminController.java
index 4d90db8..780e069 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/AdminController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/AdminController.java
@@ -112,6 +112,28 @@ public class AdminController extends BasicController {
         return configRes;
     }
 
+    @RequestMapping(value = "/config/hdfs", method = { RequestMethod.GET }, produces = { "application/json" })
+    @ResponseBody
+    public GeneralResponse getHDFSConfig() throws IOException {
+        String config = adminService.getHadoopConfigAsString();
+
+        GeneralResponse configRes = new GeneralResponse();
+        configRes.put("config", config);
+
+        return configRes;
+    }
+
+    @RequestMapping(value = "/config/hbase", method = { RequestMethod.GET }, produces = { "application/json" })
+    @ResponseBody
+    public GeneralResponse getHBaseConfig() throws IOException {
+        String config = adminService.getHBaseConfigAsString();
+
+        GeneralResponse configRes = new GeneralResponse();
+        configRes.put("config", config);
+
+        return configRes;
+    }
+
     @RequestMapping(value = "/metrics/cubes", method = { RequestMethod.GET }, produces = { "application/json" })
     @ResponseBody
     public MetricsResponse cubeMetrics(MetricsRequest request) {
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/AdminService.java b/server-base/src/main/java/org/apache/kylin/rest/service/AdminService.java
index c133a28..4e9cd03 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/AdminService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/AdminService.java
@@ -31,10 +31,12 @@ import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.common.util.OrderedProperties;
 import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.rest.constant.Constant;
 import org.apache.kylin.rest.job.StorageCleanupJob;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.security.access.prepost.PreAuthorize;
@@ -117,4 +119,16 @@ public class AdminService extends BasicService {
 
         return KylinConfig.getInstanceFromEnv().exportToString(propertyKeys);
     }
+
+    public String getHadoopConfigAsString() throws IOException {
+        logger.debug("Get Kylin Hadoop Config");
+
+        return KylinConfig.getConfigAsString(HadoopUtil.getCurrentConfiguration());
+    }
+
+    public String getHBaseConfigAsString() throws IOException {
+        logger.debug("Get Kylin HBase Config");
+
+        return KylinConfig.getConfigAsString(HBaseConnection.getCurrentHBaseConfiguration());
+    }
 }
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/HBaseInfoUtil.java b/server-base/src/main/java/org/apache/kylin/rest/service/HBaseInfoUtil.java
index 8fc1de9..012f14a 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/HBaseInfoUtil.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/HBaseInfoUtil.java
@@ -20,6 +20,7 @@ package org.apache.kylin.rest.service;
 
 import java.io.IOException;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
@@ -28,13 +29,17 @@ import org.apache.kylin.storage.hbase.util.HBaseRegionSizeCalculator;
 import org.apache.kylin.storage.hbase.util.HBaseUnionUtil;
 
 public class HBaseInfoUtil {
-    
+
     @SuppressWarnings("unused") // used by reflection
     public static HBaseResponse getHBaseInfo(String tableName, KylinConfig config) throws IOException {
         if (!config.getStorageUrl().getScheme().equals("hbase"))
             return null;
-        
+
         Connection conn = HBaseUnionUtil.getConnection(config, tableName);
+        return getHBaseInfo(tableName, conn);
+    }
+
+    public static HBaseResponse getHBaseInfo(String tableName, Connection conn) throws IOException {
         HBaseResponse hr = null;
         long tableSize = 0;
         int regionCount = 0;
@@ -54,4 +59,12 @@ public class HBaseInfoUtil {
         hr.setRegionCount(regionCount);
         return hr;
     }
+
+    public static boolean checkEquals(HBaseResponse hbaseR1, HBaseResponse hbaseR2) {
+        if (hbaseR1 == hbaseR2)
+            return true;
+        return Objects.equals(hbaseR1.getTableName(), hbaseR2.getTableName())
+                && hbaseR1.getTableSize() == hbaseR2.getTableSize()
+                && hbaseR1.getRegionCount() == hbaseR2.getRegionCount();
+    }
 }
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
index 5a3a1ca..7ff3919 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
@@ -168,11 +168,7 @@ public class ModelService extends BasicService {
         result.raiseExceptionWhenInvalid();
     }
 
-    public void checkModelCompatibility(String project, DataModelDesc dataModalDesc, List<TableDesc> tableDescList) {
-        ProjectInstance prjInstance = getProjectManager().getProject(project);
-        if (prjInstance == null) {
-            throw new BadRequestException("Project " + project + " does not exist");
-        }
+    public void checkModelCompatibility(DataModelDesc dataModalDesc, List<TableDesc> tableDescList) {
         ModelSchemaUpdateChecker checker = new ModelSchemaUpdateChecker(getTableManager(), getCubeManager(),
                 getDataModelManager());
 
@@ -181,7 +177,7 @@ public class ModelService extends BasicService {
             tableDescMap.put(tableDesc.getIdentity(), tableDesc);
         }
         dataModalDesc.init(getConfig(), tableDescMap);
-        ModelSchemaUpdateChecker.CheckResult result = checker.allowEdit(dataModalDesc, project, false);
+        ModelSchemaUpdateChecker.CheckResult result = checker.allowEdit(dataModalDesc, null, false);
         result.raiseExceptionWhenInvalid();
     }
 
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
index dd63b52..e26cd74 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.UUID;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -91,10 +92,14 @@ public class HBaseResourceStore extends PushdownResourceStore {
                 .parseInt(getConnection().getConfiguration().get("hbase.client.keyvalue.maxsize", "10485760"));
     }
 
-    Connection getConnection() throws IOException {
+    protected Connection getConnection() throws IOException {
         return HBaseConnection.get(metadataUrl);
     }
 
+    protected Configuration getCurrentHBaseConfiguration() {
+        return HBaseConnection.getCurrentHBaseConfiguration();
+    }
+
     private StorageURL buildMetadataUrl(KylinConfig kylinConfig) throws IOException {
         StorageURL url = kylinConfig.getMetadataUrl();
         if (!url.getScheme().equals("hbase"))
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
index 845a4e5..52d1c14 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
@@ -420,7 +420,7 @@ public class DeployCoprocessorCLI {
     }
 
     public static Path getNewestCoprocessorJar(KylinConfig config, FileSystem fileSystem) throws IOException {
-        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, config);
+        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, getHDFSWorkingDirectory(config));
         FileStatus newestJar = null;
         for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
             if (fileStatus.getPath().toString().endsWith(".jar")) {
@@ -440,8 +440,14 @@ public class DeployCoprocessorCLI {
         return path;
     }
 
-    public static synchronized Path uploadCoprocessorJar(String localCoprocessorJar, FileSystem fileSystem,
-            Set<String> oldJarPaths) throws IOException {
+    public synchronized static Path uploadCoprocessorJar(String localCoprocessorJar, FileSystem fileSystem,
+                                                         Set<String> oldJarPaths) throws IOException {
+        String hdfsWorkingDirectory = getHDFSWorkingDirectory(KylinConfig.getInstanceFromEnv());
+        return uploadCoprocessorJar(localCoprocessorJar, fileSystem, hdfsWorkingDirectory, oldJarPaths);
+    }
+
+    public synchronized static Path uploadCoprocessorJar(String localCoprocessorJar, FileSystem fileSystem,
+                                                         String hdfsWorkingDirectory, Set<String> oldJarPaths) throws IOException {
         Path uploadPath = null;
         File localCoprocessorFile = new File(localCoprocessorJar);
 
@@ -449,7 +455,7 @@ public class DeployCoprocessorCLI {
         if (oldJarPaths == null) {
             oldJarPaths = new HashSet<String>();
         }
-        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, KylinConfig.getInstanceFromEnv());
+        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, hdfsWorkingDirectory);
         for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
             if (isSame(localCoprocessorFile, fileStatus)) {
                 uploadPath = fileStatus.getPath();
@@ -511,9 +517,12 @@ public class DeployCoprocessorCLI {
         return baseName;
     }
 
-    private static Path getCoprocessorHDFSDir(FileSystem fileSystem, KylinConfig config) throws IOException {
+    private static String getHDFSWorkingDirectory(KylinConfig config) {
         String hdfsWorkingDirectory = config.getHdfsWorkingDirectory();
-        hdfsWorkingDirectory = HBaseConnection.makeQualifiedPathInHBaseCluster(hdfsWorkingDirectory);
+        return HBaseConnection.makeQualifiedPathInHBaseCluster(hdfsWorkingDirectory);
+    }
+
+    private static Path getCoprocessorHDFSDir(FileSystem fileSystem, String hdfsWorkingDirectory) throws IOException {
         Path coprocessorDir = new Path(hdfsWorkingDirectory, "coprocessor");
         fileSystem.mkdirs(coprocessorDir);
         return coprocessorDir;
diff --git a/tool/pom.xml b/tool/pom.xml
index 140ff93..8d4c2b4 100644
--- a/tool/pom.xml
+++ b/tool/pom.xml
@@ -76,6 +76,11 @@
             <scope>provided</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-server</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-yarn-api</artifactId>
             <scope>provided</scope>
@@ -86,6 +91,12 @@
             <scope>provided</scope>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-distcp</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
         <!--Spring-->
         <dependency>
             <groupId>org.springframework</groupId>
diff --git a/tool/src/main/java/org/apache/kylin/tool/migration/ClusterUtil.java b/tool/src/main/java/org/apache/kylin/tool/migration/ClusterUtil.java
new file mode 100644
index 0000000..14f8e90
--- /dev/null
+++ b/tool/src/main/java/org/apache/kylin/tool/migration/ClusterUtil.java
@@ -0,0 +1,165 @@
+/*
+ * 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.tool.migration;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.RawResource;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.restclient.RestClient;
+import org.apache.kylin.dict.DictionaryInfo;
+import org.apache.kylin.dict.lookup.SnapshotTable;
+import org.apache.kylin.engine.mr.JobBuilderSupport;
+import org.apache.kylin.metadata.project.ProjectInstance;
+import org.apache.kylin.rest.security.ManagedUser;
+import org.apache.kylin.rest.service.KylinUserService;
+import org.apache.kylin.storage.hbase.HBaseResourceStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Sets;
+
+public abstract class ClusterUtil {
+    private static final Logger logger = LoggerFactory.getLogger(ClusterUtil.class);
+
+    protected final KylinConfig kylinConfig;
+    protected final RestClient restClient;
+    protected final String hdfsWorkingDirectory;
+
+    protected final Configuration hbaseConf;
+    protected final Connection hbaseConn;
+    protected final ResourceStore resourceStore;
+    protected final Admin hbaseAdmin;
+
+    final Configuration jobConf;
+    final FileSystem jobFS;
+    final String jobHdfsWorkingDirectoryQualified;
+    final FileSystem hbaseFS;
+    final String hbaseHdfsWorkingDirectoryQualified;
+
+    public ClusterUtil(String configURI, boolean ifJobFSHAEnabled, boolean ifHBaseFSHAEnabled) throws IOException {
+        this.kylinConfig = KylinConfig.createInstanceFromUri(configURI);
+        this.restClient = new RestClient(configURI);
+        Path hdfsWorkingPath = Path.getPathWithoutSchemeAndAuthority(new Path(kylinConfig.getHdfsWorkingDirectory()));
+        String tmpHdfsWorkingDirectory = hdfsWorkingPath.toString();
+        this.hdfsWorkingDirectory = tmpHdfsWorkingDirectory.endsWith("/") ? tmpHdfsWorkingDirectory
+                : tmpHdfsWorkingDirectory + "/";
+
+        this.jobConf = KylinConfig.getConfigFromString(restClient.getHDFSConfiguration());
+        this.jobFS = FileSystem.get(jobConf);
+        this.jobHdfsWorkingDirectoryQualified = getQualifiedPath(jobConf, hdfsWorkingDirectory, ifJobFSHAEnabled);
+
+        this.hbaseConf = KylinConfig.getConfigFromString(restClient.getHBaseConfiguration());
+        this.hbaseFS = FileSystem.get(hbaseConf);
+        this.hbaseHdfsWorkingDirectoryQualified = getQualifiedPath(hbaseConf, hdfsWorkingDirectory, ifHBaseFSHAEnabled);
+
+        this.hbaseConn = ConnectionFactory.createConnection(hbaseConf);
+        this.resourceStore = new HBaseResourceStore(kylinConfig) {
+            @Override
+            protected Connection getConnection() {
+                return hbaseConn;
+            }
+
+            @Override
+            protected Configuration getCurrentHBaseConfiguration() {
+                return hbaseConf;
+            }
+        };
+        this.hbaseAdmin = hbaseConn.getAdmin();
+    }
+
+    public abstract ProjectInstance getProject(String projName) throws IOException;
+
+    public abstract DictionaryInfo getDictionaryInfo(String dictPath) throws IOException;
+
+    public abstract SnapshotTable getSnapshotTable(String snapshotPath) throws IOException;
+
+    public abstract String getRootDirQualifiedOfHTable(String tableName);
+
+    public ManagedUser getUserDetails(String userKey) throws IOException {
+        return resourceStore.getResource(userKey, KylinUserService.SERIALIZER);
+    }
+
+    public final RawResource getResource(String resPath) throws IOException {
+        return resourceStore.getResource(resPath);
+    }
+
+    public String getJobWorkingDirQualified(String jobId) {
+        return JobBuilderSupport.getJobWorkingDir(jobHdfsWorkingDirectoryQualified, jobId);
+    }
+
+    private static String getQualifiedPath(Configuration conf, String path, boolean ifHAEnabled) throws IOException {
+        String hdfsSchema = getReplacedDefaultFS(conf, !ifHAEnabled);
+        return hdfsSchema + path;
+    }
+
+    private static String getReplacedDefaultFS(Configuration conf, boolean ifNeedReplace) throws IOException {
+        String defaultFS = conf.get(FileSystem.FS_DEFAULT_NAME_KEY);
+        if (!ifNeedReplace) {
+            return defaultFS;
+        }
+
+        String nameServices = conf.get("dfs.nameservices");
+        if (Strings.isNullOrEmpty(nameServices)) {
+            return defaultFS;
+        }
+
+        // check whether name service is defined for the default fs
+        Set<String> nameServiceSet = Sets.newHashSet(nameServices.split(","));
+        String defaultNameService = URI.create(defaultFS).getHost();
+        if (!nameServiceSet.contains(defaultNameService)) {
+            logger.info("name service {} is not defined among {}", defaultNameService, nameServices);
+            return defaultFS;
+        }
+
+        // select one usable node as the default fs
+        String haHostNames = conf.get("dfs.ha.namenodes." + defaultNameService);
+        if (!Strings.isNullOrEmpty(haHostNames)) {
+            conf = new Configuration(conf);
+            for (String oneNodeAlias : haHostNames.split(",")) {
+                String rpcNode = conf.get("dfs.namenode.rpc-address." + defaultNameService + "." + oneNodeAlias);
+                String replaced = "hdfs://" + rpcNode;
+                conf.set(FileSystem.FS_DEFAULT_NAME_KEY, replaced);
+
+                Path rootPath = new Path(replaced + "/");
+                FileSystem fs = FileSystem.get(conf);
+                try {
+                    fs.getStatus(rootPath);
+                } catch (Exception e) {
+                    logger.warn("cannot use {} as default fs due to ", replaced, e);
+                    continue;
+                }
+                logger.info("replaced the default fs {} by {}", defaultFS, replaced);
+                return replaced;
+            }
+            throw new IllegalArgumentException("fail to replace the default fs " + defaultFS);
+        }
+        throw new IllegalArgumentException("dfs.ha.namenodes." + defaultNameService + " is not set");
+    }
+}
diff --git a/tool/src/main/java/org/apache/kylin/tool/migration/CubeMigrationCrossClusterCLI.java b/tool/src/main/java/org/apache/kylin/tool/migration/CubeMigrationCrossClusterCLI.java
new file mode 100644
index 0000000..95efab0
--- /dev/null
+++ b/tool/src/main/java/org/apache/kylin/tool/migration/CubeMigrationCrossClusterCLI.java
@@ -0,0 +1,757 @@
+/*
+ * 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.tool.migration;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.OptionGroup;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.tools.DistCp;
+import org.apache.hadoop.tools.DistCpConstants;
+import org.apache.hadoop.tools.DistCpOptions;
+import org.apache.hadoop.tools.OptionsParser;
+import org.apache.kylin.common.persistence.RawResource;
+import org.apache.kylin.common.util.AbstractApplication;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.common.util.OptionsHelper;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.DictionaryDesc;
+import org.apache.kylin.cube.model.SnapshotTableDesc;
+import org.apache.kylin.dict.DictionaryInfo;
+import org.apache.kylin.dict.GlobalDictionaryBuilder;
+import org.apache.kylin.dict.lookup.ExtTableSnapshotInfo;
+import org.apache.kylin.dict.lookup.SnapshotTable;
+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.RealizationEntry;
+import org.apache.kylin.metadata.realization.IRealization;
+import org.apache.kylin.metadata.realization.RealizationStatusEnum;
+import org.apache.kylin.metadata.realization.RealizationType;
+import org.apache.kylin.rest.response.HBaseResponse;
+import org.apache.kylin.rest.security.ManagedUser;
+import org.apache.kylin.rest.service.HBaseInfoUtil;
+import org.apache.kylin.rest.service.KylinUserService;
+import org.apache.kylin.rest.service.update.TableSchemaUpdateMapping;
+import org.apache.kylin.rest.service.update.TableSchemaUpdater;
+import org.apache.kylin.storage.hybrid.HybridInstance;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public class CubeMigrationCrossClusterCLI extends AbstractApplication {
+
+    private static final Logger logger = LoggerFactory.getLogger(CubeMigrationCrossClusterCLI.class);
+
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_KYLIN_URI_SRC = OptionBuilder.withArgName("kylinUriSrc").hasArg().isRequired(true)
+            .withDescription("Specify the source kylin uri with format user:pwd@host:port").create("kylinUriSrc");
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_KYLIN_URI_DST = OptionBuilder.withArgName("kylinUriDst").hasArg().isRequired(true)
+            .withDescription("Specify the destination kylin uri with format user:pwd@host:port").create("kylinUriDst");
+
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_UPDATE_MAPPING = OptionBuilder.withArgName("updateMappingPath").hasArg()
+            .isRequired(false).withDescription("Specify the path for the update mapping file")
+            .create("updateMappingPath");
+
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_CUBE = OptionBuilder.withArgName("cube").hasArg().isRequired(false)
+            .withDescription("Specify which cube to extract").create("cube");
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_HYBRID = OptionBuilder.withArgName("hybrid").hasArg().isRequired(false)
+            .withDescription("Specify which hybrid to extract").create("hybrid");
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_PROJECT = OptionBuilder.withArgName("project").hasArg().isRequired(false)
+            .withDescription("Specify realizations in which project to extract").create("project");
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_All = OptionBuilder.withArgName("all").hasArg(false).isRequired(false)
+            .withDescription("Specify realizations in all projects to extract").create("all");
+
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_DST_HIVE_CHECK = OptionBuilder.withArgName("dstHiveCheck").hasArg()
+            .isRequired(false).withDescription("Specify whether to check destination hive tables")
+            .create("dstHiveCheck");
+
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_OVERWRITE = OptionBuilder.withArgName("overwrite").hasArg().isRequired(false)
+            .withDescription("Specify whether to overwrite existing cubes").create("overwrite");
+
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_SCHEMA_ONLY = OptionBuilder.withArgName("schemaOnly").hasArg().isRequired(false)
+            .withDescription("Specify whether only migrate cube related schema").create("schemaOnly");
+
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_EXECUTE = OptionBuilder.withArgName("execute").hasArg().isRequired(false)
+            .withDescription("Specify whether it's to execute the migration").create("execute");
+
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_COPROCESSOR_PATH = OptionBuilder.withArgName("coprocessorPath").hasArg()
+            .isRequired(false).withDescription("Specify the path of coprocessor to be deployed")
+            .create("coprocessorPath");
+
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_FS_HA_ENABLED_CODE = OptionBuilder.withArgName("codeOfFSHAEnabled").hasArg()
+            .isRequired(false).withDescription("Specify whether to enable the namenode ha of clusters")
+            .create("codeOfFSHAEnabled");
+
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_DISTCP_JOB_QUEUE = OptionBuilder.withArgName("distCpJobQueue").hasArg()
+            .isRequired(false).withDescription("Specify the mapreduce.job.queuename for DistCp job ")
+            .create("distCpJobQueue");
+
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_DISTCP_JOB_MEMORY = OptionBuilder.withArgName("distCpJobMemory").hasArg()
+            .isRequired(false).withDescription("Specify the mapreduce.map.memory.mb for DistCp job ")
+            .create("distCpJobMemory");
+
+    @SuppressWarnings("static-access")
+    public static final Option OPTION_THREAD_NUM = OptionBuilder.withArgName("nThread").hasArg().isRequired(false)
+            .withDescription("Specify the number of threads for migrating cube data in parallel ").create("nThread");
+
+    protected final Options options;
+
+    private Configuration distCpConf;
+
+    protected SrcClusterUtil srcCluster;
+    protected DstClusterUtil dstCluster;
+
+    private int codeOfFSHAEnabled = 3;
+    protected int nThread;
+
+    private boolean ifDstHiveCheck = true;
+    private boolean ifSchemaOnly = true;
+    private boolean ifExecute = false;
+    private boolean ifOverwrite = false;
+
+    private String coprocessorJarPath;
+
+    private Set<CubeInstance> cubes = Sets.newHashSet();
+    private Set<HybridInstance> hybrids = Sets.newHashSet();
+    private Set<ProjectInstance> projects = Sets.newHashSet();
+
+    private Map<String, TableSchemaUpdateMapping> mappings = Maps.newHashMap();
+
+    private Map<String, ProjectInstance> dstProjects = Maps.newHashMap();
+
+    public CubeMigrationCrossClusterCLI() {
+        OptionGroup realizationOrProject = new OptionGroup();
+        realizationOrProject.addOption(OPTION_CUBE);
+        realizationOrProject.addOption(OPTION_HYBRID);
+        realizationOrProject.addOption(OPTION_PROJECT);
+        realizationOrProject.addOption(OPTION_All);
+        realizationOrProject.setRequired(true);
+
+        options = new Options();
+        options.addOption(OPTION_KYLIN_URI_SRC);
+        options.addOption(OPTION_KYLIN_URI_DST);
+        options.addOption(OPTION_FS_HA_ENABLED_CODE);
+        options.addOption(OPTION_UPDATE_MAPPING);
+        options.addOptionGroup(realizationOrProject);
+        options.addOption(OPTION_DST_HIVE_CHECK);
+        options.addOption(OPTION_SCHEMA_ONLY);
+        options.addOption(OPTION_OVERWRITE);
+        options.addOption(OPTION_EXECUTE);
+        options.addOption(OPTION_COPROCESSOR_PATH);
+        options.addOption(OPTION_DISTCP_JOB_QUEUE);
+        options.addOption(OPTION_THREAD_NUM);
+        options.addOption(OPTION_DISTCP_JOB_MEMORY);
+    }
+
+    protected Options getOptions() {
+        return options;
+    }
+
+    public static boolean ifFSHAEnabled(int code, int pos) {
+        int which = 1 << pos;
+        return (code & which) == which;
+    }
+
+    protected void init(OptionsHelper optionsHelper) throws Exception {
+        if (optionsHelper.hasOption(OPTION_UPDATE_MAPPING)) {
+            File mappingFile = new File(optionsHelper.getOptionValue(OPTION_UPDATE_MAPPING));
+            String content = new String(Files.readAllBytes(mappingFile.toPath()), Charset.defaultCharset());
+            Map<String, TableSchemaUpdateMapping> tmpMappings = JsonUtil.readValue(content,
+                    new TypeReference<Map<String, TableSchemaUpdateMapping>>() {
+                    });
+            mappings = Maps.newHashMapWithExpectedSize(tmpMappings.size());
+            for (Map.Entry<String, TableSchemaUpdateMapping> entry : tmpMappings.entrySet()) {
+                mappings.put(entry.getKey().toUpperCase(Locale.ROOT), entry.getValue());
+            }
+        }
+
+        ifDstHiveCheck = optionsHelper.hasOption(OPTION_DST_HIVE_CHECK)
+                ? Boolean.valueOf(optionsHelper.getOptionValue(OPTION_DST_HIVE_CHECK))
+                : true;
+        ifSchemaOnly = optionsHelper.hasOption(OPTION_SCHEMA_ONLY)
+                ? Boolean.valueOf(optionsHelper.getOptionValue(OPTION_SCHEMA_ONLY))
+                : true;
+        ifOverwrite = optionsHelper.hasOption(OPTION_OVERWRITE)
+                ? Boolean.valueOf(optionsHelper.getOptionValue(OPTION_OVERWRITE))
+                : false;
+        ifExecute = optionsHelper.hasOption(OPTION_EXECUTE)
+                ? Boolean.valueOf(optionsHelper.getOptionValue(OPTION_EXECUTE))
+                : false;
+
+        codeOfFSHAEnabled = optionsHelper.hasOption(OPTION_FS_HA_ENABLED_CODE)
+                ? Integer.valueOf(optionsHelper.getOptionValue(OPTION_FS_HA_ENABLED_CODE))
+                : 3;
+
+        String srcConfigURI = optionsHelper.getOptionValue(OPTION_KYLIN_URI_SRC);
+        srcCluster = new SrcClusterUtil(srcConfigURI, ifFSHAEnabled(codeOfFSHAEnabled, 0),
+                ifFSHAEnabled(codeOfFSHAEnabled, 1));
+        String dstConfigURI = optionsHelper.getOptionValue(OPTION_KYLIN_URI_DST);
+        dstCluster = new DstClusterUtil(dstConfigURI, ifFSHAEnabled(codeOfFSHAEnabled, 2),
+                ifFSHAEnabled(codeOfFSHAEnabled, 3), ifExecute);
+
+        distCpConf = new Configuration(srcCluster.jobConf);
+        if (optionsHelper.hasOption(OPTION_DISTCP_JOB_QUEUE)) {
+            distCpConf.set("mapreduce.job.queuename", optionsHelper.getOptionValue(OPTION_DISTCP_JOB_QUEUE));
+        }
+        int distCpMemory = optionsHelper.hasOption(OPTION_DISTCP_JOB_MEMORY)
+                ? Integer.valueOf(optionsHelper.getOptionValue(OPTION_DISTCP_JOB_MEMORY))
+                : 1500;
+        int distCpJVMMemory = distCpMemory * 4 / 5;
+        distCpConf.set("mapreduce.map.memory.mb", "" + distCpMemory);
+        distCpConf.set("mapreduce.map.java.opts",
+                "-server -Xmx" + distCpJVMMemory + "m -Djava.net.preferIPv4Stack=true");
+
+        nThread = optionsHelper.hasOption(OPTION_THREAD_NUM)
+                ? Integer.valueOf(optionsHelper.getOptionValue(OPTION_THREAD_NUM))
+                : 8;
+
+        coprocessorJarPath = optionsHelper.hasOption(OPTION_COPROCESSOR_PATH)
+                ? optionsHelper.getOptionValue(OPTION_COPROCESSOR_PATH)
+                : srcCluster.getDefaultCoprocessorJarPath();
+    }
+
+    protected void execute(OptionsHelper optionsHelper) throws Exception {
+        init(optionsHelper);
+
+        if (optionsHelper.hasOption(OPTION_All)) {
+            projects.addAll(srcCluster.listAllProjects());
+        } else if (optionsHelper.hasOption(OPTION_PROJECT)) {
+            Set<String> projectNames = Sets.newHashSet(optionsHelper.getOptionValue(OPTION_PROJECT).split(","));
+            for (String projectName : projectNames) {
+                ProjectInstance project = srcCluster.getProject(projectName);
+                if (project == null) {
+                    throw new IllegalArgumentException("No project found with name of " + projectName);
+                }
+                projects.add(project);
+            }
+        } else if (optionsHelper.hasOption(OPTION_CUBE)) {
+            String cubeNames = optionsHelper.getOptionValue(OPTION_CUBE);
+            for (String cubeName : cubeNames.split(",")) {
+                CubeInstance cube = srcCluster.getCube(cubeName);
+                if (cube == null) {
+                    throw new IllegalArgumentException("No cube found with name of " + cubeName);
+                } else {
+                    cubes.add(cube);
+                }
+            }
+        } else if (optionsHelper.hasOption(OPTION_HYBRID)) {
+            String hybridNames = optionsHelper.getOptionValue(OPTION_HYBRID);
+            for (String hybridName : hybridNames.split(",")) {
+                HybridInstance hybridInstance = srcCluster.getHybrid(hybridName);
+                if (hybridInstance != null) {
+                    hybrids.add(hybridInstance);
+                } else {
+                    throw new IllegalArgumentException("No hybrid found with name of" + hybridName);
+                }
+            }
+        }
+
+        if (!projects.isEmpty()) {
+            for (ProjectInstance project : projects) {
+                for (RealizationEntry entry : project.getRealizationEntries()) {
+                    IRealization realization = srcCluster.getRealization(entry);
+                    addRealization(realization);
+                }
+            }
+        }
+        if (!hybrids.isEmpty()) {
+            for (HybridInstance hybrid : hybrids) {
+                addHybrid(hybrid);
+            }
+        }
+
+        Map<CubeInstance, Exception> failedCubes = Maps.newHashMap();
+
+        for (CubeInstance cube : cubes) {
+            logger.info("start to migrate cube {}", cube);
+            try {
+                migrateCube(cube);
+                logger.info("finish migrating cube {}", cube);
+            } catch (Exception e) {
+                logger.error("fail to migrate cube {} due to ", cube, e);
+                failedCubes.put(cube, e);
+            }
+        }
+
+        for (HybridInstance hybrid : hybrids) {
+            dstCluster.saveHybrid(hybrid);
+
+            // update project
+            ProjectInstance srcProject = srcCluster.getProjectByRealization(RealizationType.HYBRID, hybrid.getName());
+            ProjectInstance dstProject = getDstProject(srcProject);
+
+            // update hybrids
+            Set<RealizationEntry> projReals = Sets.newHashSet(dstProject.getRealizationEntries());
+            projReals.add(RealizationEntry.create(RealizationType.HYBRID, hybrid.getName()));
+            dstProject.setRealizationEntries(Lists.newArrayList(projReals));
+
+            dstProjects.put(dstProject.getName(), dstProject);
+        }
+
+        for (String projName : dstProjects.keySet()) {
+            dstCluster.saveProject(dstProjects.get(projName));
+        }
+
+        dstCluster.updateMeta();
+
+        if (failedCubes.isEmpty()) {
+            logger.info("Migration for cubes {}, hyrbids {} all succeed", cubes, hybrids);
+        } else {
+            logger.warn("Failed to migrate cubes {} and need to check the detailed reason and retry again!!!",
+                    failedCubes.keySet());
+        }
+    }
+
+    private void migrateCube(CubeInstance cube) throws IOException {
+        if (!ifOverwrite && dstCluster.exists(CubeInstance.concatResourcePath(cube.getName()))) {
+            throw new RuntimeException(("The cube named " + cube.getName()
+                    + " already exists on target metadata store. Please delete it firstly and try again"));
+        }
+
+        ProjectInstance srcProject = srcCluster.getProjectByRealization(RealizationType.CUBE, cube.getName());
+
+        String descName = cube.getDescName();
+        CubeDesc cubeDesc = srcCluster.getCubeDesc(descName);
+
+        String modelName = cubeDesc.getModelName();
+        DataModelDesc modelDesc = srcCluster.getDataModelDesc(modelName);
+
+        Set<TableDesc> tableSet = Sets.newHashSet();
+        for (TableRef tableRef : modelDesc.getAllTables()) {
+            TableDesc tableDescOld = srcCluster.getTableDesc(tableRef.getTableIdentity(), srcProject.getName());
+            TableDesc tableDescUpdated = TableSchemaUpdater.dealWithMappingForTable(tableDescOld, mappings);
+            tableSet.add(tableDescUpdated);
+        }
+
+        modelDesc = TableSchemaUpdater.dealWithMappingForModel(modelDesc, mappings);
+
+        cubeDesc = TableSchemaUpdater.dealWithMappingForCubeDesc(cubeDesc, mappings);
+
+        { // compatibility check before migrating to the destination cluster
+            dstCluster.checkCompatibility(srcProject.getName(), tableSet, modelDesc, ifDstHiveCheck);
+        }
+
+        {
+            for (TableDesc table : tableSet) {
+                dstCluster.saveTableDesc(table);
+            }
+
+            dstCluster.saveModelDesc(modelDesc);
+
+            dstCluster.saveCubeDesc(cubeDesc);
+
+            if (ifSchemaOnly) {
+                cube = CubeInstance.getCopyOf(cube);
+                cube.getSegments().clear();
+                cube.resetSnapshots();
+                cube.setStatus(RealizationStatusEnum.DISABLED);
+                cube.clearCuboids();
+            } else {
+                // cube with global dictionary cannot be migrated with data
+                checkGlobalDict(cubeDesc);
+
+                // delete those NEW segments and only keep the READY segments
+                cube.setSegments(cube.getSegments(SegmentStatusEnum.READY));
+
+                cube = TableSchemaUpdater.dealWithMappingForCube(cube, mappings);
+
+                ExecutorService executor = Executors.newFixedThreadPool(nThread, new ThreadFactoryBuilder()
+                        .setNameFormat("Cube-" + cube.getName() + "-data-migration-pool-%d").build());
+                try {
+                    List<Future<?>> futureList = migrateCubeData(cube, cubeDesc, executor);
+                    executor.shutdown();
+                    for (Future<?> future : futureList) {
+                        try {
+                            future.get();
+                        } catch (InterruptedException e) {
+                            logger.warn(e.getMessage());
+                        } catch (ExecutionException e) {
+                            executor.shutdownNow();
+                            logger.error(e.getMessage());
+                            throw new RuntimeException(e);
+                        }
+                    }
+                } finally {
+                    // in case that exceptions are thrown when call migrateCubeData()
+                    if (!executor.isShutdown()) {
+                        logger.warn("shut down executor for cube {}", cube);
+                        executor.shutdownNow();
+                    }
+                }
+            }
+
+            dstCluster.saveCubeInstance(cube);
+        }
+
+        {
+            ProjectInstance dstProject = getDstProject(srcProject);
+
+            // update tables in project
+            Set<String> projTables = Sets.newHashSet(dstProject.getTables());
+            projTables.addAll(tableSet.stream().map(TableDesc::getIdentity).collect(Collectors.toSet()));
+            dstProject.setTables(projTables);
+
+            // update models in project
+            Set<String> projModels = Sets.newHashSet(dstProject.getModels());
+            projModels.add(modelName);
+            dstProject.setModels(Lists.newArrayList(projModels));
+
+            // update cubes in project
+            Set<RealizationEntry> projReals = Sets.newHashSet(dstProject.getRealizationEntries());
+            projReals.add(RealizationEntry.create(RealizationType.CUBE, cube.getName()));
+            dstProject.setRealizationEntries(Lists.newArrayList(projReals));
+
+            dstProjects.put(dstProject.getName(), dstProject);
+        }
+    }
+
+    private void checkGlobalDict(CubeDesc cubeDesc) {
+        if (cubeDesc.getDictionaries() != null && !cubeDesc.getDictionaries().isEmpty()) {
+            for (DictionaryDesc dictDesc : cubeDesc.getDictionaries()) {
+                if (GlobalDictionaryBuilder.class.getName().equalsIgnoreCase(dictDesc.getBuilderClass())) {
+                    throw new RuntimeException("it's not supported to migrate global dictionaries " + dictDesc
+                            + " for cube " + cubeDesc.getName());
+                }
+            }
+        }
+    }
+
+    private List<Future<?>> migrateCubeData(CubeInstance cube, CubeDesc cubeDesc, ExecutorService executor)
+            throws IOException {
+        List<Future<?>> futureList = Lists.newLinkedList();
+
+        for (final CubeSegment segment : cube.getSegments(SegmentStatusEnum.READY)) {
+            logger.info("start to migrate segment: {} {}", cube, segment.getName());
+            copyMetaResource(segment.getStatisticsResourcePath());
+            for (String dict : segment.getDictionaryPaths()) {
+                copyDictionary(cube, dict);
+            }
+            for (String snapshot : segment.getSnapshotPaths()) {
+                copySnapshot(cube, snapshot);
+            }
+            Future<?> future;
+            future = executor.submit(new MyRunnable() {
+                @Override
+                public void doRun() throws Exception {
+                    copyHDFSJobInfo(segment.getLastBuildJobID());
+                }
+            });
+            futureList.add(future);
+
+            future = executor.submit(new MyRunnable() {
+                @Override
+                public void doRun() throws Exception {
+                    copyHTable(segment);
+                }
+            });
+            futureList.add(future);
+
+            logger.info("add segment {} to migration list", segment);
+        }
+        if (cubeDesc.getSnapshotTableDescList() != null) {
+            for (SnapshotTableDesc snapshotTable : cubeDesc.getSnapshotTableDescList()) {
+                if (snapshotTable.isGlobal()) {
+                    String snapshotResPath = cube.getSnapshotResPath(snapshotTable.getTableName());
+                    if (snapshotTable.isExtSnapshotTable()) {
+                        final ExtTableSnapshotInfo extSnapshot = srcCluster.getExtTableSnapshotInfo(snapshotResPath);
+                        dstCluster.saveExtSnapshotTableInfo(extSnapshot);
+                        if (ExtTableSnapshotInfo.STORAGE_TYPE_HBASE.equals(extSnapshot.getStorageType())) {
+                            Future<?> future = executor.submit(new MyRunnable() {
+                                @Override
+                                public void doRun() throws Exception {
+                                    copyHTable(extSnapshot);
+                                }
+                            });
+                            futureList.add(future);
+                        }
+                    } else {
+                        copySnapshot(cube, snapshotResPath);
+                    }
+                    logger.info("add cube-level snapshot table {} for cube {} to migration list", snapshotResPath,
+                            cube);
+                }
+            }
+        }
+
+        return futureList;
+    }
+
+    private ProjectInstance getDstProject(ProjectInstance srcProject) throws IOException {
+        ProjectInstance dstProject = dstProjects.get(srcProject.getName());
+        if (dstProject == null) {
+            dstProject = dstCluster.getProject(srcProject.getName());
+        }
+        if (dstProject == null) {
+            dstProject = ProjectInstance.create(srcProject.getName(), srcProject.getOwner(),
+                    srcProject.getDescription(), srcProject.getOverrideKylinProps(), null, null);
+            dstProject.setUuid(srcProject.getUuid());
+        }
+        return dstProject;
+    }
+
+    private void putUserInfo(String userName) throws IOException {
+        String userKey = KylinUserService.getId(userName);
+        ManagedUser user = srcCluster.getUserDetails(userKey);
+        if (user == null) {
+            logger.warn("Cannot find user {}", userName);
+            return;
+        }
+        dstCluster.saveUserInfo(userKey, user);
+    }
+
+    private void copyMetaResource(String item) throws IOException {
+        RawResource res = srcCluster.getResource(item);
+        dstCluster.putResource(item, res);
+        res.content().close();
+    }
+
+    private void copyDictionary(CubeInstance cube, String dictPath) throws IOException {
+        if (dstCluster.exists(dictPath)) {
+            logger.info("Item {} has already existed in destination cluster", dictPath);
+            return;
+        }
+        DictionaryInfo dictInfo = srcCluster.getDictionaryInfo(dictPath);
+        String dupDict = dstCluster.saveDictionary(dictInfo);
+        if (dupDict != null) {
+            for (CubeSegment segment : cube.getSegments()) {
+                for (Map.Entry<String, String> entry : segment.getDictionaries().entrySet()) {
+                    if (entry.getValue().equalsIgnoreCase(dictPath)) {
+                        entry.setValue(dupDict);
+                    }
+                }
+            }
+            logger.info("Item {} is dup, instead {} is reused", dictPath, dupDict);
+        }
+    }
+
+    private void copySnapshot(CubeInstance cube, String snapshotTablePath) throws IOException {
+        if (dstCluster.exists(snapshotTablePath)) {
+            logger.info("Item {} has already existed in destination cluster", snapshotTablePath);
+            return;
+        }
+        SnapshotTable snapshotTable = srcCluster.getSnapshotTable(snapshotTablePath);
+        dstCluster.saveSnapshotTable(snapshotTable);
+    }
+
+    private void copyHDFSJobInfo(String jobId) throws Exception {
+        String srcDirQualified = srcCluster.getJobWorkingDirQualified(jobId);
+        String dstDirQualified = dstCluster.getJobWorkingDirQualified(jobId);
+        if (ifExecute) {
+            dstCluster.copyInitOnJobCluster(new Path(dstDirQualified));
+            copyHDFSPath(srcDirQualified, srcCluster.jobConf, dstDirQualified, dstCluster.jobConf);
+        } else {
+            logger.info("copied hdfs directory from {} to {}", srcDirQualified, dstDirQualified);
+        }
+    }
+
+    private void copyHTable(CubeSegment segment) throws IOException {
+        String tableName = segment.getStorageLocationIdentifier();
+        if (ifExecute) {
+            if (checkHTableExist(segment)) {
+                logger.info("htable {} has already existed in dst, will skip the migration", tableName);
+            } else {
+                copyHTable(tableName, true);
+                if (!checkHTableEquals(tableName)) {
+                    logger.error("htable {} is copied to dst with different size!!!", tableName);
+                }
+            }
+        }
+        logger.info("migrated htable {} for segment {}", tableName, segment);
+    }
+
+    private boolean checkHTableExist(CubeSegment segment) throws IOException {
+        String tableName = segment.getStorageLocationIdentifier();
+        TableName htableName = TableName.valueOf(tableName);
+        if (!dstCluster.checkExist(htableName, segment)) {
+            return false;
+        }
+
+        if (!checkHTableEquals(tableName)) {
+            logger.warn("although htable {} exists in destination, the details data are different", tableName);
+            dstCluster.deleteHTable(tableName);
+            return false;
+        }
+        return true;
+    }
+
+    private boolean checkHTableEquals(String tableName) throws IOException {
+        HBaseResponse respSrc = HBaseInfoUtil.getHBaseInfo(tableName, srcCluster.hbaseConn);
+        HBaseResponse respDst = HBaseInfoUtil.getHBaseInfo(tableName, dstCluster.hbaseConn);
+        return HBaseInfoUtil.checkEquals(respSrc, respDst);
+    }
+
+    private void copyHTable(ExtTableSnapshotInfo extTableSnapshotInfo) throws IOException {
+        String tableName = extTableSnapshotInfo.getStorageLocationIdentifier();
+        if (ifExecute) {
+            TableName htableName = TableName.valueOf(tableName);
+            if (dstCluster.htableExists(htableName)) {
+                logger.warn("htable {} already exists in the dst cluster and will skip the htable migration");
+            } else {
+                copyHTable(tableName, false);
+            }
+        }
+        logger.info("migrated htable {} for ext table snapshot {}", tableName, extTableSnapshotInfo.getTableName());
+    }
+
+    private void copyHTable(String tableName, boolean ifDeployCoprocessor) {
+        if (ifExecute) {
+            TableName htableName = TableName.valueOf(tableName);
+            try {
+                //migrate data first
+                copyHFileByDistCp(tableName);
+
+                //create htable metadata, especially the split keys for predefining the regions
+                Table table = srcCluster.hbaseConn.getTable(TableName.valueOf(tableName));
+                byte[][] endKeys = srcCluster.hbaseConn.getRegionLocator(htableName).getEndKeys();
+                byte[][] splitKeys = Arrays.copyOfRange(endKeys, 0, endKeys.length - 1);
+
+                HTableDescriptor tableDesc = new HTableDescriptor(table.getTableDescriptor());
+                //change the table host
+                dstCluster.resetTableHost(tableDesc);
+                if (ifDeployCoprocessor) {
+                    dstCluster.deployCoprocessor(tableDesc, coprocessorJarPath);
+                }
+                dstCluster.createTable(tableDesc, splitKeys);
+
+                //do bulk load to sync up htable data and metadata
+                dstCluster.bulkLoadTable(tableName);
+            } catch (Exception e) {
+                logger.error("fail to migrate htable {} due to {} ", tableName, e);
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    protected void copyHFileByDistCp(String tableName) throws Exception {
+        String srcDirQualified = srcCluster.getRootDirQualifiedOfHTable(tableName);
+        String dstDirQualified = dstCluster.getRootDirQualifiedOfHTable(tableName);
+        dstCluster.copyInitOnHBaseCluster(new Path(dstDirQualified));
+        copyHDFSPath(srcDirQualified, srcCluster.hbaseConf, dstDirQualified, dstCluster.hbaseConf);
+    }
+
+    protected void copyHDFSPath(String srcDir, Configuration srcConf, String dstDir, Configuration dstConf)
+            throws Exception {
+        logger.info("start to copy hdfs directory from {} to {}", srcDir, dstDir);
+        DistCpOptions distCpOptions = OptionsParser.parse(new String[] { srcDir, dstDir });
+        distCpOptions.preserve(DistCpOptions.FileAttribute.BLOCKSIZE);
+        distCpOptions.setBlocking(true);
+        setTargetPathExists(distCpOptions);
+        DistCp distCp = new DistCp(getConfOfDistCp(), distCpOptions);
+        distCp.execute();
+        logger.info("copied hdfs directory from {} to {}", srcDir, dstDir);
+    }
+
+    protected Configuration getConfOfDistCp() {
+        return distCpConf;
+    }
+
+    /**
+     * Set targetPathExists in both inputOptions and job config,
+     * for the benefit of CopyCommitter
+     */
+    public void setTargetPathExists(DistCpOptions inputOptions) throws IOException {
+        Path target = inputOptions.getTargetPath();
+        FileSystem targetFS = target.getFileSystem(dstCluster.jobConf);
+        boolean targetExists = targetFS.exists(target);
+        inputOptions.setTargetPathExists(targetExists);
+        dstCluster.jobConf.setBoolean(DistCpConstants.CONF_LABEL_TARGET_PATH_EXISTS, targetExists);
+    }
+
+    private void addHybrid(HybridInstance hybrid) {
+        hybrids.add(hybrid);
+        for (IRealization realization : hybrid.getRealizations()) {
+            addRealization(realization);
+        }
+    }
+
+    private void addRealization(IRealization realization) {
+        if (realization instanceof HybridInstance) {
+            addHybrid((HybridInstance) realization);
+        } else if (realization instanceof CubeInstance) {
+            cubes.add((CubeInstance) realization);
+        } else {
+            logger.warn("Realization {} is neither hybrid nor cube", realization);
+        }
+    }
+
+    private static abstract class MyRunnable implements Runnable {
+        @Override
+        public void run() {
+            try {
+                doRun();
+            } catch (RuntimeException e) {
+                throw e;
+            } catch (Throwable e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public abstract void doRun() throws Exception;
+    }
+
+    public static void main(String[] args) {
+        CubeMigrationCrossClusterCLI cli = new CubeMigrationCrossClusterCLI();
+        cli.execute(args);
+    }
+}
\ No newline at end of file
diff --git a/tool/src/main/java/org/apache/kylin/tool/migration/DstClusterUtil.java b/tool/src/main/java/org/apache/kylin/tool/migration/DstClusterUtil.java
new file mode 100644
index 0000000..e578935
--- /dev/null
+++ b/tool/src/main/java/org/apache/kylin/tool/migration/DstClusterUtil.java
@@ -0,0 +1,371 @@
+/*
+ * 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.tool.migration;
+
+import static org.apache.kylin.metadata.realization.IRealizationConstants.HTableSegmentTag;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.Set;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.persistence.RawResource;
+import org.apache.kylin.common.persistence.RootPersistentEntity;
+import org.apache.kylin.common.persistence.Serializer;
+import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.cube.CubeDescManager;
+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.dict.DictionaryInfo;
+import org.apache.kylin.dict.DictionaryInfoSerializer;
+import org.apache.kylin.dict.lookup.ExtTableSnapshotInfo;
+import org.apache.kylin.dict.lookup.ExtTableSnapshotInfoManager;
+import org.apache.kylin.dict.lookup.SnapshotTable;
+import org.apache.kylin.dict.lookup.SnapshotTableSerializer;
+import org.apache.kylin.metadata.TableMetadataManager;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.DataModelManager;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.project.ProjectInstance;
+import org.apache.kylin.metadata.project.ProjectManager;
+import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.apache.kylin.rest.security.ManagedUser;
+import org.apache.kylin.rest.service.KylinUserService;
+import org.apache.kylin.storage.hbase.util.DeployCoprocessorCLI;
+import org.apache.kylin.storage.hybrid.HybridInstance;
+import org.apache.kylin.storage.hybrid.HybridManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+public class DstClusterUtil extends ClusterUtil {
+    private static final Logger logger = LoggerFactory.getLogger(DstClusterUtil.class);
+
+    public static final String hbaseSubDir = "migration/hbase/data/default/";
+
+    private final String hbaseDataDirQualified;
+    private final String hbaseDataDir;
+
+    private final boolean ifExecute;
+
+    public DstClusterUtil(String configURI, boolean ifExecute) throws IOException {
+        this(configURI, true, true, ifExecute);
+    }
+
+    public DstClusterUtil(String configURI, boolean ifJobFSHAEnabled, boolean ifHBaseFSHAEnabled, boolean ifExecute)
+            throws IOException {
+        super(configURI, ifJobFSHAEnabled, ifHBaseFSHAEnabled);
+        this.hbaseDataDirQualified = hbaseHdfsWorkingDirectoryQualified + hbaseSubDir;
+        this.hbaseDataDir = hdfsWorkingDirectory + hbaseSubDir;
+        this.ifExecute = ifExecute;
+    }
+
+    @Override
+    public ProjectInstance getProject(String projName) throws IOException {
+        return resourceStore.getResource(ProjectInstance.concatResourcePath(projName),
+                ProjectManager.PROJECT_SERIALIZER);
+    }
+
+    @Override
+    public DictionaryInfo getDictionaryInfo(String dictPath) throws IOException {
+        return resourceStore.getResource(dictPath, DictionaryInfoSerializer.FULL_SERIALIZER);
+    }
+
+    @Override
+    public SnapshotTable getSnapshotTable(String snapshotPath) throws IOException {
+        return resourceStore.getResource(snapshotPath, SnapshotTableSerializer.FULL_SERIALIZER);
+    }
+
+    @Override
+    public String getRootDirQualifiedOfHTable(String tableName) {
+        return hbaseDataDirQualified + tableName;
+    }
+
+    private String getRootDirOfHTable(String tableName) {
+        return hbaseDataDir + tableName;
+    }
+
+    public boolean exists(String resPath) throws IOException {
+        return resourceStore.exists(resPath);
+    }
+
+    public void checkCompatibility(String projectName, Set<TableDesc> tableSet, DataModelDesc modelDesc,
+            boolean ifHiveCheck) throws IOException {
+        List<String> tableDataList = Lists.newArrayList();
+        for (TableDesc table : tableSet) {
+            tableDataList.add(JsonUtil.writeValueAsIndentString(table));
+        }
+
+        String modelDescData = JsonUtil.writeValueAsIndentString(modelDesc);
+
+        CompatibilityCheckRequest request = new CompatibilityCheckRequest();
+        request.setProjectName(projectName);
+        request.setTableDescDataList(tableDataList);
+        request.setModelDescData(modelDescData);
+
+        String jsonRequest = JsonUtil.writeValueAsIndentString(request);
+        restClient.checkCompatibility(jsonRequest, ifHiveCheck);
+    }
+
+    public void saveProject(ProjectInstance projInstance) throws IOException {
+        if (ifExecute) {
+            putMetaResource(ProjectInstance.concatResourcePath(projInstance.getName()), projInstance,
+                    ProjectManager.PROJECT_SERIALIZER);
+        }
+        logger.info("saved project {}", projInstance);
+    }
+
+    public void saveHybrid(HybridInstance hybridInstance) throws IOException {
+        if (ifExecute) {
+            putMetaResource(HybridInstance.concatResourcePath(hybridInstance.getName()), hybridInstance,
+                    HybridManager.HYBRID_SERIALIZER);
+        }
+        logger.info("saved hybrid {}", hybridInstance);
+    }
+
+    public void saveTableDesc(TableDesc table) throws IOException {
+        if (ifExecute) {
+            putMetaResource(TableDesc.concatResourcePath(table.getIdentity(), table.getProject()), table,
+                    TableMetadataManager.TABLE_SERIALIZER);
+        }
+        logger.info("saved table {}", table);
+    }
+
+    public void saveModelDesc(DataModelDesc modelDesc) throws IOException {
+        if (ifExecute) {
+            putMetaResource(DataModelDesc.concatResourcePath(modelDesc.getName()), modelDesc,
+                    DataModelManager.MODELDESC_SERIALIZER);
+        }
+        logger.info("saved model {}", modelDesc);
+    }
+
+    public void saveCubeDesc(CubeDesc cubeDesc) throws IOException {
+        if (ifExecute) {
+            putMetaResource(CubeDesc.concatResourcePath(cubeDesc.getName()), cubeDesc,
+                    CubeDescManager.CUBE_DESC_SERIALIZER);
+        }
+        logger.info("saved cube desc {}", cubeDesc);
+    }
+
+    public void saveCubeInstance(CubeInstance cube) throws IOException {
+        if (ifExecute) {
+            putMetaResource(CubeInstance.concatResourcePath(cube.getName()), cube, CubeManager.CUBE_SERIALIZER);
+        }
+        logger.info("saved cube instance {}", cube);
+    }
+
+    public String saveDictionary(DictionaryInfo dictInfo) throws IOException {
+        String dupDict = checkDupDict(dictInfo);
+        if (dupDict == null) {
+            putMetaResource(dictInfo.getResourcePath(), dictInfo, DictionaryInfoSerializer.FULL_SERIALIZER);
+            logger.info("saved dictionary {}", dictInfo.getResourcePath());
+        }
+        return dupDict;
+    }
+
+    private String checkDupDict(DictionaryInfo dictInfo) throws IOException {
+        NavigableSet<String> existings = resourceStore.listResources(dictInfo.getResourceDir());
+        if (existings == null)
+            return null;
+
+        logger.info("{} existing dictionaries of the same column", existings.size());
+        if (existings.size() > 100) {
+            logger.warn("Too many dictionaries under {}, dict count: {}", dictInfo.getResourceDir(), existings.size());
+        }
+
+        for (String existing : existings) {
+            DictionaryInfo existingInfo = getDictionaryInfo(existing);
+            if (existingInfo != null && dictInfo.getDictionaryObject().equals(existingInfo.getDictionaryObject())) {
+                return existing;
+            }
+        }
+
+        return null;
+    }
+
+    public void saveSnapshotTable(SnapshotTable snapshotTable) throws IOException {
+        putMetaResource(snapshotTable.getResourcePath(), snapshotTable, SnapshotTableSerializer.FULL_SERIALIZER);
+        logger.info("saved snapshot table {}", snapshotTable.getResourcePath());
+    }
+
+    public void saveExtSnapshotTableInfo(ExtTableSnapshotInfo extTableSnapshotInfo) throws IOException {
+        putMetaResource(extTableSnapshotInfo.getResourcePath(), extTableSnapshotInfo,
+                ExtTableSnapshotInfoManager.SNAPSHOT_SERIALIZER);
+        logger.info("saved ext snapshot table info {}", extTableSnapshotInfo.getResourcePath());
+    }
+
+    public void saveUserInfo(String userKey, ManagedUser user) throws IOException {
+        if (ifExecute) {
+            putMetaResource(userKey, user, KylinUserService.SERIALIZER);
+        }
+        logger.info("saved user info {}", userKey);
+    }
+
+    private <T extends RootPersistentEntity> void putMetaResource(String resPath, T obj, Serializer<T> serializer)
+            throws IOException {
+        putMetaResource(resPath, obj, serializer, true);
+    }
+
+    public <T extends RootPersistentEntity> void putMetaResource(String resPath, T obj, Serializer<T> serializer,
+            boolean withoutCheck) throws IOException {
+        if (ifExecute) {
+            if (withoutCheck) {
+                resourceStore.putResource(resPath, obj, System.currentTimeMillis(), serializer);
+            } else {
+                resourceStore.checkAndPutResource(resPath, obj, System.currentTimeMillis(), serializer);
+            }
+        }
+        logger.info("saved resource {}", resPath);
+    }
+
+    public void putResource(String resPath, RawResource res) throws IOException {
+        if (ifExecute) {
+            resourceStore.putResource(resPath, res.content(), res.lastModified());
+        }
+        logger.info("saved resource {}", resPath);
+    }
+
+    // if htable does not exist in dst, return false;
+    // if htable exists in dst, and the segment tags are the same, if the htable is enabled, then return true;
+    //                                                             else delete the htable and return false;
+    //                          else the htable is used by others, should throw runtime exception
+    public boolean checkExist(TableName htableName, CubeSegment segment) throws IOException {
+        if (!htableExists(htableName)) {
+            return false;
+        }
+        Table table = hbaseConn.getTable(htableName);
+        HTableDescriptor tableDesc = table.getTableDescriptor();
+        if (segment.toString().equals(tableDesc.getValue(HTableSegmentTag))) {
+            if (hbaseAdmin.isTableEnabled(htableName)) {
+                return true;
+            } else {
+                hbaseAdmin.deleteTable(htableName);
+                logger.info("htable {} is deleted", htableName);
+                return false;
+            }
+        }
+        throw new RuntimeException(
+                "htable name " + htableName + " has been used by " + tableDesc.getValue(HTableSegmentTag));
+    }
+
+    public void deleteHTable(String tableName) throws IOException {
+        TableName htableName = TableName.valueOf(tableName);
+        if (hbaseAdmin.isTableEnabled(htableName)) {
+            hbaseAdmin.disableTable(htableName);
+        }
+        hbaseAdmin.deleteTable(htableName);
+        logger.info("htable {} is deleted", htableName);
+    }
+
+    public boolean htableExists(TableName htableName) throws IOException {
+        return hbaseAdmin.tableExists(htableName);
+    }
+
+    public void resetTableHost(HTableDescriptor tableDesc) {
+        tableDesc.setValue(IRealizationConstants.HTableTag, kylinConfig.getMetadataUrlPrefix());
+    }
+
+    public void deployCoprocessor(HTableDescriptor tableDesc, String localCoprocessorJar) throws IOException {
+        List<String> existingCoprocessors = tableDesc.getCoprocessors();
+        for (String existingCoprocessor : existingCoprocessors) {
+            tableDesc.removeCoprocessor(existingCoprocessor);
+        }
+
+        Path hdfsCoprocessorJar = DeployCoprocessorCLI.uploadCoprocessorJar(localCoprocessorJar, hbaseFS,
+                hdfsWorkingDirectory, null);
+
+        if (User.isHBaseSecurityEnabled(hbaseConf)) {
+            // add coprocessor for bulk load
+            tableDesc.addCoprocessor("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
+        }
+        DeployCoprocessorCLI.addCoprocessorOnHTable(tableDesc, hdfsCoprocessorJar);
+
+        logger.info("deployed hbase table {} with coprocessor.", tableDesc.getTableName());
+    }
+
+    public void createTable(HTableDescriptor tableDesc, byte[][] splitKeys) throws IOException {
+        hbaseAdmin.createTable(tableDesc, splitKeys);
+
+        logger.info("htable {} successfully created!", tableDesc.getTableName());
+    }
+
+    public void copyInitOnJobCluster(Path path) throws IOException {
+        copyInit(jobFS, path);
+    }
+
+    public void copyInitOnHBaseCluster(Path path) throws IOException {
+        copyInit(hbaseFS, path);
+    }
+
+    public static void copyInit(FileSystem fs, Path path) throws IOException {
+        path = Path.getPathWithoutSchemeAndAuthority(path);
+        Path pathP = path.getParent();
+        if (!fs.exists(pathP)) {
+            fs.mkdirs(pathP);
+        }
+        if (fs.exists(path)) {
+            logger.warn("path {} already existed and will be deleted", path);
+            HadoopUtil.deletePath(fs.getConf(), path);
+        }
+    }
+
+    public void bulkLoadTable(String tableName) throws Exception {
+        Path rootPathOfTable = new Path(getRootDirOfHTable(tableName));
+        FileStatus[] regionFiles = hbaseFS.listStatus(rootPathOfTable, new PathFilter() {
+            @Override
+            public boolean accept(Path path) {
+                return !path.getName().startsWith(".");
+            }
+        });
+
+        for (FileStatus regionFileStatus : regionFiles) {
+            ToolRunner.run(new LoadIncrementalHFiles(hbaseConf),
+                    new String[] { regionFileStatus.getPath().toString(), tableName });
+        }
+
+        logger.info("succeed to migrate htable {}", tableName);
+    }
+
+    public void updateMeta() {
+        if (ifExecute) {
+            try {
+                logger.info("update meta cache for {}", restClient);
+                restClient.announceWipeCache(Broadcaster.SYNC_ALL, Broadcaster.Event.UPDATE.getType(),
+                        Broadcaster.SYNC_ALL);
+            } catch (IOException e) {
+                logger.error(e.getMessage());
+            }
+        }
+    }
+}
diff --git a/tool/src/main/java/org/apache/kylin/tool/migration/SrcClusterUtil.java b/tool/src/main/java/org/apache/kylin/tool/migration/SrcClusterUtil.java
new file mode 100644
index 0000000..7f865d5
--- /dev/null
+++ b/tool/src/main/java/org/apache/kylin/tool/migration/SrcClusterUtil.java
@@ -0,0 +1,148 @@
+/*
+ * 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.tool.migration;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.kylin.cube.CubeDescManager;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.dict.DictionaryInfo;
+import org.apache.kylin.dict.DictionaryManager;
+import org.apache.kylin.dict.lookup.ExtTableSnapshotInfo;
+import org.apache.kylin.dict.lookup.ExtTableSnapshotInfoManager;
+import org.apache.kylin.dict.lookup.SnapshotManager;
+import org.apache.kylin.dict.lookup.SnapshotTable;
+import org.apache.kylin.metadata.TableMetadataManager;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.DataModelManager;
+import org.apache.kylin.metadata.model.TableDesc;
+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.storage.hybrid.HybridInstance;
+import org.apache.kylin.storage.hybrid.HybridManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SrcClusterUtil extends ClusterUtil {
+    private static final Logger logger = LoggerFactory.getLogger(SrcClusterUtil.class);
+
+    private static final String hbaseRootDirConfKey = "hbase.rootdir";
+    private final String hbaseDataDir;
+
+    private final TableMetadataManager metadataManager;
+    private final DataModelManager modelManager;
+    private final ProjectManager projectManager;
+    private final HybridManager hybridManager;
+    private final CubeManager cubeManager;
+    private final CubeDescManager cubeDescManager;
+    private final RealizationRegistry realizationRegistry;
+    private final DictionaryManager dictionaryManager;
+    private final SnapshotManager snapshotManager;
+    private final ExtTableSnapshotInfoManager extSnapshotInfoManager;
+
+    public SrcClusterUtil(String configURI, boolean ifJobFSHAEnabled, boolean ifHBaseFSHAEnabled) throws IOException {
+        super(configURI, ifJobFSHAEnabled, ifHBaseFSHAEnabled);
+
+        this.hbaseDataDir = hbaseConf.get(hbaseRootDirConfKey) + "/data/default/";
+        metadataManager = TableMetadataManager.getInstance(kylinConfig);
+        modelManager = DataModelManager.getInstance(kylinConfig);
+        projectManager = ProjectManager.getInstance(kylinConfig);
+        hybridManager = HybridManager.getInstance(kylinConfig);
+        cubeManager = CubeManager.getInstance(kylinConfig);
+        cubeDescManager = CubeDescManager.getInstance(kylinConfig);
+        realizationRegistry = RealizationRegistry.getInstance(kylinConfig);
+        dictionaryManager = DictionaryManager.getInstance(kylinConfig);
+        snapshotManager = SnapshotManager.getInstance(kylinConfig);
+        extSnapshotInfoManager = ExtTableSnapshotInfoManager.getInstance(kylinConfig);
+    }
+
+    public String getDefaultCoprocessorJarPath() {
+        return kylinConfig.getCoprocessorLocalJar();
+    }
+
+    @Override
+    public ProjectInstance getProject(String projectName) throws IOException {
+        return projectManager.getProject(projectName);
+    }
+
+    public List<ProjectInstance> listAllProjects() throws IOException {
+        return projectManager.listAllProjects();
+    }
+
+    public ProjectInstance getProjectByRealization(RealizationType type, String realizationName) throws IOException {
+        List<ProjectInstance> ret = projectManager.findProjects(type, realizationName);
+        return ret.isEmpty() ? null : ret.get(0);
+    }
+
+    public CubeInstance getCube(String name) throws IOException {
+        return cubeManager.getCube(name);
+    }
+
+    public CubeDesc getCubeDesc(String name) throws IOException {
+        return cubeDescManager.getCubeDesc(name);
+    }
+
+    public HybridInstance getHybrid(String name) throws IOException {
+        return hybridManager.getHybridInstance(name);
+    }
+
+    public IRealization getRealization(RealizationEntry entry) throws IOException {
+        return realizationRegistry.getRealization(entry.getType(), entry.getRealization());
+    }
+
+    public DataModelDesc getDataModelDesc(String modelName) throws IOException {
+        return modelManager.getDataModelDesc(modelName);
+    }
+
+    public TableDesc getTableDesc(String tableIdentity, String projectName) throws IOException {
+        TableDesc ret = metadataManager.getStore().getResource(TableDesc.concatResourcePath(tableIdentity, projectName),
+                TableMetadataManager.TABLE_SERIALIZER);
+        if (projectName != null && ret == null) {
+            ret = metadataManager.getStore().getResource(TableDesc.concatResourcePath(tableIdentity, null),
+                    TableMetadataManager.TABLE_SERIALIZER);
+        }
+        return ret;
+    }
+
+    @Override
+    public DictionaryInfo getDictionaryInfo(String dictPath) throws IOException {
+        return dictionaryManager.getDictionaryInfo(dictPath);
+    }
+
+    @Override
+    public SnapshotTable getSnapshotTable(String snapshotPath) throws IOException {
+        return snapshotManager.getSnapshotTable(snapshotPath);
+    }
+
+    public ExtTableSnapshotInfo getExtTableSnapshotInfo(String snapshotPath) throws IOException {
+        return extSnapshotInfoManager.getSnapshot(snapshotPath);
+    }
+
+    @Override
+    public String getRootDirQualifiedOfHTable(String tableName) {
+        return hbaseDataDir + tableName;
+    }
+}
\ No newline at end of file


[kylin] 10/15: KYLIN-4488 Revert KYLIN-2999

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit e0d92d13e9cb78474f542590127421098667eff9
Author: Zhong, Yanghong <nj...@apache.org>
AuthorDate: Mon May 18 13:14:06 2020 +0800

    KYLIN-4488 Revert KYLIN-2999
---
 .../org/apache/kylin/common/KylinConfigBase.java   | 25 +----------
 .../kylin/rest/controller/CubeController.java      |  7 ----
 .../org/apache/kylin/rest/service/CubeService.java | 49 ----------------------
 .../kylin/rest/service/AdminServiceTest.java       |  1 -
 webapp/app/js/controllers/cubes.js                 | 29 -------------
 webapp/app/js/services/cubes.js                    |  1 -
 webapp/app/js/services/kylinProperties.js          |  8 ----
 webapp/app/partials/cubes/cubes.html               |  1 -
 8 files changed, 1 insertion(+), 120 deletions(-)

diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 62021a9..7db46a6 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -2257,7 +2257,7 @@ public abstract class KylinConfigBase implements Serializable {
                 + "kylin.web.help,kylin.web.hide-measures,kylin.web.link-streaming-guide,kylin.server.external-acl-provider,"
                 + "kylin.security.profile,kylin.security.additional-profiles,"
                 + "kylin.htrace.show-gui-trace-toggle,kylin.web.export-allow-admin,kylin.web.export-allow-other,"
-                + "kylin.cube.cubeplanner.enabled,kylin.web.dashboard-enabled,kylin.tool.auto-migrate-cube.enabled,"
+                + "kylin.cube.cubeplanner.enabled,kylin.web.dashboard-enabled,"
                 + "kylin.job.scheduler.default,kylin.web.default-time-filter");
     }
 
@@ -2404,29 +2404,6 @@ public abstract class KylinConfigBase implements Serializable {
     }
 
     // ============================================================================
-    // tool
-    // ============================================================================
-    public boolean isAllowAutoMigrateCube() {
-        return Boolean.parseBoolean(getOptional("kylin.tool.auto-migrate-cube.enabled", FALSE));
-    }
-
-    public boolean isAutoMigrateCubeCopyAcl() {
-        return Boolean.parseBoolean(getOptional("kylin.tool.auto-migrate-cube.copy-acl", TRUE));
-    }
-
-    public boolean isAutoMigrateCubePurge() {
-        return Boolean.parseBoolean(getOptional("kylin.tool.auto-migrate-cube.purge-src-cube", TRUE));
-    }
-
-    public String getAutoMigrateCubeSrcConfig() {
-        return getOptional("kylin.tool.auto-migrate-cube.src-config", "");
-    }
-
-    public String getAutoMigrateCubeDestConfig() {
-        return getOptional("kylin.tool.auto-migrate-cube.dest-config", "");
-    }
-
-    // ============================================================================
     // jdbc metadata resource store
     // ============================================================================
 
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
index 518e83c..1b5f1e4 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
@@ -1038,13 +1038,6 @@ public class CubeController extends BasicController {
         }
     }
 
-    @RequestMapping(value = "/{cube}/{project}/migrate", method = { RequestMethod.POST })
-    @ResponseBody
-    public void migrateCube(@PathVariable String cube, @PathVariable String project) {
-        CubeInstance cubeInstance = cubeService.getCubeManager().getCube(cube);
-        cubeService.migrateCube(cubeInstance, project);
-    }
-
     public void setCubeService(CubeService cubeService) {
         this.cubeService = cubeService;
     }
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
index 94f24ca..f11f31a 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.lock.DistributedLock;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
-import org.apache.kylin.common.util.CliCommandExecutor;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeInstance;
@@ -51,7 +50,6 @@ import org.apache.kylin.engine.mr.JobBuilderSupport;
 import org.apache.kylin.engine.mr.common.CubeJobLockUtil;
 import org.apache.kylin.engine.mr.common.CuboidRecommenderUtil;
 import org.apache.kylin.job.JobInstance;
-import org.apache.kylin.job.common.PatternedLogger;
 import org.apache.kylin.job.constant.JobStatusEnum;
 import org.apache.kylin.job.constant.JobTimeFilterEnum;
 import org.apache.kylin.job.exception.JobException;
@@ -78,7 +76,6 @@ import org.apache.kylin.metrics.property.QueryCubePropertyEnum;
 import org.apache.kylin.rest.constant.Constant;
 import org.apache.kylin.rest.exception.BadRequestException;
 import org.apache.kylin.rest.exception.ForbiddenException;
-import org.apache.kylin.rest.exception.InternalErrorException;
 import org.apache.kylin.rest.msg.Message;
 import org.apache.kylin.rest.msg.MsgPicker;
 import org.apache.kylin.rest.request.MetricsRequest;
@@ -106,7 +103,6 @@ import org.springframework.security.access.prepost.PreAuthorize;
 import org.springframework.security.core.context.SecurityContextHolder;
 import org.springframework.stereotype.Component;
 
-import org.apache.kylin.shaded.com.google.common.base.Preconditions;
 import org.apache.kylin.shaded.com.google.common.cache.Cache;
 import org.apache.kylin.shaded.com.google.common.cache.CacheBuilder;
 import org.apache.kylin.shaded.com.google.common.collect.Lists;
@@ -1107,51 +1103,6 @@ public class CubeService extends BasicService implements InitializingBean {
         return queryService.doQueryWithCache(sqlRequest, false).getResults();
     }
 
-    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN
-            + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
-    public void migrateCube(CubeInstance cube, String projectName) {
-        KylinConfig config = KylinConfig.getInstanceFromEnv();
-        if (!config.isAllowAutoMigrateCube()) {
-            throw new InternalErrorException("One click migration is disabled, please contact your ADMIN");
-        }
-
-        for (CubeSegment segment : cube.getSegments()) {
-            if (segment.getStatus() != SegmentStatusEnum.READY) {
-                throw new InternalErrorException(
-                        "At least one segment is not in READY state. Please check whether there are Running or Error jobs.");
-            }
-        }
-
-        String srcCfgUri = config.getAutoMigrateCubeSrcConfig();
-        String dstCfgUri = config.getAutoMigrateCubeDestConfig();
-
-        Preconditions.checkArgument(StringUtils.isNotEmpty(srcCfgUri), "Source configuration should not be empty.");
-        Preconditions.checkArgument(StringUtils.isNotEmpty(dstCfgUri),
-                "Destination configuration should not be empty.");
-
-        String stringBuilder = ("%s/bin/kylin.sh org.apache.kylin.tool.CubeMigrationCLI %s %s %s %s %s %s true true");
-        String cmd = String.format(Locale.ROOT,
-                stringBuilder,
-                KylinConfig.getKylinHome(),
-                CliCommandExecutor.checkParameterWhiteList(srcCfgUri),
-                CliCommandExecutor.checkParameterWhiteList(dstCfgUri),
-                cube.getName(),
-                CliCommandExecutor.checkParameterWhiteList(projectName),
-                config.isAutoMigrateCubeCopyAcl(),
-                config.isAutoMigrateCubePurge());
-
-        logger.info("One click migration cmd: " + cmd);
-
-        CliCommandExecutor exec = new CliCommandExecutor();
-        PatternedLogger patternedLogger = new PatternedLogger(logger);
-
-        try {
-            exec.execute(cmd, patternedLogger);
-        } catch (IOException e) {
-            throw new InternalErrorException("Failed to perform one-click migrating", e);
-        }
-    }
-
     private class HTableInfoSyncListener extends Broadcaster.Listener {
         @Override
         public void onClearAll(Broadcaster broadcaster) throws IOException {
diff --git a/server/src/test/java/org/apache/kylin/rest/service/AdminServiceTest.java b/server/src/test/java/org/apache/kylin/rest/service/AdminServiceTest.java
index 27b59a0..dd098de 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/AdminServiceTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/AdminServiceTest.java
@@ -59,7 +59,6 @@ public class AdminServiceTest extends ServiceTestBase {
                     "kylin.web.help.length=4\n" +
                     "kylin.web.timezone=\n" +
                     "kylin.server.external-acl-provider=\n" +
-                    "kylin.tool.auto-migrate-cube.enabled=\n" +
                     "kylin.storage.default=2\n" +
                     "kylin.cube.cubeplanner.enabled=true\n" +
                     "kylin.web.help=\n" +
diff --git a/webapp/app/js/controllers/cubes.js b/webapp/app/js/controllers/cubes.js
index 7000581..2d9c032 100644
--- a/webapp/app/js/controllers/cubes.js
+++ b/webapp/app/js/controllers/cubes.js
@@ -359,35 +359,6 @@ KylinApp.controller('CubesCtrl', function ($scope, $q, $routeParams, $location,
       });
     };
 
-    $scope.isAutoMigrateCubeEnabled = function(){
-      return kylinConfig.isAutoMigrateCubeEnabled();
-    };
-
-    $scope.migrateCube = function (cube) {
-      SweetAlert.swal({
-        title: '',
-        text: "The cube will overwrite the same cube in prod env" +
-        "\nMigrating cube will elapse a couple of minutes." +
-        "\nPlease wait.",
-        type: '',
-        showCancelButton: true,
-        confirmButtonColor: '#DD6B55',
-        confirmButtonText: "Yes",
-        closeOnConfirm: true
-      }, function(isConfirm) {
-        if(isConfirm){
-          loadingRequest.show();
-          CubeService.autoMigrate({cubeId: cube.name, propName: $scope.projectModel.selectedProject}, {}, function (result) {
-            loadingRequest.hide();
-            MessageBox.successNotify(cube.name + ' migrate successfully!');
-          },function(e){
-            loadingRequest.hide();
-            SweetAlert.swal('Migrate failed!', "Please contact your ADMIN.", 'error');
-          });
-        }
-      });
-    };
-
     $scope.startJobSubmit = function (cube) {
 
       $scope.metaModel={
diff --git a/webapp/app/js/services/cubes.js b/webapp/app/js/services/cubes.js
index 52cb8fe..65a4d5d 100644
--- a/webapp/app/js/services/cubes.js
+++ b/webapp/app/js/services/cubes.js
@@ -79,7 +79,6 @@ KylinApp.factory('CubeService', ['$resource', function ($resource, config) {
       }
     },
     optimize: {method: 'PUT', params: {action: 'optimize'}, isArray: false},
-    autoMigrate: {method: 'POST', params: {action: 'migrate'}, isArray: false},
     lookupRefresh: {method: 'PUT', params: {action: 'refresh_lookup'}, isArray: false},
     checkDuplicateCubeName: {method: 'GET', params: {action: 'validate'}, isArray: false}
   });
diff --git a/webapp/app/js/services/kylinProperties.js b/webapp/app/js/services/kylinProperties.js
index 05d0621..05fce98 100644
--- a/webapp/app/js/services/kylinProperties.js
+++ b/webapp/app/js/services/kylinProperties.js
@@ -167,14 +167,6 @@ KylinApp.service('kylinConfig', function (AdminService, $log) {
     return angular.isString(_config);
   }
 
-  this.isAutoMigrateCubeEnabled = function(){
-    var status = this.getProperty("kylin.tool.auto-migrate-cube.enabled").trim();
-    if(status && status =='true'){
-      return true;
-    }
-    return false;
-  }
-
   this.getSourceType = function(){
     this.sourceType = this.getProperty("kylin.source.default").trim();
     if (!this.sourceType) {
diff --git a/webapp/app/partials/cubes/cubes.html b/webapp/app/partials/cubes/cubes.html
index fd176f7..65ebc87 100644
--- a/webapp/app/partials/cubes/cubes.html
+++ b/webapp/app/partials/cubes/cubes.html
@@ -102,7 +102,6 @@
                         <li ng-if="cube.status=='DISABLED' && (userService.hasRole('ROLE_ADMIN') || hasPermission('cube',cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask))"><a ng-click="startDeleteSegment(cube)">Delete Segment</a></li>
                         <li ng-if="cube.status=='DISABLED' && (userService.hasRole('ROLE_ADMIN') || hasPermission('cube',cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask))"><a ng-click="purge(cube)">Purge</a></li>
                         <li ng-if="cube.status!='DESCBROKEN' && (userService.hasRole('ROLE_ADMIN') || hasPermission('cube',cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask))"><a ng-click="cloneCube(cube)">Clone</a></li>
-                        <li ng-if="cube.status=='READY' && isAutoMigrateCubeEnabled() && (userService.hasRole('ROLE_ADMIN') || hasPermission('cube',cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask)) "><a ng-click="migrateCube(cube)">Migrate</a></li>
 
                     </ul>
                     <ul ng-if="(userService.hasRole('ROLE_ADMIN') || hasPermission('cube', cube, permissions.ADMINISTRATION.mask, permissions.MANAGEMENT.mask, permissions.OPERATION.mask)) && cube.streamingV2 && actionLoaded" class="dropdown-menu" role="menu" style="right:0;left:auto;">