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/25 07:36:15 UTC

kylin git commit: enable query level timeout backdoor toggle and correct ExpectedSizeIterator wait timeout

Repository: kylin
Updated Branches:
  refs/heads/master e1c2b4f53 -> 85d9dff37


enable query level timeout backdoor toggle and correct ExpectedSizeIterator wait timeout


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

Branch: refs/heads/master
Commit: 85d9dff37f5363fa3a1be6ee64324b88610f0fc4
Parents: e1c2b4f
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Apr 25 13:35:56 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Mon Apr 25 13:35:56 2016 +0800

----------------------------------------------------------------------
 .../kylin/common/debug/BackdoorToggles.java     | 18 ++++++++++++++++++
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java     | 20 +++++++++++++++++---
 2 files changed, 35 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/85d9dff3/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java b/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java
index f3745d7..0feff2d 100644
--- a/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java
+++ b/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java
@@ -54,6 +54,14 @@ public class BackdoorToggles {
         return getBoolean(DEBUG_TOGGLE_LOCAL_COPROCESSOR);
     }
 
+    public static int getQueryTimeout() {
+        String v = getString(DEBUG_TOGGLE_QUERY_TIMEOUT);
+        if (v == null)
+            return -1;
+        else
+            return Integer.valueOf(v);
+    }
+
     private static String getString(String key) {
         Map<String, String> toggles = _backdoorToggles.get();
         if (toggles == null) {
@@ -135,4 +143,14 @@ public class BackdoorToggles {
      }
      */
     public final static String DEBUG_TOGGLE_LOCAL_COPROCESSOR = "DEBUG_TOGGLE_LOCAL_COPROCESSOR";
+
+    /**
+     * set DEBUG_TOGGLE_QUERY_TIMEOUT="timeout_millis" to overwrite the global timeout settings
+     *
+     example:(put it into request body)
+     "backdoorToggles": {
+     "DEBUG_TOGGLE_QUERY_TIMEOUT": "120000"
+     }
+     */
+    public final static String DEBUG_TOGGLE_QUERY_TIMEOUT = "DEBUG_TOGGLE_QUERY_TIMEOUT";
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/85d9dff3/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 88c17ae..525625b 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
@@ -114,19 +114,28 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
 
     static class ExpectedSizeIterator implements Iterator<byte[]> {
 
+        BlockingQueue<byte[]> queue;
+
         int expectedSize;
         int current = 0;
-        BlockingQueue<byte[]> queue;
         long timeout;
+        long timeoutTS;
 
         public ExpectedSizeIterator(int expectedSize) {
             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();
-
             this.timeout *= 1.1;//allow for some delay 
+
+            if (BackdoorToggles.getQueryTimeout() != -1) {
+                this.timeout = BackdoorToggles.getQueryTimeout();
+            }
+
             logger.info("Timeout for ExpectedSizeIterator is: " + this.timeout);
+
+            this.timeoutTS = System.currentTimeMillis() + this.timeout;
         }
 
         @Override
@@ -141,7 +150,12 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
             }
             try {
                 current++;
-                byte[] ret = queue.poll(timeout, TimeUnit.MILLISECONDS);
+                long tsRemaining = this.timeoutTS - System.currentTimeMillis();
+                if (tsRemaining < 0) {
+                    throw new RuntimeException("Timeout visiting cube!");
+                }
+
+                byte[] ret = queue.poll(tsRemaining, TimeUnit.MILLISECONDS);
                 if (ret == null) {
                     throw new RuntimeException("Timeout visiting cube!");
                 } else {