This is an automated email from the ASF dual-hosted git repository. xxyu pushed a commit to branch kylin-on-parquet-v2 in repository https://gitbox.apache.org/repos/asf/kylin.git
commit 3d6db494526094c6f7f1134e9fe576d7e0c66307 Author: XiaoxiangYu <x...@apache.org> AuthorDate: Wed Dec 2 15:26:42 2020 +0800 KYLIN-4818 Support Cube Planner Phase One in Kylin 4 - Use Spark to calculate cuboid's HllCounter for the first segment - Re-enable Cube planner by default - Change default precision from 10 to 14 for HLLCounter --- .../kylin/engine/mr/common/CubeStatsReader.java | 4 +- .../src/main/resources/kylin-defaults.properties | 2 +- .../org/apache/kylin/measure/hllc/HLLCounter.java | 2 +- .../kylin/engine/spark/job/CubeBuildJob.java | 74 +++++++++++++++++++--- ...dStatistics.scala => CuboidStatisticsJob.scala} | 39 +++++++++--- .../engine/spark/job/ParentSourceChooser.scala | 24 +++---- .../spark/job/ResourceDetectBeforeCubingJob.java | 1 - .../engine/spark/metadata/MetadataConverter.scala | 2 +- .../kylin/rest/controller/CubeController.java | 51 +++++++-------- webapp/app/partials/cubes/cube_detail.html | 2 +- 10 files changed, 138 insertions(+), 63 deletions(-) diff --git a/build-engine/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java b/build-engine/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java index 3c93d05..88b94b7 100644 --- a/build-engine/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java +++ b/build-engine/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java @@ -86,7 +86,7 @@ public class CubeStatsReader { final double mapperOverlapRatioOfFirstBuild; // becomes meaningless after merge final Map<Long, HLLCounter> cuboidRowEstimatesHLL; final CuboidScheduler cuboidScheduler; - final long sourceRowCount; + public final long sourceRowCount; public CubeStatsReader(CubeSegment cubeSegment, KylinConfig kylinConfig) throws IOException { this(cubeSegment, cubeSegment.getCuboidScheduler(), kylinConfig); @@ -105,7 +105,7 @@ public class CubeStatsReader { File tmpSeqFile = writeTmpSeqFile(resource.content()); Path path = new Path(HadoopUtil.fixWindowsPath("file://" + tmpSeqFile.getAbsolutePath())); - + logger.info("Reading statistics from {}", path); CubeStatsResult cubeStatsResult = new CubeStatsResult(path, kylinConfig.getCubeStatsHLLPrecision()); tmpSeqFile.delete(); diff --git a/core-common/src/main/resources/kylin-defaults.properties b/core-common/src/main/resources/kylin-defaults.properties index c5cd317..e33dbe9 100644 --- a/core-common/src/main/resources/kylin-defaults.properties +++ b/core-common/src/main/resources/kylin-defaults.properties @@ -153,7 +153,7 @@ kylin.cube.algorithm.inmem-auto-optimize=true kylin.cube.aggrgroup.max-combination=32768 -kylin.cube.cubeplanner.enabled=false +kylin.cube.cubeplanner.enabled=true kylin.cube.cubeplanner.enabled-for-existing-cube=false kylin.cube.cubeplanner.expansion-threshold=15.0 kylin.cube.cubeplanner.recommend-cache-max-size=200 diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCounter.java index 6a3caec..1c1371f 100644 --- a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCounter.java +++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCounter.java @@ -52,7 +52,7 @@ public class HLLCounter implements Serializable, Comparable<HLLCounter> { private Register register; public HLLCounter() { - this(10, RegisterType.SINGLE_VALUE, Hashing.murmur3_128()); + this(14, RegisterType.SINGLE_VALUE, Hashing.murmur3_128()); } public HLLCounter(int p) { diff --git a/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CubeBuildJob.java b/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CubeBuildJob.java index 1028911..10514df 100644 --- a/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CubeBuildJob.java +++ b/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CubeBuildJob.java @@ -22,14 +22,23 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.kylin.common.persistence.ResourceStore; +import org.apache.kylin.engine.mr.JobBuilderSupport; +import org.apache.kylin.engine.mr.common.BatchConstants; +import org.apache.kylin.engine.mr.common.CubeStatsWriter; +import org.apache.kylin.engine.mr.common.StatisticsDecisionUtil; +import org.apache.kylin.measure.hllc.HLLCounter; import org.apache.kylin.shaded.com.google.common.base.Joiner; import org.apache.kylin.shaded.com.google.common.collect.Maps; import org.apache.hadoop.fs.FileStatus; @@ -74,6 +83,8 @@ import org.apache.kylin.shaded.com.google.common.collect.Sets; import scala.Tuple2; import scala.collection.JavaConversions; +import static org.apache.kylin.engine.mr.common.BatchConstants.CFG_OUTPUT_STATISTICS; + public class CubeBuildJob extends SparkApplication { protected static final Logger logger = LoggerFactory.getLogger(CubeBuildJob.class); protected static String TEMP_DIR_SUFFIX = "_temp"; @@ -83,6 +94,7 @@ public class CubeBuildJob extends SparkApplication { private BuildLayoutWithUpdate buildLayoutWithUpdate; private Map<Long, Short> cuboidShardNum = Maps.newConcurrentMap(); private Map<Long, Long> cuboidsRowCount = Maps.newConcurrentMap(); + public static void main(String[] args) { CubeBuildJob cubeBuildJob = new CubeBuildJob(); cubeBuildJob.execute(args); @@ -92,19 +104,62 @@ public class CubeBuildJob extends SparkApplication { protected void doExecute() throws Exception { long start = System.currentTimeMillis(); - logger.info("Start building cube job..."); - buildLayoutWithUpdate = new BuildLayoutWithUpdate(); + logger.info("Start building cube job for {} ...", getParam(MetadataConstants.P_SEGMENT_IDS)); Set<String> segmentIds = Sets.newHashSet(StringUtils.split(getParam(MetadataConstants.P_SEGMENT_IDS))); + + // For now, Kylin should only build one segment in one time, cube planner has this restriction (maybe we can remove this limitation later) + Preconditions.checkArgument(segmentIds.size() == 1, "Build one segment in one time."); + + String firstSegmentId = segmentIds.iterator().next(); + String cubeName = getParam(MetadataConstants.P_CUBE_ID); + SegmentInfo seg = ManagerHub.getSegmentInfo(config, cubeName, firstSegmentId); cubeManager = CubeManager.getInstance(config); - cubeInstance = cubeManager.getCubeByUuid(getParam(MetadataConstants.P_CUBE_ID)); + cubeInstance = cubeManager.getCubeByUuid(cubeName); + CubeSegment newSegment = cubeInstance.getSegmentById(firstSegmentId); + SpanningTree spanningTree ; + ParentSourceChooser sourceChooser; + + boolean needStatistics = StatisticsDecisionUtil.isAbleToOptimizeCubingPlan(newSegment); // Cuboid Statistics is served for Cube Planner Phase One + + if (needStatistics) { + // 1.1 Call CuboidStatistics#statistics + long startMills = System.currentTimeMillis(); + spanningTree = new ForestSpanningTree(JavaConversions.asJavaCollection(seg.toBuildLayouts())); + sourceChooser = new ParentSourceChooser(spanningTree, seg, jobId, ss, config, false); + sourceChooser.setNeedStatistics(); + sourceChooser.decideFlatTableSource(null); + Map<Long, HLLCounter> hllMap = new HashMap<>(); + for (Tuple2<String, AggInfo> cuboidData : sourceChooser.aggInfo()) { + hllMap.put(Long.parseLong(cuboidData._1), cuboidData._2.cuboid().counter()); + } + logger.info("Cuboid statistics return {} records and cost {} ms.", hllMap.size(), (System.currentTimeMillis() - startMills)); + + // 1.2 Save cuboid statistics + String jobWorkingDirPath = JobBuilderSupport.getJobWorkingDir(cubeInstance.getConfig().getHdfsWorkingDirectory(), jobId); + CubeStatsWriter.writeCuboidStatistics(HadoopUtil.getCurrentConfiguration(), new Path(jobWorkingDirPath + "/" + firstSegmentId + "/" + CFG_OUTPUT_STATISTICS), hllMap, 1); + + FileSystem fs = HadoopUtil.getWorkingFileSystem(); + ResourceStore rs = ResourceStore.getStore(config); + String resPath = newSegment.getStatisticsResourcePath(); + Path statisticsFile = new Path(jobWorkingDirPath + "/" + firstSegmentId + "/" + CFG_OUTPUT_STATISTICS + "/" + BatchConstants.CFG_STATISTICS_CUBOID_ESTIMATION_FILENAME); + FSDataInputStream is = fs.open(statisticsFile); + rs.putResource(resPath, is, System.currentTimeMillis()); + logger.info("{} stats saved to resource {}", newSegment, resPath); + + // 1.3 Trigger cube planner phase one and save optimized cuboid set into CubeInstance + logger.info("Trigger cube planner phase one ."); + StatisticsDecisionUtil.optimizeCubingPlan(newSegment); + } + + buildLayoutWithUpdate = new BuildLayoutWithUpdate(); List<String> persistedFlatTable = new ArrayList<>(); List<String> persistedViewFactTable = new ArrayList<>(); Path shareDir = config.getJobTmpShareDir(project, jobId); try { //TODO: what if a segment is deleted during building? for (String segId : segmentIds) { - SegmentInfo seg = ManagerHub.getSegmentInfo(config, getParam(MetadataConstants.P_CUBE_ID), segId); - SpanningTree spanningTree = new ForestSpanningTree( + seg = ManagerHub.getSegmentInfo(config, cubeName, segId); + spanningTree = new ForestSpanningTree( JavaConversions.asJavaCollection(seg.toBuildLayouts())); logger.debug("There are {} cuboids to be built in segment {}.", seg.toBuildLayouts().size(), seg.name()); @@ -114,9 +169,8 @@ public class CubeBuildJob extends SparkApplication { } // choose source - ParentSourceChooser sourceChooser = new ParentSourceChooser(spanningTree, seg, jobId, ss, config, true); + sourceChooser = new ParentSourceChooser(spanningTree, seg, jobId, ss, config, true); sourceChooser.decideSources(); - Tuple2<String, AggInfo>[] aggInfos = sourceChooser.getAggInfo(); NBuildSourceInfo buildFromFlatTable = sourceChooser.flatTableSource(); Map<Long, NBuildSourceInfo> buildFromLayouts = sourceChooser.reuseSources(); @@ -166,7 +220,7 @@ public class CubeBuildJob extends SparkApplication { String template = "{\"cuboid\":%d, \"rows\": %d, \"size\": %d}"; for (LayoutEntity layoutEntity : segmentInfo.getAllLayoutJava()) { - cuboidStatics.add(String.format(template, layoutEntity.getId(), layoutEntity.getRows(), layoutEntity.getByteSize())); + cuboidStatics.add(String.format(Locale.getDefault(), template, layoutEntity.getId(), layoutEntity.getRows(), layoutEntity.getByteSize())); } JavaSparkContext jsc = JavaSparkContext.fromSparkContext(ss.sparkContext()); @@ -331,7 +385,7 @@ public class CubeBuildJob extends SparkApplication { } private LayoutEntity buildCuboid(SegmentInfo seg, LayoutEntity cuboid, Dataset<Row> parent, - SpanningTree spanningTree, long parentId) throws IOException { + SpanningTree spanningTree, long parentId) throws IOException { String parentName = String.valueOf(parentId); if (parentId == ParentSourceChooser.FLAT_TABLE_FLAG()) { parentName = "flat table"; @@ -399,7 +453,7 @@ public class CubeBuildJob extends SparkApplication { } int shardNum = BuildUtils.repartitionIfNeed(layout, storage, path, tempPath, cubeInstance.getConfig(), ss); layout.setShardNum(shardNum); - cuboidShardNum.put(layoutId, (short)shardNum); + cuboidShardNum.put(layoutId, (short) shardNum); ss.sparkContext().setLocalProperty(QueryExecutionCache.N_EXECUTION_ID_KEY(), null); QueryExecutionCache.removeQueryExecution(queryExecutionId); BuildUtils.fillCuboidInfo(layout, path); diff --git a/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CuboidStatistics.scala b/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CuboidStatisticsJob.scala similarity index 76% rename from kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CuboidStatistics.scala rename to kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CuboidStatisticsJob.scala index d0ea199..c08a6ae 100644 --- a/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CuboidStatistics.scala +++ b/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CuboidStatisticsJob.scala @@ -26,27 +26,35 @@ import org.apache.spark.sql.{Dataset, Row} import scala.collection.mutable +/** + * Calculate HLLCounter for each cuboid, to serve Cube Planner (to calculate cost and benefit of each cuboid). + */ +object CuboidStatisticsJob { -object CuboidStatistics { + /** + * @param inputDs Part of FlatTable which contains all normal dimensions + * @return Cuboid level statistics data + */ + def statistics(inputDs: Dataset[Row], seg: SegmentInfo): Array[(String, AggInfo)] = { - def sample(inputDs: Dataset[Row], seg: SegmentInfo): Array[(String, AggInfo)] = { - seg.getAllLayout.map(x => x.getId) val rkc = seg.allColumns.count(c => c.rowKey) - val res = inputDs.rdd //.sample(withReplacement = false, 0.3) - .mapPartitions(new CuboidStatistics(seg.getAllLayout.map(x => x.getId), rkc).statisticsInPartition) + // maybe we should use sample operation to reduce cost later + val res = inputDs.rdd + .mapPartitions(new CuboidStatisticsJob(seg.getAllLayout.map(x => x.getId), rkc).statisticsWithinPartition) val l = res.map(a => (a.key, a)).reduceByKey((a, b) => a.merge(b)).collect() - l.foreach(x => println(x._1 + " >>><<< " + x._2.cuboid.counter.getCountEstimate)) + // l.foreach(x => println(x._1 + " >>><<< " + x._2.cuboid.counter.getCountEstimate)) l } } -class CuboidStatistics(ids: List[Long], rkc: Int) extends Serializable { +class CuboidStatisticsJob(ids: List[Long], rkc: Int) extends Serializable { private val info = mutable.Map[String, AggInfo]() val allCuboidsBitSet: Array[Array[Integer]] = getCuboidBitSet(ids, rkc) private val hf: HashFunction = Hashing.murmur3_128 private val rowHashCodesLong = new Array[Long](rkc) + private var idx = 0 - def statisticsInPartition(rows: Iterator[Row]): Iterator[AggInfo] = { + def statisticsWithinPartition(rows: Iterator[Row]): Iterator[AggInfo] = { init() rows.foreach(update) info.valuesIterator @@ -57,7 +65,9 @@ class CuboidStatistics(ids: List[Long], rkc: Int) extends Serializable { } def update(r: Row): Unit = { - println(r) + idx += 1 + if (idx <= 5 || idx % 300 == 0) + println(r) updateCuboid(r) } @@ -122,8 +132,17 @@ case class AggInfo(key: String, } } -case class CuboidInfo(counter: HLLCounter = new HLLCounter(14)) +/** + * @param counter HLLCounter will could get est row count for specific cuboid + */ +case class CuboidInfo(counter: HLLCounter = new HLLCounter()) +/** + * @param data Maybe some sample data + */ case class SampleInfo(data: Array[String] = new Array(3)) +/** + * @param range Maybe to save min/max of a specific dimension + */ case class DimensionInfo(range: mutable.Map[String, String] = mutable.Map[String, String]()) diff --git a/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/ParentSourceChooser.scala b/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/ParentSourceChooser.scala index 446f3e6..ab2089d 100644 --- a/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/ParentSourceChooser.scala +++ b/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/ParentSourceChooser.scala @@ -47,15 +47,15 @@ class ParentSourceChooser( // build from flatTable. var flatTableSource: NBuildSourceInfo = _ - var detectStep = false + private var needStatistics = false //TODO: MetadataConverter don't have getCubeDesc() now /*val flatTableDesc = new CubeJoinedFlatTableDesc( MetadataConverter.getCubeDesc(seg.getCube), ParentSourceChooser.needJoinLookupTables(seg.getModel, toBuildTree))*/ - def setDetectStep(): Unit = - detectStep = true + def setNeedStatistics(): Unit = + needStatistics = true def getAggInfo : Array[(String, AggInfo)] = aggInfo @@ -70,7 +70,7 @@ class ParentSourceChooser( } } - private def decideFlatTableSource(entity: LayoutEntity): Unit = { + def decideFlatTableSource(entity: LayoutEntity): Unit = { if (flatTableSource == null) { if (needEncoding) { // hacked, for some case, you do not want to trigger buildSnapshot @@ -83,18 +83,20 @@ class ParentSourceChooser( flatTableSource = getFlatTable val rowKeyColumns: Seq[String] = seg.allColumns.filter(c => c.rowKey).map(c => c.id.toString) - if (aggInfo == null && !detectStep) { - logInfo("Start sampling ...") + if (aggInfo == null && needStatistics) { + val startMs = System.currentTimeMillis() + logInfo("Sampling start ...") val coreDs = flatTableSource.getFlatTableDS.select(rowKeyColumns.head, rowKeyColumns.tail: _*) - aggInfo = CuboidStatistics.sample(coreDs, seg) - logInfo("Finish sampling ...") - val statisticsStr = aggInfo.sortBy(x => x._1).map(x => x._1 + ":" + x._2.cuboid.counter.getCountEstimate).mkString("\n") - logInfo(statisticsStr) + aggInfo = CuboidStatisticsJob.statistics(coreDs, seg) + logInfo("Sampling finished and cost " + (System.currentTimeMillis() - startMs) + " s .") + val statisticsStr = aggInfo.sortBy(x => x._1).map(x => x._1 + ":" + x._2.cuboid.counter.getCountEstimate).mkString(", ") + logInfo("Cuboid Statistics results : \t" + statisticsStr) } else { logInfo("Skip sampling ...") } } - flatTableSource.addCuboid(entity) + if (entity != null) + flatTableSource.addCuboid(entity) } private def decideParentLayoutSource(entity: LayoutEntity, parentLayout: LayoutEntity): Unit = { diff --git a/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/ResourceDetectBeforeCubingJob.java b/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/ResourceDetectBeforeCubingJob.java index c3828cd..065bd5e 100644 --- a/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/ResourceDetectBeforeCubingJob.java +++ b/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/ResourceDetectBeforeCubingJob.java @@ -58,7 +58,6 @@ public class ResourceDetectBeforeCubingJob extends SparkApplication { ResourceDetectUtils.write(new Path(config.getJobTmpShareDir(project, jobId), ResourceDetectUtils.countDistinctSuffix()), ResourceDetectUtils.findCountDistinctMeasure(JavaConversions.asJavaCollection(seg.toBuildLayouts()))); ParentSourceChooser datasetChooser = new ParentSourceChooser(spanningTree, seg, jobId, ss, config, false); - datasetChooser.setDetectStep(); datasetChooser.decideSources(); NBuildSourceInfo buildFromFlatTable = datasetChooser.flatTableSource(); if (buildFromFlatTable != null) { diff --git a/kylin-spark-project/kylin-spark-metadata/src/main/scala/org/apache/kylin/engine/spark/metadata/MetadataConverter.scala b/kylin-spark-project/kylin-spark-metadata/src/main/scala/org/apache/kylin/engine/spark/metadata/MetadataConverter.scala index 30067ae..0d12c25 100644 --- a/kylin-spark-project/kylin-spark-metadata/src/main/scala/org/apache/kylin/engine/spark/metadata/MetadataConverter.scala +++ b/kylin-spark-project/kylin-spark-metadata/src/main/scala/org/apache/kylin/engine/spark/metadata/MetadataConverter.scala @@ -122,7 +122,7 @@ object MetadataConverter { def extractEntityAndMeasures(cubeInstance: CubeInstance): (List[LayoutEntity], Map[Integer, FunctionDesc]) = { val (columnIndexes, shardByColumnsId, idToColumnMap, measureId) = genIDToColumnMap(cubeInstance) - (cubeInstance.getDescriptor.getInitialCuboidScheduler + (cubeInstance.getCuboidScheduler .getAllCuboidIds .asScala .map { long => 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 96872b9..059915c 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 @@ -47,6 +47,7 @@ import org.apache.kylin.cube.model.HBaseColumnFamilyDesc; import org.apache.kylin.cube.model.RowKeyColDesc; import org.apache.kylin.dimension.DimensionEncoding; import org.apache.kylin.dimension.DimensionEncodingFactory; +import org.apache.kylin.engine.mr.common.CuboidStatsReaderUtil; import org.apache.kylin.job.JobInstance; import org.apache.kylin.job.JoinedFlatTable; import org.apache.kylin.job.exception.JobException; @@ -880,31 +881,31 @@ public class CubeController extends BasicController { } } -// @RequestMapping(value = "/{cubeName}/cuboids/current", method = RequestMethod.GET) -// @ResponseBody -// public CuboidTreeResponse getCurrentCuboids(@PathVariable String cubeName) { -// checkCubeExists(cubeName); -// CubeInstance cube = cubeService.getCubeManager().getCube(cubeName); -// // The cuboid tree displayed should be consistent with the current one -// CuboidScheduler cuboidScheduler = cube.getCuboidScheduler(); -// Map<Long, Long> cuboidStatsMap = cube.getCuboids(); -// if (cuboidStatsMap == null) { -// cuboidStatsMap = CuboidStatsReaderUtil.readCuboidStatsFromCube(cuboidScheduler.getAllCuboidIds(), cube); -// } -// -// Map<Long, Long> hitFrequencyMap = null; -// Map<Long, Long> queryMatchMap = null; -// try { -// hitFrequencyMap = getTargetCuboidHitFrequency(cubeName); -// queryMatchMap = cubeService.getCuboidQueryMatchCount(cubeName); -// } catch (Exception e) { -// logger.warn("Fail to query on system cube due to " + e); -// } -// -// Set<Long> currentCuboidSet = cube.getCuboidScheduler().getAllCuboidIds(); -// return cubeService.getCuboidTreeResponse(cuboidScheduler, cuboidStatsMap, hitFrequencyMap, queryMatchMap, -// currentCuboidSet); -// } + @RequestMapping(value = "/{cubeName}/cuboids/current", method = RequestMethod.GET) + @ResponseBody + public CuboidTreeResponse getCurrentCuboids(@PathVariable String cubeName) { + checkCubeExists(cubeName); + CubeInstance cube = cubeService.getCubeManager().getCube(cubeName); + // The cuboid tree displayed should be consistent with the current one + CuboidScheduler cuboidScheduler = cube.getCuboidScheduler(); + Map<Long, Long> cuboidStatsMap = cube.getCuboids(); + if (cuboidStatsMap == null) { + cuboidStatsMap = CuboidStatsReaderUtil.readCuboidStatsFromCube(cuboidScheduler.getAllCuboidIds(), cube); + } + + Map<Long, Long> hitFrequencyMap = null; + Map<Long, Long> queryMatchMap = null; + try { + hitFrequencyMap = getTargetCuboidHitFrequency(cubeName); + queryMatchMap = cubeService.getCuboidQueryMatchCount(cubeName); + } catch (Exception e) { + logger.warn("Fail to query on system cube due to " + e); + } + + Set<Long> currentCuboidSet = cube.getCuboidScheduler().getAllCuboidIds(); + return cubeService.getCuboidTreeResponse(cuboidScheduler, cuboidStatsMap, hitFrequencyMap, queryMatchMap, + currentCuboidSet); + } @RequestMapping(value = "/{cubeName}/cuboids/recommend", method = RequestMethod.GET) @ResponseBody diff --git a/webapp/app/partials/cubes/cube_detail.html b/webapp/app/partials/cubes/cube_detail.html index 0076069..253e6d7 100755 --- a/webapp/app/partials/cubes/cube_detail.html +++ b/webapp/app/partials/cubes/cube_detail.html @@ -41,7 +41,7 @@ ng-if="userService.hasRole('ROLE_ADMIN') || hasPermission('cube' ,cube, permissions.ADMINISTRATION.mask) && !newAccess"> <a href="" ng-click="cube.visiblePage='hbase';getStorageInfo(cube)">Storage</a> </li> - <li class="{{cube.visiblePage=='planner'? 'active':''}}" ng-if="(userService.hasRole('ROLE_ADMIN') || hasPermission('cube', cube, permissions.ADMINISTRATION.mask) && !newAccess) && isShowCubeplanner && cube.detail.engine_type != 6"> + <li class="{{cube.visiblePage=='planner'? 'active':''}}" ng-if="(userService.hasRole('ROLE_ADMIN') || hasPermission('cube', cube, permissions.ADMINISTRATION.mask) && !newAccess) && isShowCubeplanner && cube.detail.engine_type != 0"> <a href="" ng-click="cube.visiblePage='planner';getCubePlanner(cube);">Planner</a> </li> <li class="{{cube.visiblePage=='streaming'? 'active':''}}" ng-if="(userService.hasRole('ROLE_ADMIN') || hasPermission('cube', cube, permissions.ADMINISTRATION.mask) && !newAccess) && cube.streamingV2">