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 2e9597a099dd24ce54ad8d8b564dc3cfdf2618c9 Author: XiaoxiangYu <x...@apache.org> AuthorDate: Tue Dec 8 10:58:02 2020 +0800 KYLIN-4818 Reduce toString method call --- .../scala/org/apache/kylin/engine/spark/job/CubeBuildJob.java | 4 ++-- .../apache/kylin/engine/spark/job/CuboidStatisticsJob.scala | 10 +++++----- .../apache/kylin/engine/spark/job/ParentSourceChooser.scala | 4 ++-- 3 files changed, 9 insertions(+), 9 deletions(-) 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 9504ac8..4a0226d 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 @@ -134,8 +134,8 @@ public class CubeBuildJob extends SparkApplication { 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()); + for (Tuple2<Object, AggInfo> cuboidData : sourceChooser.aggInfo()) { + hllMap.put((Long) cuboidData._1, cuboidData._2.cuboid().counter()); } logger.info("Cuboid statistics return {} records and cost {} ms.", hllMap.size(), (System.currentTimeMillis() - startMills)); diff --git a/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CuboidStatisticsJob.scala b/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CuboidStatisticsJob.scala index 3b963e4..eb2a815 100644 --- a/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CuboidStatisticsJob.scala +++ b/kylin-spark-project/kylin-spark-engine/src/main/scala/org/apache/kylin/engine/spark/job/CuboidStatisticsJob.scala @@ -35,7 +35,7 @@ object CuboidStatisticsJob { * @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 statistics(inputDs: Dataset[Row], seg: SegmentInfo): Array[(Long, AggInfo)] = { val rkc = seg.allColumns.count(c => c.rowKey) // maybe we should use sample operation to reduce cost later @@ -48,7 +48,7 @@ object CuboidStatisticsJob { } class CuboidStatisticsJob(ids: List[Long], rkc: Int) extends Serializable { - private val info = mutable.Map[String, AggInfo]() + private val info = mutable.Map[Long, AggInfo]() private var allCuboidsBitSet: Array[Array[Integer]] = Array() private val hf: HashFunction = Hashing.murmur3_128 private val rowHashCodesLong = new Array[Long](rkc) @@ -71,7 +71,7 @@ class CuboidStatisticsJob(ids: List[Long], rkc: Int) extends Serializable { def init(): Unit = { println("CuboidStatisticsJob-Init1-" + System.currentTimeMillis()) allCuboidsBitSet = getCuboidBitSet(ids, rkc) - ids.foreach(i => info.put(i.toString, AggInfo(i.toString))) + ids.foreach(i => info.put(i, AggInfo(i))) println("CuboidStatisticsJob-Init2-" + System.currentTimeMillis()) } @@ -109,7 +109,7 @@ class CuboidStatisticsJob(ids: List[Long], rkc: Int) extends Serializable { value += rowHashCodesLong(allCuboidsBitSet(idx)(position)) position += 1 } - info(ids(idx).toString).cuboid.counter.addHashDirectly(value) + info(ids(idx)).cuboid.counter.addHashDirectly(value) idx += 1 } endMills = System.currentTimeMillis() @@ -146,7 +146,7 @@ class CuboidStatisticsJob(ids: List[Long], rkc: Int) extends Serializable { } } -case class AggInfo(key: String, +case class AggInfo(key: Long, cuboid: CuboidInfo = CuboidInfo(new HLLCounter()), sample: SampleInfo = SampleInfo(), dimension: DimensionInfo = DimensionInfo()) { 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 9b80447..216fbf8 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 @@ -39,7 +39,7 @@ class ParentSourceChooser( config: KylinConfig, needEncoding: Boolean) extends Logging { - var aggInfo : Array[(String, AggInfo)] = _ + var aggInfo : Array[(Long, AggInfo)] = _ // build from built cuboid. var reuseSources: java.util.Map[java.lang.Long, NBuildSourceInfo] = Maps.newHashMap() @@ -57,7 +57,7 @@ class ParentSourceChooser( def setNeedStatistics(): Unit = needStatistics = true - def getAggInfo : Array[(String, AggInfo)] = aggInfo + def getAggInfo : Array[(Long, AggInfo)] = aggInfo def decideSources(): Unit = { toBuildTree.getRootIndexEntities.asScala.foreach { entity =>