[41/50] [abbrv] kylin git commit: KYLIN-2246 redesign the way to decide layer cubing reducer count

2016-12-06 Thread lidong
KYLIN-2246 redesign the way to decide layer cubing reducer count


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/76999735
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/76999735
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/76999735

Branch: refs/heads/master-hbase1.x
Commit: 76999735f4157b6064eb099b59a7c79fbc9b6007
Parents: d2aaf27
Author: Hongbin Ma 
Authored: Mon Dec 5 21:02:36 2016 +0800
Committer: Hongbin Ma 
Committed: Tue Dec 6 13:50:28 2016 +0800

--
 .../kylin/cube/cuboid/CuboidScheduler.java  | 31 +++-
 .../kylin/engine/mr/common/CubeStatsReader.java | 26 ++-
 .../apache/kylin/engine/mr/steps/CuboidJob.java | 52 +
 .../engine/mr/steps/LayerReduerNumSizing.java   | 82 
 .../kylin/engine/mr/steps/MergeCuboidJob.java   |  2 +-
 5 files changed, 138 insertions(+), 55 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/76999735/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java 
b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java
index bd6a37a..733aded 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java
@@ -18,7 +18,7 @@
 
 package org.apache.kylin.cube.cuboid;
 
-/** 
+/**
  */
 
 import java.util.Collections;
@@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.kylin.cube.model.AggregationGroup;
 import org.apache.kylin.cube.model.CubeDesc;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
@@ -39,6 +40,7 @@ public class CuboidScheduler {
 private final CubeDesc cubeDesc;
 private final long max;
 private final Map cache;
+private List cuboidsByLayer;
 
 public CuboidScheduler(CubeDesc cubeDesc) {
 this.cubeDesc = cubeDesc;
@@ -232,4 +234,31 @@ public class CuboidScheduler {
 getSubCuboidIds(cuboidId, result);
 }
 }
+
+public List getCuboidsByLayer() {
+if (cuboidsByLayer != null) {
+return cuboidsByLayer;
+}
+
+int totalNum = 0;
+int layerNum = cubeDesc.getBuildLevel();
+cuboidsByLayer = Lists.newArrayList();
+
+
cuboidsByLayer.add(Collections.singletonList(Cuboid.getBaseCuboidId(cubeDesc)));
+totalNum++;
+
+for (int i = 1; i <= layerNum; i++) {
+List lastLayer = cuboidsByLayer.get(i - 1);
+List newLayer = Lists.newArrayList();
+for (Long parent : lastLayer) {
+newLayer.addAll(getSpanningCuboid(parent));
+}
+cuboidsByLayer.add(newLayer);
+totalNum += newLayer.size();
+}
+
+int size = getAllCuboidIds().size();
+Preconditions.checkState(totalNum == size, "total Num: " + totalNum + 
" actual size: " + size);
+return cuboidsByLayer;
+}
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/76999735/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
--
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
index c917cfb..1cf5da6 100644
--- 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
+++ 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
@@ -29,6 +29,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.BytesWritable;
@@ -75,9 +76,11 @@ public class CubeStatsReader {
 final int mapperNumberOfFirstBuild; // becomes meaningless after merge
 final double mapperOverlapRatioOfFirstBuild; // becomes meaningless after 
merge
 final Map cuboidRowEstimatesHLL;
+final CuboidScheduler cuboidScheduler;
 
 public CubeStatsReader(CubeSegment cubeSegment, KylinConfig kylinConfig) 
throws IOException {
 ResourceStore store = ResourceStore.getStore(kylinConfig);
+cuboidScheduler = new CuboidScheduler(cubeSegment.getCubeDesc());
 String statsKey = cubeSegment.getStatisticsResourcePath();
 File tmpSeqFile = 

kylin git commit: KYLIN-2246 redesign the way to decide layer cubing reducer count

2016-12-05 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/KYLIN-2246 [created] 815887e73


KYLIN-2246 redesign the way to decide layer cubing reducer count


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/815887e7
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/815887e7
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/815887e7

Branch: refs/heads/KYLIN-2246
Commit: 815887e73a5c3b0852b6cf5650400235797d7ce9
Parents: 59a30f6
Author: Hongbin Ma 
Authored: Mon Dec 5 21:02:36 2016 +0800
Committer: Hongbin Ma 
Committed: Mon Dec 5 21:02:44 2016 +0800

--
 .../kylin/cube/cuboid/CuboidScheduler.java  | 31 +++-
 .../kylin/engine/mr/common/CubeStatsReader.java | 26 ++-
 .../apache/kylin/engine/mr/steps/CuboidJob.java | 52 +
 .../engine/mr/steps/LayerReduerNumSizing.java   | 82 
 .../kylin/engine/mr/steps/MergeCuboidJob.java   |  2 +-
 5 files changed, 138 insertions(+), 55 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/815887e7/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java 
b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java
index bd6a37a..733aded 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java
@@ -18,7 +18,7 @@
 
 package org.apache.kylin.cube.cuboid;
 
-/** 
+/**
  */
 
 import java.util.Collections;
@@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.kylin.cube.model.AggregationGroup;
 import org.apache.kylin.cube.model.CubeDesc;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
@@ -39,6 +40,7 @@ public class CuboidScheduler {
 private final CubeDesc cubeDesc;
 private final long max;
 private final Map cache;
+private List cuboidsByLayer;
 
 public CuboidScheduler(CubeDesc cubeDesc) {
 this.cubeDesc = cubeDesc;
@@ -232,4 +234,31 @@ public class CuboidScheduler {
 getSubCuboidIds(cuboidId, result);
 }
 }
+
+public List getCuboidsByLayer() {
+if (cuboidsByLayer != null) {
+return cuboidsByLayer;
+}
+
+int totalNum = 0;
+int layerNum = cubeDesc.getBuildLevel();
+cuboidsByLayer = Lists.newArrayList();
+
+
cuboidsByLayer.add(Collections.singletonList(Cuboid.getBaseCuboidId(cubeDesc)));
+totalNum++;
+
+for (int i = 1; i <= layerNum; i++) {
+List lastLayer = cuboidsByLayer.get(i - 1);
+List newLayer = Lists.newArrayList();
+for (Long parent : lastLayer) {
+newLayer.addAll(getSpanningCuboid(parent));
+}
+cuboidsByLayer.add(newLayer);
+totalNum += newLayer.size();
+}
+
+int size = getAllCuboidIds().size();
+Preconditions.checkState(totalNum == size, "total Num: " + totalNum + 
" actual size: " + size);
+return cuboidsByLayer;
+}
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/815887e7/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
--
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
index c917cfb..1cf5da6 100644
--- 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
+++ 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
@@ -29,6 +29,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.BytesWritable;
@@ -75,9 +76,11 @@ public class CubeStatsReader {
 final int mapperNumberOfFirstBuild; // becomes meaningless after merge
 final double mapperOverlapRatioOfFirstBuild; // becomes meaningless after 
merge
 final Map cuboidRowEstimatesHLL;
+final CuboidScheduler cuboidScheduler;
 
 public CubeStatsReader(CubeSegment cubeSegment, KylinConfig kylinConfig) 
throws IOException {
 ResourceStore store = ResourceStore.getStore(kylinConfig);
+cuboidScheduler = new CuboidScheduler(cubeSegment.getCubeDesc());
 String statsKey = cubeSegment.getStatisticsResourcePath();