[06/11] kylin git commit: KYLIN-2451 rework to respect cube-level coprocessor-timeout-seconds

2017-02-20 Thread lidong
KYLIN-2451 rework to respect cube-level coprocessor-timeout-seconds


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

Branch: refs/heads/master-hbase0.98
Commit: d7ecd6e2f1a87bf82d3cf6645eab013b873dde8d
Parents: c52b698
Author: Li Yang 
Authored: Mon Feb 20 17:21:21 2017 +0800
Committer: Li Yang 
Committed: Mon Feb 20 17:21:21 2017 +0800

--
 .../kylin/storage/hbase/HBaseConnection.java|  6 
 .../storage/hbase/cube/v2/CubeHBaseRPC.java | 37 
 2 files changed, 22 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/d7ecd6e2/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
--
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
index e3d2308..73f31c5 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
@@ -163,11 +162,6 @@ public class HBaseConnection {
 conf.set("hbase.fs.tmp.dir", "/tmp");
 }
 
-// set RPC timeout
-if (kylinConf.getQueryCoprocessorTimeoutSeconds() > 0) {
-conf.set(HConstants.HBASE_RPC_TIMEOUT_KEY, "" + (1000 * 
kylinConf.getQueryCoprocessorTimeoutSeconds()));
-}
-
 return conf;
 }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d7ecd6e2/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
--
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
index f24290c..88e7176 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
@@ -35,7 +35,6 @@ import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.kv.FuzzyKeyEncoder;
 import org.apache.kylin.cube.kv.FuzzyMaskEncoder;
@@ -49,6 +48,7 @@ import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.gridtable.GTRecord;
 import org.apache.kylin.gridtable.GTScanRange;
 import org.apache.kylin.gridtable.IGTStorage;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -283,23 +283,30 @@ public abstract class CubeHBaseRPC implements IGTStorage {
 }
 
 protected int getCoprocessorTimeoutMillis() {
-int configTimeout = 
cubeSeg.getConfig().getQueryCoprocessorTimeoutSeconds() * 1000;
-if (configTimeout == 0) {
-configTimeout = Integer.MAX_VALUE;
+int coopTimeout;
+if (BackdoorToggles.getQueryTimeout() != -1) {
+coopTimeout = BackdoorToggles.getQueryTimeout();
+} else {
+coopTimeout = 
cubeSeg.getConfig().getQueryCoprocessorTimeoutSeconds() * 1000;
 }
-
+
+int rpcTimeout;
 Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
-int rpcTimeout = hconf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 
HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
-// final timeout should be smaller than rpc timeout
-int upper = (int) (rpcTimeout * 0.9);
-
-int timeout = Math.min(upper, configTimeout);
-if (BackdoorToggles.getQueryTimeout() != -1) {
-timeout = Math.min(upper, BackdoorToggles.getQueryTimeout());
+rpcTimeout = hconf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 
HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
+
+// HBase rpc timeout must be longer than coprocessor timeout
+if ((int) (coopTimeout * 1.1) > rpcTimeout) {
+  

kylin git commit: KYLIN-2451 rework to respect cube-level coprocessor-timeout-seconds

2017-02-20 Thread liyang
Repository: kylin
Updated Branches:
  refs/heads/master c52b6981f -> d7ecd6e2f


KYLIN-2451 rework to respect cube-level coprocessor-timeout-seconds


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

Branch: refs/heads/master
Commit: d7ecd6e2f1a87bf82d3cf6645eab013b873dde8d
Parents: c52b698
Author: Li Yang 
Authored: Mon Feb 20 17:21:21 2017 +0800
Committer: Li Yang 
Committed: Mon Feb 20 17:21:21 2017 +0800

--
 .../kylin/storage/hbase/HBaseConnection.java|  6 
 .../storage/hbase/cube/v2/CubeHBaseRPC.java | 37 
 2 files changed, 22 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/d7ecd6e2/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
--
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
index e3d2308..73f31c5 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
@@ -163,11 +162,6 @@ public class HBaseConnection {
 conf.set("hbase.fs.tmp.dir", "/tmp");
 }
 
-// set RPC timeout
-if (kylinConf.getQueryCoprocessorTimeoutSeconds() > 0) {
-conf.set(HConstants.HBASE_RPC_TIMEOUT_KEY, "" + (1000 * 
kylinConf.getQueryCoprocessorTimeoutSeconds()));
-}
-
 return conf;
 }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d7ecd6e2/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
--
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
index f24290c..88e7176 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
@@ -35,7 +35,6 @@ import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.kv.FuzzyKeyEncoder;
 import org.apache.kylin.cube.kv.FuzzyMaskEncoder;
@@ -49,6 +48,7 @@ import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.gridtable.GTRecord;
 import org.apache.kylin.gridtable.GTScanRange;
 import org.apache.kylin.gridtable.IGTStorage;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -283,23 +283,30 @@ public abstract class CubeHBaseRPC implements IGTStorage {
 }
 
 protected int getCoprocessorTimeoutMillis() {
-int configTimeout = 
cubeSeg.getConfig().getQueryCoprocessorTimeoutSeconds() * 1000;
-if (configTimeout == 0) {
-configTimeout = Integer.MAX_VALUE;
+int coopTimeout;
+if (BackdoorToggles.getQueryTimeout() != -1) {
+coopTimeout = BackdoorToggles.getQueryTimeout();
+} else {
+coopTimeout = 
cubeSeg.getConfig().getQueryCoprocessorTimeoutSeconds() * 1000;
 }
-
+
+int rpcTimeout;
 Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
-int rpcTimeout = hconf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 
HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
-// final timeout should be smaller than rpc timeout
-int upper = (int) (rpcTimeout * 0.9);
-
-int timeout = Math.min(upper, configTimeout);
-if (BackdoorToggles.getQueryTimeout() != -1) {
-timeout = Math.min(upper, BackdoorToggles.getQueryTimeout());
+rpcTimeout = hconf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 
HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
+
+// HBase rpc timeout must be longer than