You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2015/09/19 01:47:02 UTC
[26/50] [abbrv] incubator-kylin git commit: KYLIN-965 Allow user to
configure the region split size for cube
KYLIN-965 Allow user to configure the region split size for cube
Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/fa925f70
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/fa925f70
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/fa925f70
Branch: refs/heads/master
Commit: fa925f70cb2ff8ecdcd3287784a637d60e672b3c
Parents: 60a95f3
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 7 16:09:46 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 7 16:11:39 2015 +0800
----------------------------------------------------------------------
.../org/apache/kylin/common/KylinConfig.java | 52 ++++++++++++--------
conf/kylin.properties | 16 +++---
.../test_case_data/sandbox/kylin.properties | 16 +++---
.../kylin/job/constant/BatchConstants.java | 2 +
.../hadoop/cube/RangeKeyDistributionJob.java | 10 ++--
.../cube/RangeKeyDistributionReducer.java | 13 +++--
6 files changed, 67 insertions(+), 42 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fa925f70/common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/KylinConfig.java b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
index 5566fa2..f1a8e92 100644
--- a/common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -127,11 +127,14 @@ public class KylinConfig {
public static final String VERSION = "${project.version}";
public static final String HTABLE_DEFAULT_COMPRESSION_CODEC = "kylin.hbase.default.compression.codec";
-
- public static final String HBASE_REGION_SIZE = "kylin.hbase.region.size";
-
- public static final String HBASE_REGION_MAX_COUNT = "kylin.hbase.region.max.count";
+ public static final String HBASE_REGION_CUT_SMALL = "kylin.hbase.region.cut.small";
+ public static final String HBASE_REGION_CUT_MEDIUM = "kylin.hbase.region.cut.medium";
+ public static final String HBASE_REGION_CUT_LARGE = "kylin.hbase.region.cut.large";
+
+ public static final String HBASE_REGION_COUNT_MIN = "kylin.hbase.region.count.min";
+ public static final String HBASE_REGION_COUNT_MAX = "kylin.hbase.region.count.max";
+
// static cached instances
private static KylinConfig ENV_INSTANCE = null;
@@ -655,27 +658,34 @@ public class KylinConfig {
public void setRemoteHadoopCliPassword(String v) {
kylinConfig.setProperty(KYLIN_JOB_REMOTE_CLI_PASSWORD, v);
}
+
+ public int getHBaseRegionCountMin() {
+ return Integer.parseInt(getOptional(HBASE_REGION_COUNT_MIN, "1"));
+ }
+
+ public int getHBaseRegionCountMax() {
+ return Integer.parseInt(getOptional(HBASE_REGION_COUNT_MAX, "500"));
+ }
- public int getHBaseRegionSizeGB(String capacity) {
- String key = HBASE_REGION_SIZE + "." + capacity.toLowerCase();
-
- int cut = 20;
- if (kylinConfig.containsKey(key)) {
- cut = kylinConfig.getInt(key);
- } else if ("small".equalsIgnoreCase(capacity)) {
- cut = 10;
- } else if ("medium".equalsIgnoreCase(capacity)) {
- cut = 20;
- } else if ("large".equalsIgnoreCase(capacity)) {
- cut = 100;
+ public int getHBaseRegionCut(String capacity) {
+ String cut;
+ switch (capacity) {
+ case "SMALL":
+ cut = getProperty(HBASE_REGION_CUT_SMALL, "10");
+ break;
+ case "MEDIUM":
+ cut = getProperty(HBASE_REGION_CUT_MEDIUM, "20");
+ break;
+ case "LARGE":
+ cut = getProperty(HBASE_REGION_CUT_LARGE, "100");
+ break;
+ default:
+ throw new IllegalArgumentException("Capacity not recognized: " + capacity);
}
- return cut;
+
+ return Integer.valueOf(cut);
}
- public int getHBaseMaxRegionCount() {
- return Integer.valueOf(getOptional(HBASE_REGION_MAX_COUNT, "500"));
- }
-
public String getProperty(String key, String defaultValue) {
return kylinConfig.getString(key, defaultValue);
}
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fa925f70/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index 0d1827e..0aa2898 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -71,13 +71,15 @@ kylin.job.hive.database.for.intermediatetable=default
#default compression codec for htable,snappy,lzo,gzip,lz4
kylin.hbase.default.compression.codec=snappy
-# Region cut size for SMALL, MEDIUM and LARGE cubes, in GB
-kylin.hbase.region.size.small=10
-kylin.hbase.region.size.medium=20
-kylin.hbase.region.size.large=100
-
-# Max region count when create HTable
-kylin.hbase.region.max.count=500
+# The cut size for hbase region, in GB.
+# E.g, for cube whose capacity be marked as "SMALL", split region per 10GB by default
+kylin.hbase.region.cut.small=10
+kylin.hbase.region.cut.medium=20
+kylin.hbase.region.cut.large=100
+
+# HBase min and max region count
+kylin.hbase.region.count.min=1
+kylin.hbase.region.count.min=500
## Config for Restful APP ##
# database connection settings:
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fa925f70/examples/test_case_data/sandbox/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin.properties b/examples/test_case_data/sandbox/kylin.properties
index 985f65e..b3ec4d0 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -59,13 +59,15 @@ kylin.job.hive.database.for.intermediatetable=default
#default compression codec for htable,snappy,lzo,gzip,lz4
kylin.hbase.default.compression.codec=gzip
-# Region cut size for SMALL, MEDIUM and LARGE cubes, in GB
-kylin.hbase.region.size.small=2
-kylin.hbase.region.size.medium=5
-kylin.hbase.region.size.large=10
-
-# Max region count when create HTable
-kylin.hbase.region.max.count=10
+# The cut size for hbase region, in GB.
+# E.g, for cube whose capacity be marked as "SMALL", split region per 10GB by default
+kylin.hbase.region.cut.small=10
+kylin.hbase.region.cut.medium=20
+kylin.hbase.region.cut.large=100
+
+# HBase min and max region count
+kylin.hbase.region.count.min=1
+kylin.hbase.region.count.min=500
## Config for Restful APP ##
# database connection settings:
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fa925f70/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java b/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
index 03eb2ef..ca8b2c9 100644
--- a/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
+++ b/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
@@ -43,6 +43,8 @@ public interface BatchConstants {
String MAPPER_SAMPLE_NUMBER = "mapper.sample.number";
String REGION_NUMBER = "region.number";
+ String REGION_NUMBER_MIN = "region.number.min";
+ String REGION_NUMBER_MAX = "region.number.max";
String REGION_SPLIT_SIZE = "region.split.size";
String CUBE_CAPACITY = "cube.capacity";
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fa925f70/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
index 2ca75a5..9c50122 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
@@ -32,6 +32,7 @@ import org.apache.kylin.cube.CubeInstance;
import org.apache.kylin.cube.CubeManager;
import org.apache.kylin.job.constant.BatchConstants;
import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.metadata.model.DataModelDesc;
import org.apache.kylin.metadata.model.DataModelDesc.RealizationCapacity;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -91,10 +92,13 @@ public class RangeKeyDistributionJob extends AbstractHadoopJob {
String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
CubeInstance cube = cubeMgr.getCube(cubeName);
- int regionSplitSize = KylinConfig.getInstanceFromEnv().getHBaseRegionSizeGB(cube.getDescriptor().getModel().getCapacity().toString());
- int maxRegionCount = KylinConfig.getInstanceFromEnv().getHBaseMaxRegionCount();
+ DataModelDesc.RealizationCapacity cubeCapacity = cube.getDescriptor().getModel().getCapacity();
+ int regionSplitSize = KylinConfig.getInstanceFromEnv().getHBaseRegionCut(cubeCapacity.toString());
+ int maxRegionCount = KylinConfig.getInstanceFromEnv().getHBaseRegionCountMax();
+ int minRegionCount = KylinConfig.getInstanceFromEnv().getHBaseRegionCountMin();
job.getConfiguration().set(BatchConstants.REGION_SPLIT_SIZE, String.valueOf(regionSplitSize));
- job.getConfiguration().set(BatchConstants.REGION_NUMBER, String.valueOf(maxRegionCount));
+ job.getConfiguration().set(BatchConstants.REGION_NUMBER_MAX, String.valueOf(maxRegionCount));
+ job.getConfiguration().set(BatchConstants.REGION_NUMBER_MIN, String.valueOf(minRegionCount));
return waitForCompletion(job);
} catch (Exception e) {
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fa925f70/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
index c97ce24..b3ab4db 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
@@ -43,6 +43,7 @@ public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable
private LongWritable outputValue = new LongWritable(0);
+ private int minRegionCount = 1;
private int maxRegionCount = 500;
private int cut = 10;
private long bytesRead = 0;
@@ -56,11 +57,15 @@ public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable
cut = Integer.valueOf(context.getConfiguration().get(BatchConstants.REGION_SPLIT_SIZE));
}
- if (context.getConfiguration().get(BatchConstants.REGION_NUMBER) != null) {
- maxRegionCount = Integer.valueOf(context.getConfiguration().get(BatchConstants.REGION_NUMBER));
+ if (context.getConfiguration().get(BatchConstants.REGION_NUMBER_MIN) != null) {
+ minRegionCount = Integer.valueOf(context.getConfiguration().get(BatchConstants.REGION_NUMBER_MIN));
+ }
+
+ if (context.getConfiguration().get(BatchConstants.REGION_NUMBER_MAX) != null) {
+ maxRegionCount = Integer.valueOf(context.getConfiguration().get(BatchConstants.REGION_NUMBER_MAX));
}
- logger.info("Chosen cut for htable is " + cut + ", max region count is " + maxRegionCount);
+ logger.info("Chosen cut for htable is " + cut + ", max region count=" + maxRegionCount + ", min region count =" + minRegionCount);
}
@Override
@@ -78,7 +83,7 @@ public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable
@Override
protected void cleanup(Context context) throws IOException, InterruptedException {
int nRegion = Math.round((float) gbPoints.size() / (float) cut);
- nRegion = Math.max(1, nRegion);
+ nRegion = Math.max(minRegionCount, nRegion);
nRegion = Math.min(maxRegionCount, nRegion);
int gbPerRegion = gbPoints.size() / nRegion;