You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by sh...@apache.org on 2015/09/06 16:08:21 UTC
incubator-kylin git commit: KYLIN-965 Allow user to configure the
region split size for cube
Repository: incubator-kylin
Updated Branches:
refs/heads/0.7-staging 8c8765c68 -> 50e0875d5
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/50e0875d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/50e0875d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/50e0875d
Branch: refs/heads/0.7-staging
Commit: 50e0875d500cda2a5d5f54246b21b179f6a4f6c2
Parents: 8c8765c
Author: shaofengshi <sh...@apache.org>
Authored: Sun Sep 6 22:07:55 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sun Sep 6 22:07:55 2015 +0800
----------------------------------------------------------------------
.../org/apache/kylin/common/KylinConfig.java | 13 +++++++++
conf/kylin.properties | 8 ++++++
.../test_case_data/sandbox/kylin.properties | 8 ++++++
.../kylin/job/constant/BatchConstants.java | 1 +
.../hadoop/cube/RangeKeyDistributionJob.java | 6 ++--
.../cube/RangeKeyDistributionReducer.java | 29 +++++++-------------
6 files changed, 44 insertions(+), 21 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/50e0875d/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 76031c2..59337d8 100644
--- a/common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -127,6 +127,10 @@ 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";
// static cached instances
private static KylinConfig ENV_INSTANCE = null;
@@ -651,6 +655,15 @@ public class KylinConfig {
public void setRemoteHadoopCliPassword(String v) {
kylinConfig.setProperty(KYLIN_JOB_REMOTE_CLI_PASSWORD, v);
}
+
+ public int getHBaseRegionSizeGB(String capacity) {
+ String key = HBASE_REGION_SIZE + "." + capacity.toLowerCase();
+ return Integer.valueOf(getOptional(key, "10"));
+ }
+
+ 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/50e0875d/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index af860bd..98eeafc 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -71,6 +71,14 @@ 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
+
## Config for Restful APP ##
# database connection settings:
ldap.server=
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/50e0875d/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 359174d..985f65e 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -59,6 +59,14 @@ 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
+
## Config for Restful APP ##
# database connection settings:
ldap.server=
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/50e0875d/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 7776988..03eb2ef 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,7 @@ public interface BatchConstants {
String MAPPER_SAMPLE_NUMBER = "mapper.sample.number";
String REGION_NUMBER = "region.number";
+ String REGION_SPLIT_SIZE = "region.split.size";
String CUBE_CAPACITY = "cube.capacity";
String CFG_KYLIN_LOCAL_TEMP_DIR = "/tmp/kylin/";
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/50e0875d/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 190bf0c..2ca75a5 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
@@ -91,8 +91,10 @@ public class RangeKeyDistributionJob extends AbstractHadoopJob {
String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
CubeInstance cube = cubeMgr.getCube(cubeName);
- RealizationCapacity realizationCapacity = cube.getDescriptor().getModel().getCapacity();
- job.getConfiguration().set(BatchConstants.CUBE_CAPACITY, realizationCapacity.toString());
+ int regionSplitSize = KylinConfig.getInstanceFromEnv().getHBaseRegionSizeGB(cube.getDescriptor().getModel().getCapacity().toString());
+ int maxRegionCount = KylinConfig.getInstanceFromEnv().getHBaseMaxRegionCount();
+ job.getConfiguration().set(BatchConstants.REGION_SPLIT_SIZE, String.valueOf(regionSplitSize));
+ job.getConfiguration().set(BatchConstants.REGION_NUMBER, String.valueOf(maxRegionCount));
return waitForCompletion(job);
} catch (Exception e) {
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/50e0875d/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 018d952..c97ce24 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
@@ -38,17 +38,13 @@ import org.slf4j.LoggerFactory;
public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable, Text, LongWritable> {
public static final long ONE_GIGA_BYTES = 1024L * 1024L * 1024L;
- public static final int SMALL_CUT = 10; // 10 GB per region
- public static final int MEDIUM_CUT = 20; // 20 GB per region
- public static final int LARGE_CUT = 100; // 100 GB per region
-
- public static final int MAX_REGION = 500;
private static final Logger logger = LoggerFactory.getLogger(RangeKeyDistributionReducer.class);
private LongWritable outputValue = new LongWritable(0);
- private int cut;
+ private int maxRegionCount = 500;
+ private int cut = 10;
private long bytesRead = 0;
private List<Text> gbPoints = new ArrayList<Text>();
@@ -56,20 +52,15 @@ public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable
protected void setup(Context context) throws IOException {
super.publishConfiguration(context.getConfiguration());
- CubeCapacity cubeCapacity = CubeCapacity.valueOf(context.getConfiguration().get(BatchConstants.CUBE_CAPACITY));
- switch (cubeCapacity) {
- case SMALL:
- cut = SMALL_CUT;
- break;
- case MEDIUM:
- cut = MEDIUM_CUT;
- break;
- case LARGE:
- cut = LARGE_CUT;
- break;
+ if (context.getConfiguration().get(BatchConstants.REGION_SPLIT_SIZE) != null) {
+ 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));
}
- logger.info("Chosen cut for htable is " + cut);
+ logger.info("Chosen cut for htable is " + cut + ", max region count is " + maxRegionCount);
}
@Override
@@ -88,7 +79,7 @@ public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable
protected void cleanup(Context context) throws IOException, InterruptedException {
int nRegion = Math.round((float) gbPoints.size() / (float) cut);
nRegion = Math.max(1, nRegion);
- nRegion = Math.min(MAX_REGION, nRegion);
+ nRegion = Math.min(maxRegionCount, nRegion);
int gbPerRegion = gbPoints.size() / nRegion;
gbPerRegion = Math.max(1, gbPerRegion);