KYLIN-2446 Support project names filter in DeployCoprocessorCLI
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/69f89848 Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/69f89848 Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/69f89848 Branch: refs/heads/master-hbase0.98 Commit: 69f898482499d530dddeb35b3327f65a50899dd5 Parents: e2ff784 Author: kangkaisen <kangkai...@163.com> Authored: Tue Feb 14 17:25:44 2017 +0800 Committer: kangkaisen <kangkai...@163.com> Committed: Tue Feb 14 17:43:14 2017 +0800 ---------------------------------------------------------------------- .../hbase/util/DeployCoprocessorCLI.java | 40 +++++++++++++++++++- 1 file changed, 39 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kylin/blob/69f89848/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java ---------------------------------------------------------------------- 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 e72859d..29c738e 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 @@ -54,7 +54,11 @@ import org.apache.kylin.cube.CubeInstance; import org.apache.kylin.cube.CubeManager; import org.apache.kylin.cube.CubeSegment; import org.apache.kylin.metadata.model.SegmentStatusEnum; +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.IRealizationConstants; +import org.apache.kylin.metadata.realization.RealizationType; import org.apache.kylin.storage.hbase.HBaseConnection; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -102,11 +106,14 @@ public class DeployCoprocessorCLI { tableNames = filterByTables(tableNames, Arrays.asList(args).subList(2, args.length)); } else if (filterType.equals("-cube")) { tableNames = filterByCubes(tableNames, Arrays.asList(args).subList(2, args.length)); + } else if (filterType.equals("-project")) { + tableNames = filterByProjects(tableNames, Arrays.asList(args).subList(2, args.length)); } else if (!filterType.equals("all")) { printUsageAndExit(); } logger.info("Will execute tables " + tableNames); + long start = System.currentTimeMillis(); Set<String> oldJarPaths = getCoprocessorJarPaths(hbaseAdmin, tableNames); logger.info("Old coprocessor jar: " + oldJarPaths); @@ -121,16 +128,47 @@ public class DeployCoprocessorCLI { hbaseAdmin.close(); + logger.info("Processed time: " + (System.currentTimeMillis() - start)); logger.info("Processed tables count: " + processedTables.size()); logger.info("Processed tables: " + processedTables); logger.info("Active coprocessor jar: " + hdfsCoprocessorJar); } private static void printUsageAndExit() { - logger.info("Probe run, exiting. Append argument 'all' or specific tables/cubes to execute."); + logger.info("Usage: "); + logger.info("$KYLIN_HOME/bin/kylin.sh org.apache.kylin.storage.hbase.util.DeployCoprocessorCLI $KYLIN_HOME/lib/kylin-coprocessor-*.jar all"); + logger.info("$KYLIN_HOME/bin/kylin.sh org.apache.kylin.storage.hbase.util.DeployCoprocessorCLI $KYLIN_HOME/lib/kylin-coprocessor-*.jar -table tableName1 tableName2 ..."); + logger.info("$KYLIN_HOME/bin/kylin.sh org.apache.kylin.storage.hbase.util.DeployCoprocessorCLI $KYLIN_HOME/lib/kylin-coprocessor-*.jar -cube cubeName1 cubeName2 ... "); + logger.info("$KYLIN_HOME/bin/kylin.sh org.apache.kylin.storage.hbase.util.DeployCoprocessorCLI $KYLIN_HOME/lib/kylin-coprocessor-*.jar -project projectName1 projectName2 ..."); System.exit(0); } + private static List<String> filterByProjects(List<String> allTableNames, List<String> projectNames) { + ProjectManager projectManager = ProjectManager.getInstance(KylinConfig.getInstanceFromEnv()); + CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()); + + List<String> result = Lists.newArrayList(); + for (String p : projectNames) { + p = p.trim(); + if (p.endsWith(",")) { + p = p.substring(0, p.length() - 1); + } + + ProjectInstance projectInstance = projectManager.getProject(p); + List<RealizationEntry> cubeList = projectInstance.getRealizationEntries(RealizationType.CUBE); + for (RealizationEntry cube: cubeList) { + CubeInstance cubeInstance = cubeManager.getCube(cube.getRealization()); + for (CubeSegment segment : cubeInstance.getSegments()) { + String tableName = segment.getStorageLocationIdentifier(); + if (allTableNames.contains(tableName)) { + result.add(tableName); + } + } + } + } + return result; + } + private static List<String> filterByCubes(List<String> allTableNames, List<String> cubeNames) { CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()); List<String> result = Lists.newArrayList();