You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by ma...@apache.org on 2016/04/01 08:49:02 UTC
[2/4] kylin git commit: KYLIN-1308 allow to multiply cube visit
timeout by a factor
KYLIN-1308 allow to multiply cube visit timeout by a factor
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/5dd133f6
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/5dd133f6
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/5dd133f6
Branch: refs/heads/master
Commit: 5dd133f6d05b8e8c65c67d557966ebcdd88a6093
Parents: 27f7917
Author: Hongbin Ma <ma...@apache.org>
Authored: Fri Apr 1 11:15:37 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Fri Apr 1 14:45:57 2016 +0800
----------------------------------------------------------------------
.../org/apache/kylin/common/KylinConfigBase.java | 16 ++++++++--------
.../storage/hbase/cube/v2/CubeHBaseEndpointRPC.java | 2 ++
2 files changed, 10 insertions(+), 8 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/5dd133f6/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index e97051b..3d9a338 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -399,13 +399,13 @@ abstract public class KylinConfigBase implements Serializable {
String cut;
switch (capacity) {
case "SMALL":
- cut = getOptional("kylin.hbase.region.cut.small", "5");
+ cut = getOptional("kylin.hbase.region.cut.small", "10");
break;
case "MEDIUM":
- cut = getOptional("kylin.hbase.region.cut.medium", "10");
+ cut = getOptional("kylin.hbase.region.cut.medium", "20");
break;
case "LARGE":
- cut = getOptional("kylin.hbase.region.cut.large", "50");
+ cut = getOptional("kylin.hbase.region.cut.large", "100");
break;
default:
throw new IllegalArgumentException("Capacity not recognized: " + capacity);
@@ -433,6 +433,10 @@ abstract public class KylinConfigBase implements Serializable {
public int getScanThreshold() {
return Integer.parseInt(getOptional("kylin.query.scan.threshold", "10000000"));
}
+
+ public int getCubeVisitTimeoutTimes() {
+ return Integer.parseInt(getOptional("kylin.query.cube.visit.timeout.times", "1"));
+ }
public int getBadQueryStackTraceDepth() {
return Integer.parseInt(getOptional("kylin.query.badquery.stacktrace.depth", "10"));
@@ -443,11 +447,7 @@ abstract public class KylinConfigBase implements Serializable {
}
public int getBadQueryDefaultAlertingSeconds() {
- return Integer.parseInt(getOptional("kylin.query.badquery.alerting.seconds", "90"));
- }
-
- public int getBadQueryDefaultDetectIntervalSeconds() {
- return Integer.parseInt(getOptional("kylin.query.badquery.detect.interval.seconds", "60"));
+ return Integer.parseInt(getOptional("kylin.query.badquery.default.alerting.seconds", "90"));
}
public int getCachedDictMaxEntrySize() {
http://git-wip-us.apache.org/repos/asf/kylin/blob/5dd133f6/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
index 2138bcc..f36ab29 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
+import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.debug.BackdoorToggles;
import org.apache.kylin.common.util.Bytes;
import org.apache.kylin.common.util.BytesSerializer;
@@ -120,6 +121,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
this.expectedSize = expectedSize;
this.queue = new ArrayBlockingQueue<byte[]>(expectedSize);
this.timeout = HadoopUtil.getCurrentConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
+ this.timeout *= KylinConfig.getInstanceFromEnv().getCubeVisitTimeoutTimes();
logger.info("Timeout for ExpectedSizeIterator is " + this.timeout);
}