KYLIN-1884 Reload metadata automatically after migrating cube Signed-off-by: shaofengshi <shaofeng...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/b7fdafd8 Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/b7fdafd8 Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/b7fdafd8 Branch: refs/heads/1.5.x-HBase1.x Commit: b7fdafd86b4d1cabd92d4ed0c9762acba19de123 Parents: 28a2b9f Author: kangkaisen <kangkai...@live.com> Authored: Wed Jul 13 17:42:33 2016 +0800 Committer: shaofengshi <shaofeng...@apache.org> Committed: Mon Jul 18 21:33:31 2016 +0800 ---------------------------------------------------------------------- .../kylin/storage/hbase/util/CubeMigrationCLI.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kylin/blob/b7fdafd8/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java ---------------------------------------------------------------------- diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java index 32aa4b0..9a83a3f 100644 --- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java +++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java @@ -42,6 +42,8 @@ import org.apache.kylin.common.persistence.JsonSerializer; import org.apache.kylin.common.persistence.RawResource; import org.apache.kylin.common.persistence.ResourceStore; import org.apache.kylin.common.persistence.Serializer; +import org.apache.kylin.common.restclient.Broadcaster; +import org.apache.kylin.common.restclient.RestClient; import org.apache.kylin.common.util.Bytes; import org.apache.kylin.common.util.Dictionary; import org.apache.kylin.cube.CubeInstance; @@ -148,6 +150,7 @@ public class CubeMigrationCLI { if (realExecute.equalsIgnoreCase("true")) { doOpts(); checkMigrationSuccess(dstConfig, cubeName, true); + updateMeta(dstConfig); } else { showOpts(); } @@ -547,4 +550,17 @@ public class CubeMigrationCLI { } } } + + private static void updateMeta(KylinConfig config){ + String[] nodes = config.getRestServers(); + for (String node : nodes) { + RestClient restClient = new RestClient(node); + try { + logger.info("update meta cache for " + node); + restClient.wipeCache(Broadcaster.TYPE.ALL.getType(), Broadcaster.EVENT.UPDATE.getType(), "all"); + } catch (IOException e) { + logger.error(e.getMessage()); + } + } + } }