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 2017/02/12 14:31:25 UTC

[01/13] kylin git commit: KYLIN-2441 protocol for REST API result format [Forced Update!]

Repository: kylin
Updated Branches:
  refs/heads/master-hbase0.98 d6a844bb1 -> 0dc56aa4a (forced update)


KYLIN-2441 protocol for REST API result format


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

Branch: refs/heads/master-hbase0.98
Commit: 41332f3ed430b095efe5edefc831f0ec27f4b2ee
Parents: f2e8b69
Author: Hongbin Ma <ma...@apache.org>
Authored: Fri Feb 10 12:34:49 2017 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Fri Feb 10 12:35:14 2017 +0800

----------------------------------------------------------------------
 .../rest/controller/EncodingController.java     |  3 +-
 .../rest/exception/BadRequestException.java     | 37 +++++++++-----------
 .../kylin/rest/response/EnvelopeResponse.java   | 34 ++++--------------
 .../kylin/rest/response/ErrorResponse.java      | 32 ++++++++++++-----
 .../kylin/rest/response/ResponseCode.java       | 30 ++++++++++++++++
 5 files changed, 79 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/41332f3e/server-base/src/main/java/org/apache/kylin/rest/controller/EncodingController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/EncodingController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/EncodingController.java
index b95394c..4a8b122 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/EncodingController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/EncodingController.java
@@ -24,6 +24,7 @@ import java.util.Set;
 
 import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.rest.response.EnvelopeResponse;
+import org.apache.kylin.rest.response.ResponseCode;
 import org.apache.kylin.rest.service.EncodingService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -65,6 +66,6 @@ public class EncodingController extends BasicController {
             datatypeValidEncodings.put(dataTypeStr, encodingService.getValidEncodings(DataType.getType(dataTypeStr)));
         }
 
-        return new EnvelopeResponse(EnvelopeResponse.CODE_SUCCESS, datatypeValidEncodings, "");
+        return new EnvelopeResponse(ResponseCode.CODE_SUCCESS, datatypeValidEncodings, "");
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/41332f3e/server-base/src/main/java/org/apache/kylin/rest/exception/BadRequestException.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/exception/BadRequestException.java b/server-base/src/main/java/org/apache/kylin/rest/exception/BadRequestException.java
index 42b671b..af1995b 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/exception/BadRequestException.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/exception/BadRequestException.java
@@ -18,6 +18,7 @@
 
 package org.apache.kylin.rest.exception;
 
+import org.apache.kylin.rest.response.ResponseCode;
 import org.springframework.http.HttpStatus;
 import org.springframework.web.bind.annotation.ResponseStatus;
 
@@ -27,35 +28,31 @@ import org.springframework.web.bind.annotation.ResponseStatus;
  */
 @ResponseStatus(value = HttpStatus.BAD_REQUEST)
 public class BadRequestException extends RuntimeException {
-    /**
-     * 
-     */
+    
     private static final long serialVersionUID = -6798154278095441848L;
 
-    public BadRequestException(String s) {
-        super(s);
-    }
+    private String code;
 
     /**
-     * 
+     * legacy support, new APIs should not call this. Instead, new APIs should provide return code
      */
-    public BadRequestException() {
-        super();
+    public BadRequestException(String msg) {
+        super(msg);
+        this.code = ResponseCode.CODE_UNDEFINED;
     }
 
-    /**
-     * @param arg0
-     * @param arg1
-     */
-    public BadRequestException(String arg0, Throwable arg1) {
-        super(arg0, arg1);
+    public BadRequestException(String msg, String code) {
+        super(msg);
+        this.code = code;
     }
 
-    /**
-     * @param arg0
-     */
-    public BadRequestException(Throwable arg0) {
-        super(arg0);
+    public BadRequestException(String msg, String code, Throwable cause) {
+        super(msg, cause);
+        this.code = code;
+    }
+
+    public String getCode() {
+        return code;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/41332f3e/server-base/src/main/java/org/apache/kylin/rest/response/EnvelopeResponse.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/response/EnvelopeResponse.java b/server-base/src/main/java/org/apache/kylin/rest/response/EnvelopeResponse.java
index 564db70..7855dee 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/response/EnvelopeResponse.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/response/EnvelopeResponse.java
@@ -20,39 +20,17 @@ package org.apache.kylin.rest.response;
 
 public class EnvelopeResponse {
 
-    public final static String CODE_SUCCESS = "000";
+    public String code;
+    public Object data;
+    public String msg;
 
-    private String code;
-    private Object data;
-    private String msg;
-
-    public EnvelopeResponse(String code, Object data, String msg) {
-        this.code = code;
-        this.data = data;
-        this.msg = msg;
-    }
-
-    public String getCode() {
-        return code;
+    //only for child
+    protected EnvelopeResponse() {
     }
 
-    public void setCode(String code) {
+    public EnvelopeResponse(String code, Object data, String msg) {
         this.code = code;
-    }
-
-    public Object getData() {
-        return data;
-    }
-
-    public void setData(Object data) {
         this.data = data;
-    }
-
-    public String getMsg() {
-        return msg;
-    }
-
-    public void setMsg(String msg) {
         this.msg = msg;
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/41332f3e/server-base/src/main/java/org/apache/kylin/rest/response/ErrorResponse.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/response/ErrorResponse.java b/server-base/src/main/java/org/apache/kylin/rest/response/ErrorResponse.java
index 3327cc0..508a35f 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/response/ErrorResponse.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/response/ErrorResponse.java
@@ -18,21 +18,37 @@
 
 package org.apache.kylin.rest.response;
 
+import org.apache.kylin.rest.exception.BadRequestException;
+
+import com.google.common.base.Throwables;
+
 /**
- * @author xduo
- * 
+ * response to client when the return HTTP code is not 200
  */
-public class ErrorResponse {
+public class ErrorResponse extends EnvelopeResponse {
 
-    public String url;
+    //stacktrace of the exception
+    public String stacktrace;
+
+    //same as EnvelopeResponse.msg, kept for legacy reasons
     public String exception;
 
-    /**
-     * @param exception
-     */
+    //request URL, kept from legacy codes
+    public String url;
+
     public ErrorResponse(String url, Exception exception) {
+        super();
+        
         this.url = url;
         this.exception = exception.getLocalizedMessage();
-    }
+        this.msg = exception.getLocalizedMessage();
+        this.stacktrace = Throwables.getStackTraceAsString(exception);
+        this.data = null;
 
+        if (exception instanceof BadRequestException) {
+            this.code = ((BadRequestException) exception).getCode();
+        } else {
+            this.code = ResponseCode.CODE_UNDEFINED;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/41332f3e/server-base/src/main/java/org/apache/kylin/rest/response/ResponseCode.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/response/ResponseCode.java b/server-base/src/main/java/org/apache/kylin/rest/response/ResponseCode.java
new file mode 100644
index 0000000..8c3860a
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/response/ResponseCode.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.rest.response;
+
+/**
+ * It's not HTTP return code!
+ * It represents code for business states
+ * each API can specify different semantics to each code
+ * 000 and 999 are reserved
+ */
+public class ResponseCode {
+    public final static String CODE_SUCCESS = "000";
+    public final static String CODE_UNDEFINED = "999";
+}


[03/13] kylin git commit: KYLIN-2437 collect number of bytes scanned to query metrics

Posted by li...@apache.org.
KYLIN-2437 collect number of bytes scanned to query metrics


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

Branch: refs/heads/master-hbase0.98
Commit: e09338b34c0b07a7167096e45bf9185aa0d0cbd5
Parents: ecf6a69
Author: gaodayue <ga...@meituan.com>
Authored: Wed Feb 8 13:59:31 2017 +0800
Committer: gaodayue <ga...@meituan.com>
Committed: Fri Feb 10 15:39:11 2017 +0800

----------------------------------------------------------------------
 .../kylin/gridtable/GTAggregateScanner.java     |  11 +-
 .../apache/kylin/gridtable/GTScanRequest.java   |  10 +-
 .../apache/kylin/gridtable/ScannerWorker.java   |  64 ---------
 .../apache/kylin/storage/StorageContext.java    |   9 ++
 .../storage/gtrecord/CubeSegmentScanner.java    |   3 +-
 .../kylin/storage/gtrecord/ScannerWorker.java   |  71 ++++++++++
 .../apache/kylin/rest/response/SQLResponse.java |  10 ++
 .../apache/kylin/rest/service/QueryService.java |   9 +-
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java     |   7 +-
 .../storage/hbase/cube/v2/CubeHBaseRPC.java     |   6 +-
 .../storage/hbase/cube/v2/CubeHBaseScanRPC.java |  17 ++-
 .../coprocessor/endpoint/CubeVisitService.java  | 142 +++++++++++--------
 .../endpoint/generated/CubeVisitProtos.java     | 107 ++++++++++++--
 .../endpoint/protobuf/CubeVisit.proto           |   1 +
 14 files changed, 309 insertions(+), 158 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
index 147dbc1..dd359f8 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
@@ -65,7 +65,6 @@ public class GTAggregateScanner implements IGTScanner {
     final AggregationCache aggrCache;
     final long spillThreshold; // 0 means no memory control && no spill
     final int storagePushDownLimit;//default to be Int.MAX
-    final long deadline;
     final boolean spillEnabled;
 
     private int aggregatedRowCount = 0;
@@ -73,10 +72,10 @@ public class GTAggregateScanner implements IGTScanner {
     private boolean[] aggrMask;
 
     public GTAggregateScanner(IGTScanner inputScanner, GTScanRequest req) {
-        this(inputScanner, req, Long.MAX_VALUE, true);
+        this(inputScanner, req, true);
     }
 
-    public GTAggregateScanner(IGTScanner inputScanner, GTScanRequest req, long deadline, boolean spillEnabled) {
+    public GTAggregateScanner(IGTScanner inputScanner, GTScanRequest req, boolean spillEnabled) {
         if (!req.hasAggregation())
             throw new IllegalStateException();
 
@@ -90,7 +89,6 @@ public class GTAggregateScanner implements IGTScanner {
         this.spillThreshold = (long) (req.getAggCacheMemThreshold() * MemoryBudgetController.ONE_GB);
         this.aggrMask = new boolean[metricsAggrFuncs.length];
         this.storagePushDownLimit = req.getStoragePushDownLimit();
-        this.deadline = deadline;
         this.spillEnabled = spillEnabled;
 
         Arrays.fill(aggrMask, true);
@@ -145,11 +143,6 @@ public class GTAggregateScanner implements IGTScanner {
         long count = 0;
         for (GTRecord r : inputScanner) {
 
-            //check deadline
-            if (count % GTScanRequest.terminateCheckInterval == 1 && System.currentTimeMillis() > deadline) {
-                throw new GTScanTimeoutException("Timeout in GTAggregateScanner with scanned count " + count);
-            }
-
             if (getNumOfSpills() == 0) {
                 //check limit
                 boolean ret = aggrCache.aggregate(r, storagePushDownLimit);

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java
index 7c94f5a..651e5c4 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java
@@ -156,7 +156,7 @@ public class GTScanRequest {
     }
 
     public IGTScanner decorateScanner(IGTScanner scanner) throws IOException {
-        return decorateScanner(scanner, true, true, Long.MAX_VALUE);
+        return decorateScanner(scanner, true, true);
     }
 
     /**
@@ -165,14 +165,14 @@ public class GTScanRequest {
      * 
      * Refer to CoprocessorBehavior for explanation
      */
-    public IGTScanner decorateScanner(IGTScanner scanner, boolean filterToggledOn, boolean aggrToggledOn, long deadline) throws IOException {
-        return decorateScanner(scanner, filterToggledOn, aggrToggledOn, false, deadline, true);
+    public IGTScanner decorateScanner(IGTScanner scanner, boolean filterToggledOn, boolean aggrToggledOn) throws IOException {
+        return decorateScanner(scanner, filterToggledOn, aggrToggledOn, false, true);
     }
 
     /**
      * hasPreFiltered indicate the data has been filtered before scanning
      */
-    public IGTScanner decorateScanner(IGTScanner scanner, boolean filterToggledOn, boolean aggrToggledOn, boolean hasPreFiltered, long deadline, boolean spillEnabled) throws IOException {
+    public IGTScanner decorateScanner(IGTScanner scanner, boolean filterToggledOn, boolean aggrToggledOn, boolean hasPreFiltered, boolean spillEnabled) throws IOException {
         IGTScanner result = scanner;
         if (!filterToggledOn) { //Skip reading this section if you're not profiling! 
             int scanned = lookAndForget(result);
@@ -194,7 +194,7 @@ public class GTScanRequest {
             } else if (this.hasAggregation()) {
                 logger.info("pre aggregating results before returning");
                 this.doingStorageAggregation = true;
-                result = new GTAggregateScanner(result, this, deadline, spillEnabled);
+                result = new GTAggregateScanner(result, this, spillEnabled);
             } else {
                 logger.info("has no aggregation, skip it");
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/core-cube/src/main/java/org/apache/kylin/gridtable/ScannerWorker.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/ScannerWorker.java b/core-cube/src/main/java/org/apache/kylin/gridtable/ScannerWorker.java
deleted file mode 100644
index f26d993..0000000
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/ScannerWorker.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.gridtable;
-
-import java.io.IOException;
-import java.lang.reflect.InvocationTargetException;
-import java.util.Iterator;
-
-import org.apache.kylin.metadata.model.ISegment;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ScannerWorker {
-
-    private static final Logger logger = LoggerFactory.getLogger(ScannerWorker.class);
-    private IGTScanner internal = null;
-
-    public ScannerWorker(ISegment segment, Cuboid cuboid, GTScanRequest scanRequest, String gtStorage) {
-        if (scanRequest == null) {
-            logger.info("Segment {} will be skipped", segment);
-            internal = new EmptyGTScanner(0);
-            return;
-        }
-
-        final GTInfo info = scanRequest.getInfo();
-
-        try {
-            IGTStorage rpc = (IGTStorage) Class.forName(gtStorage).getConstructor(ISegment.class, Cuboid.class, GTInfo.class).newInstance(segment, cuboid, info); // default behavior
-            internal = rpc.getGTScanner(scanRequest);
-        } catch (IOException | InstantiationException | InvocationTargetException | IllegalAccessException | ClassNotFoundException | NoSuchMethodException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public Iterator<GTRecord> iterator() {
-        return internal.iterator();
-    }
-
-    public void close() throws IOException {
-        internal.close();
-    }
-
-    public long getScannedRowCount() {
-        return internal.getScannedRowCount();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java b/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
index ab0ea73..708dfde 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
@@ -50,6 +50,7 @@ public class StorageContext {
 
     private IStorageQuery storageQuery;
     private AtomicLong totalScanCount = new AtomicLong();
+    private AtomicLong totalScanBytes = new AtomicLong();
     private Cuboid cuboid;
     private boolean partialResultReturned = false;
 
@@ -161,6 +162,14 @@ public class StorageContext {
         return this.totalScanCount.addAndGet(count);
     }
 
+    public long getTotalScanBytes() {
+        return totalScanBytes.get();
+    }
+
+    public long increaseTotalScanBytes(long bytes) {
+        return totalScanBytes.addAndGet(bytes);
+    }
+
     public boolean isAcceptPartialResult() {
         return acceptPartialResult;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
index 9d6f946..974b8ea 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
@@ -30,7 +30,6 @@ import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.gridtable.GTRecord;
 import org.apache.kylin.gridtable.GTScanRequest;
 import org.apache.kylin.gridtable.IGTScanner;
-import org.apache.kylin.gridtable.ScannerWorker;
 import org.apache.kylin.metadata.filter.ITupleFilterTransformer;
 import org.apache.kylin.metadata.filter.StringCodeSystem;
 import org.apache.kylin.metadata.filter.TupleFilter;
@@ -79,7 +78,7 @@ public class CubeSegmentScanner implements IGTScanner {
         }
         scanRequest = scanRangePlanner.planScanRequest();
         String gtStorage = ((GTCubeStorageQueryBase) context.getStorageQuery()).getGTStorage();
-        scanner = new ScannerWorker(cubeSeg, cuboid, scanRequest, gtStorage);
+        scanner = new ScannerWorker(cubeSeg, cuboid, scanRequest, gtStorage, context);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/ScannerWorker.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/ScannerWorker.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/ScannerWorker.java
new file mode 100644
index 0000000..2a2a86a
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/ScannerWorker.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.storage.gtrecord;
+
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.gridtable.EmptyGTScanner;
+import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTRecord;
+import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.IGTScanner;
+import org.apache.kylin.gridtable.IGTStorage;
+import org.apache.kylin.metadata.model.ISegment;
+import org.apache.kylin.storage.StorageContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Iterator;
+
+public class ScannerWorker {
+
+    private static final Logger logger = LoggerFactory.getLogger(ScannerWorker.class);
+    private IGTScanner internal = null;
+
+    public ScannerWorker(ISegment segment, Cuboid cuboid, GTScanRequest scanRequest, String gtStorage, StorageContext context) {
+        if (scanRequest == null) {
+            logger.info("Segment {} will be skipped", segment);
+            internal = new EmptyGTScanner(0);
+            return;
+        }
+
+        final GTInfo info = scanRequest.getInfo();
+
+        try {
+            IGTStorage rpc = (IGTStorage) Class.forName(gtStorage).getConstructor(ISegment.class, Cuboid.class, GTInfo.class, StorageContext.class).newInstance(segment, cuboid, info, context); // default behavior
+            internal = rpc.getGTScanner(scanRequest);
+        } catch (IOException | InstantiationException | InvocationTargetException | IllegalAccessException | ClassNotFoundException | NoSuchMethodException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public Iterator<GTRecord> iterator() {
+        return internal.iterator();
+    }
+
+    public void close() throws IOException {
+        internal.close();
+    }
+
+    public long getScannedRowCount() {
+        return internal.getScannedRowCount();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/server-base/src/main/java/org/apache/kylin/rest/response/SQLResponse.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/response/SQLResponse.java b/server-base/src/main/java/org/apache/kylin/rest/response/SQLResponse.java
index 9c4e9da..387e6c9 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/response/SQLResponse.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/response/SQLResponse.java
@@ -55,6 +55,8 @@ public class SQLResponse implements Serializable {
 
     protected long totalScanCount;
 
+    protected long totalScanBytes;
+
     protected boolean hitExceptionCache = false;
 
     protected boolean storageCacheUsed = false;
@@ -150,6 +152,14 @@ public class SQLResponse implements Serializable {
         this.totalScanCount = totalScanCount;
     }
 
+    public long getTotalScanBytes() {
+        return totalScanBytes;
+    }
+
+    public void setTotalScanBytes(long totalScanBytes) {
+        this.totalScanBytes = totalScanBytes;
+    }
+
     public boolean isHitExceptionCache() {
         return hitExceptionCache;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 7ce38ea..9ccda03 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -276,6 +276,7 @@ public class QueryService extends BasicService {
         stringBuilder.append("Realization Names: ").append(realizationNames).append(newLine);
         stringBuilder.append("Cuboid Ids: ").append(cuboidIds).append(newLine);
         stringBuilder.append("Total scan count: ").append(response.getTotalScanCount()).append(newLine);
+        stringBuilder.append("Total scan bytes: ").append(response.getTotalScanBytes()).append(newLine);
         stringBuilder.append("Result row count: ").append(resultRowCount).append(newLine);
         stringBuilder.append("Accept Partial: ").append(request.isAcceptPartial()).append(newLine);
         stringBuilder.append("Is Partial Result: ").append(response.isPartial()).append(newLine);
@@ -580,15 +581,18 @@ public class QueryService extends BasicService {
 
         boolean isPartialResult = false;
         String cube = "";
-        StringBuilder sb = new StringBuilder("Scan count for each storageContext: ");
+        StringBuilder sb = new StringBuilder("Scan stats for each storageContext: ");
         long totalScanCount = 0;
+        long totalScanBytes = 0;
         if (OLAPContext.getThreadLocalContexts() != null) { // contexts can be null in case of 'explain plan for'
             for (OLAPContext ctx : OLAPContext.getThreadLocalContexts()) {
                 if (ctx.realization != null) {
                     isPartialResult |= ctx.storageContext.isPartialResultReturned();
                     cube = ctx.realization.getName();
                     totalScanCount += ctx.storageContext.getTotalScanCount();
-                    sb.append(ctx.storageContext.getTotalScanCount() + ",");
+                    totalScanBytes += ctx.storageContext.getTotalScanBytes();
+                    sb.append("{rows=").append(ctx.storageContext.getTotalScanCount()).
+                       append(" bytes=").append(ctx.storageContext.getTotalScanBytes()).append("} ");
                 }
             }
         }
@@ -596,6 +600,7 @@ public class QueryService extends BasicService {
 
         SQLResponse response = new SQLResponse(columnMetas, results, cube, 0, false, null, isPartialResult);
         response.setTotalScanCount(totalScanCount);
+        response.setTotalScanBytes(totalScanBytes);
 
         return response;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/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 dd9f74c..a2b2611 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
@@ -47,6 +47,7 @@ import org.apache.kylin.gridtable.GTScanRequest;
 import org.apache.kylin.gridtable.GTScanSelfTerminatedException;
 import org.apache.kylin.gridtable.IGTScanner;
 import org.apache.kylin.metadata.model.ISegment;
+import org.apache.kylin.storage.StorageContext;
 import org.apache.kylin.storage.gtrecord.DummyPartitionStreamer;
 import org.apache.kylin.storage.gtrecord.StorageResponseGTScatter;
 import org.apache.kylin.storage.hbase.HBaseConnection;
@@ -69,8 +70,8 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
 
     private static ExecutorService executorService = new LoggableCachedThreadPool();
 
-    public CubeHBaseEndpointRPC(ISegment segment, Cuboid cuboid, GTInfo fullGTInfo) {
-        super(segment, cuboid, fullGTInfo);
+    public CubeHBaseEndpointRPC(ISegment segment, Cuboid cuboid, GTInfo fullGTInfo, StorageContext context) {
+        super(segment, cuboid, fullGTInfo, context);
     }
 
     private byte[] getByteArrayForShort(short v) {
@@ -198,6 +199,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
                                         if (region == null)
                                             return;
 
+                                        context.increaseTotalScanBytes(result.getStats().getScannedBytes());
                                         totalScannedCount.addAndGet(result.getStats().getScannedRowCount());
                                         logger.info(logHeader + getStatsString(region, result));
 
@@ -280,6 +282,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
         Stats stats = result.getStats();
         sb.append("Endpoint RPC returned from HTable ").append(cubeSeg.getStorageLocationIdentifier()).append(" Shard ").append(BytesUtil.toHex(region)).append(" on host: ").append(stats.getHostname()).append(".");
         sb.append("Total scanned row: ").append(stats.getScannedRowCount()).append(". ");
+        sb.append("Total scanned bytes: ").append(stats.getScannedBytes()).append(". ");
         sb.append("Total filtered/aggred row: ").append(stats.getAggregatedRowCount()).append(". ");
         sb.append("Time elapsed in EP: ").append(stats.getServiceEndTime() - stats.getServiceStartTime()).append("(ms). ");
         sb.append("Server CPU usage: ").append(stats.getSystemCpuLoad()).append(", server physical mem left: ").append(stats.getFreePhysicalMemorySize()).append(", server swap mem left:").append(stats.getFreeSwapSpaceSize()).append(".");

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/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 05b34c7..11fbb19 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
@@ -48,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.storage.StorageContext;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -62,15 +63,18 @@ public abstract class CubeHBaseRPC implements IGTStorage {
     final protected CubeSegment cubeSeg;
     final protected Cuboid cuboid;
     final protected GTInfo fullGTInfo;
+    final protected StorageContext context;
+
     final private RowKeyEncoder fuzzyKeyEncoder;
     final private RowKeyEncoder fuzzyMaskEncoder;
 
-    public CubeHBaseRPC(ISegment segment, Cuboid cuboid, GTInfo fullGTInfo) {
+    public CubeHBaseRPC(ISegment segment, Cuboid cuboid, GTInfo fullGTInfo, StorageContext context) {
         Preconditions.checkArgument(segment instanceof CubeSegment, "segment must be CubeSegment");
         
         this.cubeSeg = (CubeSegment) segment;
         this.cuboid = cuboid;
         this.fullGTInfo = fullGTInfo;
+        this.context = context;
 
         this.fuzzyKeyEncoder = new FuzzyKeyEncoder(cubeSeg, cuboid);
         this.fuzzyMaskEncoder = new FuzzyMaskEncoder(cubeSeg, cuboid);

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
index a52af90..b94346c 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
@@ -24,6 +24,7 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
@@ -41,6 +42,7 @@ import org.apache.kylin.gridtable.GTScanRequest;
 import org.apache.kylin.gridtable.IGTScanner;
 import org.apache.kylin.gridtable.IGTStore;
 import org.apache.kylin.metadata.model.ISegment;
+import org.apache.kylin.storage.StorageContext;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -86,8 +88,8 @@ public class CubeHBaseScanRPC extends CubeHBaseRPC {
         }
     }
 
-    public CubeHBaseScanRPC(ISegment segment, Cuboid cuboid, final GTInfo fullGTInfo) {
-        super(segment, cuboid, fullGTInfo);
+    public CubeHBaseScanRPC(ISegment segment, Cuboid cuboid, final GTInfo fullGTInfo, StorageContext context) {
+        super(segment, cuboid, fullGTInfo, context);
     }
 
     @Override
@@ -180,12 +182,15 @@ public class CubeHBaseScanRPC extends CubeHBaseRPC {
         final Iterator<Result> allResultsIterator = Iterators.concat(resultIterators.iterator());
 
         CellListIterator cellListIterator = new CellListIterator() {
+            long scanBytes = 0;
+
             @Override
             public void close() throws IOException {
                 for (ResultScanner scanner : scanners) {
                     scanner.close();
                 }
                 hbaseTable.close();
+                context.increaseTotalScanBytes(scanBytes);
             }
 
             @Override
@@ -195,7 +200,11 @@ public class CubeHBaseScanRPC extends CubeHBaseRPC {
 
             @Override
             public List<Cell> next() {
-                return allResultsIterator.next().listCells();
+                List<Cell> result = allResultsIterator.next().listCells();
+                for (Cell cell : result) {
+                    scanBytes += CellUtil.estimatedSizeOf(cell);
+                }
+                return result;
             }
 
             @Override
@@ -232,4 +241,4 @@ public class CubeHBaseScanRPC extends CubeHBaseRPC {
             }
         };
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
index 3e0a065..1f6425f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
@@ -33,6 +33,7 @@ import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.client.Scan;
@@ -87,7 +88,19 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
 
     private long serviceStartTime;
 
-    static class InnerScannerAsIterator implements CellListIterator {
+    abstract static class BaseCellListIterator implements CellListIterator {
+        @Override
+        public final void remove() {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public final void close() throws IOException {
+            // no op. we close all region scanners at final block.
+        }
+    }
+
+    static class InnerScannerAsIterator extends BaseCellListIterator {
         private RegionScanner regionScanner;
         private List<Cell> nextOne = Lists.newArrayList();
         private List<Cell> ret = Lists.newArrayList();
@@ -127,15 +140,58 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
             }
             return ret;
         }
+    }
+
+    // TODO move this logic to HBaseReadonlyStore once it's been refactored
+    static class ResourceTrackingCellListIterator extends BaseCellListIterator {
+        private final Iterator<List<Cell>> delegate;
+        private final long rowCountLimit;
+        private final long bytesLimit;
+        private final long timeout;
+        private final long deadline;
+
+        private long rowCount;
+        private long rowBytes;
+
+        ResourceTrackingCellListIterator(Iterator<List<Cell>> delegate,
+                                         long rowCountLimit, long bytesLimit, long timeout) {
+            this.delegate = delegate;
+            this.rowCountLimit = rowCountLimit;
+            this.bytesLimit = bytesLimit;
+            this.timeout = timeout;
+            this.deadline = System.currentTimeMillis() + timeout;
+        }
 
         @Override
-        public void remove() {
-            throw new UnsupportedOperationException();
+        public boolean hasNext() {
+            if (rowCount > rowCountLimit) {
+                throw new GTScanExceedThresholdException("Number of rows scanned exceeds threshold " + rowCountLimit);
+            }
+            if (rowBytes > bytesLimit) {
+                throw new GTScanExceedThresholdException("Scanned " + rowBytes + " bytes exceeds threshold " + bytesLimit);
+            }
+            if ((rowCount % GTScanRequest.terminateCheckInterval == 1) && System.currentTimeMillis() > deadline) {
+                throw new GTScanTimeoutException("Scan timeout after " + timeout + " ms");
+            }
+            return delegate.hasNext();
         }
 
         @Override
-        public void close() throws IOException {
-            //does not need to close as regionScanner will be closed in finally block
+        public List<Cell> next() {
+            List<Cell> result = delegate.next();
+            rowCount++;
+            for (Cell cell : result) {
+                rowBytes += CellUtil.estimatedSizeOf(cell);
+            }
+            return result;
+        }
+
+        public long getTotalScannedRowCount() {
+            return rowCount;
+        }
+
+        public long getTotalScannedRowBytes() {
+            return rowBytes;
         }
     }
 
@@ -237,51 +293,18 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
             }
 
             final MutableBoolean scanNormalComplete = new MutableBoolean(true);
-            final long deadline = serviceStartTime + scanReq.getTimeout();
-            logger.info("deadline(local) is " + deadline);
             final long storagePushDownLimit = scanReq.getStoragePushDownLimit();
 
-            final CellListIterator cellListIterator = new CellListIterator() {
-
-                int counter = 0;
-
-                @Override
-                public void close() throws IOException {
-                    for (CellListIterator closeable : cellListsForeachRawScan) {
-                        closeable.close();
-                    }
-                }
-
-                @Override
-                public boolean hasNext() {
-
-                    counter++;
-
-                    if (counter > scanReq.getStorageScanRowNumThreshold()) {
-                        throw new GTScanExceedThresholdException("Exceed scan threshold at " + counter + ", consider increasing kylin.query.memory-budget-bytes and kylin.query.scan-threshold");
-                    }
-
-                    if (counter % (10 * GTScanRequest.terminateCheckInterval) == 1) {
-                        logger.info("scanning " + counter + "th row from HBase.");
-                    }
-                    return allCellLists.hasNext();
-                }
-
-                @Override
-                public List<Cell> next() {
-                    return allCellLists.next();
-                }
-
-                @Override
-                public void remove() {
-                    throw new UnsupportedOperationException();
-                }
-            };
+            ResourceTrackingCellListIterator cellListIterator = new ResourceTrackingCellListIterator(
+                    allCellLists,
+                    scanReq.getStorageScanRowNumThreshold(),
+                    Long.MAX_VALUE,
+                    scanReq.getTimeout());
 
             IGTStore store = new HBaseReadonlyStore(cellListIterator, scanReq, hbaseRawScans.get(0).hbaseColumns, hbaseColumnsToGT, request.getRowkeyPreambleSize(), behavior.delayToggledOn());
 
             IGTScanner rawScanner = store.scan(scanReq);
-            IGTScanner finalScanner = scanReq.decorateScanner(rawScanner, behavior.filterToggledOn(), behavior.aggrToggledOn(), false, deadline, request.getSpillEnabled());
+            IGTScanner finalScanner = scanReq.decorateScanner(rawScanner, behavior.filterToggledOn(), behavior.aggrToggledOn(), false, request.getSpillEnabled());
 
             ByteBuffer buffer = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);
 
@@ -290,13 +313,6 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
 
             try {
                 for (GTRecord oneRecord : finalScanner) {
-
-                    if (finalRowCount % GTScanRequest.terminateCheckInterval == 1) {
-                        if (System.currentTimeMillis() > deadline) {
-                            throw new GTScanTimeoutException("finalScanner timeouts after contributed " + finalRowCount);
-                        }
-                    }
-
                     buffer.clear();
                     try {
                         oneRecord.exportColumns(scanReq.getColumns(), buffer);
@@ -326,6 +342,8 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
             }
 
             appendProfileInfo(sb, "agg done");
+            logger.info("Total scanned {} rows and {} bytes",
+                    cellListIterator.getTotalScannedRowCount(), cellListIterator.getTotalScannedRowBytes());
 
             //outputStream.close() is not necessary
             byte[] compressedAllRows;
@@ -341,6 +359,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
             }
 
             appendProfileInfo(sb, "compress done");
+            logger.info("Size of final result = {} ({} before compressing)", compressedAllRows.length, allRows.length);
 
             OperatingSystemMXBean operatingSystemMXBean = (OperatingSystemMXBean) ManagementFactory.getOperatingSystemMXBean();
             double systemCpuLoad = operatingSystemMXBean.getSystemCpuLoad();
@@ -353,16 +372,17 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
             CubeVisitProtos.CubeVisitResponse.Builder responseBuilder = CubeVisitProtos.CubeVisitResponse.newBuilder();
             done.run(responseBuilder.//
                     setCompressedRows(HBaseZeroCopyByteString.wrap(compressedAllRows)).//too many array copies 
-                    setStats(CubeVisitProtos.CubeVisitResponse.Stats.newBuilder().//
-                            setAggregatedRowCount(finalScanner.getScannedRowCount() - finalRowCount).//
-                            setScannedRowCount(finalScanner.getScannedRowCount()).//
-                            setServiceStartTime(serviceStartTime).//
-                            setServiceEndTime(System.currentTimeMillis()).//
-                            setSystemCpuLoad(systemCpuLoad).//
-                            setFreePhysicalMemorySize(freePhysicalMemorySize).//
-                            setFreeSwapSpaceSize(freeSwapSpaceSize).//
-                            setHostname(InetAddress.getLocalHost().getHostName()).// 
-                            setEtcMsg(sb.toString()).//
+                    setStats(CubeVisitProtos.CubeVisitResponse.Stats.newBuilder().
+                            setAggregatedRowCount(cellListIterator.getTotalScannedRowCount() - finalRowCount).
+                            setScannedRowCount(cellListIterator.getTotalScannedRowCount()).
+                            setScannedBytes(cellListIterator.getTotalScannedRowBytes()).
+                            setServiceStartTime(serviceStartTime).
+                            setServiceEndTime(System.currentTimeMillis()).
+                            setSystemCpuLoad(systemCpuLoad).
+                            setFreePhysicalMemorySize(freePhysicalMemorySize).
+                            setFreeSwapSpaceSize(freeSwapSpaceSize).
+                            setHostname(InetAddress.getLocalHost().getHostName()).
+                            setEtcMsg(sb.toString()).
                             setNormalComplete(scanNormalComplete.booleanValue() ? 1 : 0).build())
                     .//
                     build());

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
index def0182..5a3aa5a 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
@@ -2255,6 +2255,16 @@ public final class CubeVisitProtos {
        * </pre>
        */
       int getNormalComplete();
+
+      // optional int64 scannedBytes = 11;
+      /**
+       * <code>optional int64 scannedBytes = 11;</code>
+       */
+      boolean hasScannedBytes();
+      /**
+       * <code>optional int64 scannedBytes = 11;</code>
+       */
+      long getScannedBytes();
     }
     /**
      * Protobuf type {@code CubeVisitResponse.Stats}
@@ -2357,6 +2367,11 @@ public final class CubeVisitProtos {
                 normalComplete_ = input.readInt32();
                 break;
               }
+              case 88: {
+                bitField0_ |= 0x00000400;
+                scannedBytes_ = input.readInt64();
+                break;
+              }
             }
           }
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -2619,6 +2634,22 @@ public final class CubeVisitProtos {
         return normalComplete_;
       }
 
+      // optional int64 scannedBytes = 11;
+      public static final int SCANNEDBYTES_FIELD_NUMBER = 11;
+      private long scannedBytes_;
+      /**
+       * <code>optional int64 scannedBytes = 11;</code>
+       */
+      public boolean hasScannedBytes() {
+        return ((bitField0_ & 0x00000400) == 0x00000400);
+      }
+      /**
+       * <code>optional int64 scannedBytes = 11;</code>
+       */
+      public long getScannedBytes() {
+        return scannedBytes_;
+      }
+
       private void initFields() {
         serviceStartTime_ = 0L;
         serviceEndTime_ = 0L;
@@ -2630,6 +2661,7 @@ public final class CubeVisitProtos {
         hostname_ = "";
         etcMsg_ = "";
         normalComplete_ = 0;
+        scannedBytes_ = 0L;
       }
       private byte memoizedIsInitialized = -1;
       public final boolean isInitialized() {
@@ -2673,6 +2705,9 @@ public final class CubeVisitProtos {
         if (((bitField0_ & 0x00000200) == 0x00000200)) {
           output.writeInt32(10, normalComplete_);
         }
+        if (((bitField0_ & 0x00000400) == 0x00000400)) {
+          output.writeInt64(11, scannedBytes_);
+        }
         getUnknownFields().writeTo(output);
       }
 
@@ -2722,6 +2757,10 @@ public final class CubeVisitProtos {
           size += com.google.protobuf.CodedOutputStream
             .computeInt32Size(10, normalComplete_);
         }
+        if (((bitField0_ & 0x00000400) == 0x00000400)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeInt64Size(11, scannedBytes_);
+        }
         size += getUnknownFields().getSerializedSize();
         memoizedSerializedSize = size;
         return size;
@@ -2792,6 +2831,11 @@ public final class CubeVisitProtos {
           result = result && (getNormalComplete()
               == other.getNormalComplete());
         }
+        result = result && (hasScannedBytes() == other.hasScannedBytes());
+        if (hasScannedBytes()) {
+          result = result && (getScannedBytes()
+              == other.getScannedBytes());
+        }
         result = result &&
             getUnknownFields().equals(other.getUnknownFields());
         return result;
@@ -2848,6 +2892,10 @@ public final class CubeVisitProtos {
           hash = (37 * hash) + NORMALCOMPLETE_FIELD_NUMBER;
           hash = (53 * hash) + getNormalComplete();
         }
+        if (hasScannedBytes()) {
+          hash = (37 * hash) + SCANNEDBYTES_FIELD_NUMBER;
+          hash = (53 * hash) + hashLong(getScannedBytes());
+        }
         hash = (29 * hash) + getUnknownFields().hashCode();
         memoizedHashCode = hash;
         return hash;
@@ -2977,6 +3025,8 @@ public final class CubeVisitProtos {
           bitField0_ = (bitField0_ & ~0x00000100);
           normalComplete_ = 0;
           bitField0_ = (bitField0_ & ~0x00000200);
+          scannedBytes_ = 0L;
+          bitField0_ = (bitField0_ & ~0x00000400);
           return this;
         }
 
@@ -3045,6 +3095,10 @@ public final class CubeVisitProtos {
             to_bitField0_ |= 0x00000200;
           }
           result.normalComplete_ = normalComplete_;
+          if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
+            to_bitField0_ |= 0x00000400;
+          }
+          result.scannedBytes_ = scannedBytes_;
           result.bitField0_ = to_bitField0_;
           onBuilt();
           return result;
@@ -3095,6 +3149,9 @@ public final class CubeVisitProtos {
           if (other.hasNormalComplete()) {
             setNormalComplete(other.getNormalComplete());
           }
+          if (other.hasScannedBytes()) {
+            setScannedBytes(other.getScannedBytes());
+          }
           this.mergeUnknownFields(other.getUnknownFields());
           return this;
         }
@@ -3550,6 +3607,39 @@ public final class CubeVisitProtos {
           return this;
         }
 
+        // optional int64 scannedBytes = 11;
+        private long scannedBytes_ ;
+        /**
+         * <code>optional int64 scannedBytes = 11;</code>
+         */
+        public boolean hasScannedBytes() {
+          return ((bitField0_ & 0x00000400) == 0x00000400);
+        }
+        /**
+         * <code>optional int64 scannedBytes = 11;</code>
+         */
+        public long getScannedBytes() {
+          return scannedBytes_;
+        }
+        /**
+         * <code>optional int64 scannedBytes = 11;</code>
+         */
+        public Builder setScannedBytes(long value) {
+          bitField0_ |= 0x00000400;
+          scannedBytes_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>optional int64 scannedBytes = 11;</code>
+         */
+        public Builder clearScannedBytes() {
+          bitField0_ = (bitField0_ & ~0x00000400);
+          scannedBytes_ = 0L;
+          onChanged();
+          return this;
+        }
+
         // @@protoc_insertion_point(builder_scope:CubeVisitResponse.Stats)
       }
 
@@ -4349,20 +4439,21 @@ public final class CubeVisitProtos {
       "ze\030\003 \002(\005\0223\n\020hbaseColumnsToGT\030\004 \003(\0132\031.Cub" +
       "eVisitRequest.IntList\022\027\n\017kylinProperties" +
       "\030\005 \002(\t\022\017\n\007queryId\030\006 \001(\t\022\032\n\014spillEnabled\030" +
-      "\007 \001(\010:\004true\032\027\n\007IntList\022\014\n\004ints\030\001 \003(\005\"\321\002\n" +
+      "\007 \001(\010:\004true\032\027\n\007IntList\022\014\n\004ints\030\001 \003(\005\"\347\002\n" +
       "\021CubeVisitResponse\022\026\n\016compressedRows\030\001 \002",
       "(\014\022\'\n\005stats\030\002 \002(\0132\030.CubeVisitResponse.St" +
-      "ats\032\372\001\n\005Stats\022\030\n\020serviceStartTime\030\001 \001(\003\022" +
+      "ats\032\220\002\n\005Stats\022\030\n\020serviceStartTime\030\001 \001(\003\022" +
       "\026\n\016serviceEndTime\030\002 \001(\003\022\027\n\017scannedRowCou" +
       "nt\030\003 \001(\003\022\032\n\022aggregatedRowCount\030\004 \001(\003\022\025\n\r" +
       "systemCpuLoad\030\005 \001(\001\022\036\n\026freePhysicalMemor" +
       "ySize\030\006 \001(\001\022\031\n\021freeSwapSpaceSize\030\007 \001(\001\022\020" +
       "\n\010hostname\030\010 \001(\t\022\016\n\006etcMsg\030\t \001(\t\022\026\n\016norm" +
-      "alComplete\030\n \001(\0052F\n\020CubeVisitService\0222\n\t" +
-      "visitCube\022\021.CubeVisitRequest\032\022.CubeVisit" +
-      "ResponseB`\nEorg.apache.kylin.storage.hba",
-      "se.cube.v2.coprocessor.endpoint.generate" +
-      "dB\017CubeVisitProtosH\001\210\001\001\240\001\001"
+      "alComplete\030\n \001(\005\022\024\n\014scannedBytes\030\013 \001(\0032F" +
+      "\n\020CubeVisitService\0222\n\tvisitCube\022\021.CubeVi" +
+      "sitRequest\032\022.CubeVisitResponseB`\nEorg.ap",
+      "ache.kylin.storage.hbase.cube.v2.coproce" +
+      "ssor.endpoint.generatedB\017CubeVisitProtos" +
+      "H\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -4392,7 +4483,7 @@ public final class CubeVisitProtos {
           internal_static_CubeVisitResponse_Stats_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_CubeVisitResponse_Stats_descriptor,
-              new java.lang.String[] { "ServiceStartTime", "ServiceEndTime", "ScannedRowCount", "AggregatedRowCount", "SystemCpuLoad", "FreePhysicalMemorySize", "FreeSwapSpaceSize", "Hostname", "EtcMsg", "NormalComplete", });
+              new java.lang.String[] { "ServiceStartTime", "ServiceEndTime", "ScannedRowCount", "AggregatedRowCount", "SystemCpuLoad", "FreePhysicalMemorySize", "FreeSwapSpaceSize", "Hostname", "EtcMsg", "NormalComplete", "ScannedBytes", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/kylin/blob/e09338b3/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
index c7c2954..f416669 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
@@ -54,6 +54,7 @@ message CubeVisitResponse {
         optional string hostname = 8;
         optional string etcMsg = 9;
         optional int32 normalComplete =10;//when time outs, normalComplete will be false
+        optional int64 scannedBytes = 11;
     }
     required bytes compressedRows = 1;
     required Stats stats = 2;


[05/13] kylin git commit: KYLIN-2438 replace scan threshold with max scan bytes

Posted by li...@apache.org.
KYLIN-2438 replace scan threshold with max scan bytes


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

Branch: refs/heads/master-hbase0.98
Commit: 09a086688a664585c57b715046a9869b75351a52
Parents: edf6cef
Author: gaodayue <ga...@meituan.com>
Authored: Thu Feb 9 20:18:54 2017 +0800
Committer: gaodayue <ga...@meituan.com>
Committed: Fri Feb 10 18:07:58 2017 +0800

----------------------------------------------------------------------
 build/conf/kylin.properties                     |  16 ++-
 .../apache/kylin/common/KylinConfigBase.java    |  15 ++-
 .../org/apache/kylin/common/QueryContext.java   |  54 ++++----
 .../kylin-backward-compatibility.properties     |   8 +-
 .../apache/kylin/storage/StorageContext.java    |  14 ---
 .../exception/ScanOutOfLimitException.java      |  31 -----
 .../storage/gtrecord/CubeScanRangePlanner.java  |   4 +-
 .../gtrecord/GTCubeStorageQueryBase.java        |  28 -----
 .../gtrecord/SequentialCubeTupleIterator.java   |  15 +--
 .../org/apache/kylin/query/KylinTestBase.java   |   4 +-
 .../kylin/storage/hbase/ITStorageTest.java      |  12 --
 .../kylin/query/enumerator/OLAPEnumerator.java  |  18 ---
 .../kylin/query/enumerator/OLAPQuery.java       |   2 -
 .../apache/kylin/rest/service/CacheService.java |   6 -
 .../apache/kylin/rest/service/QueryService.java |   9 +-
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java     |  12 +-
 .../coprocessor/endpoint/CubeVisitService.java  |   4 +-
 .../endpoint/generated/CubeVisitProtos.java     | 124 ++++++++++++++++---
 .../endpoint/protobuf/CubeVisit.proto           |   1 +
 19 files changed, 186 insertions(+), 191 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/build/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/build/conf/kylin.properties b/build/conf/kylin.properties
index 1232c47..095a53f 100644
--- a/build/conf/kylin.properties
+++ b/build/conf/kylin.properties
@@ -86,6 +86,14 @@ kylin.storage.hbase.owner-tag=whoami@kylin.apache.org
 
 kylin.storage.hbase.coprocessor-mem-gb=3
 
+# By default kylin can spill query's intermediate results to disks when it's consuming too much memory.
+# Set it to false if you want query to abort immediately in such condition.
+kylin.storage.hbase.coprocessor-spill-enabled=true
+
+# The maximum number of bytes each coprocessor is allowed to scan.
+# To allow arbitrary large scan, you can set it to 0.
+kylin.storage.hbase.coprocessor-max-scan-bytes=3221225472
+
 # The default coprocessor timeout is (hbase.rpc.timeout * 0.9) / 1000 seconds,
 # You can set it to a smaller value. 0 means use default.
 # kylin.storage.hbase.coprocessor-timeout-seconds=0
@@ -148,13 +156,13 @@ kylin.snapshot.max-mb=300
 
 ### QUERY ###
 
-kylin.query.scan-threshold=10000000
+# Controls the maximum number of bytes a query is allowed to scan storage.
+# The default value 0 means no limit.
+# The counterpart kylin.storage.hbase.coprocessor-max-scan-bytes sets the maximum per coprocessor.
+kylin.query.max-scan-bytes=0
 
 kylin.query.udf.version=org.apache.kylin.query.udf.VersionUDF
 
-# 3G
-kylin.query.memory-budget-bytes=3221225472
-
 kylin.query.cache-enabled=true
 
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/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 894e28a..c77788b 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
@@ -613,6 +613,11 @@ abstract public class KylinConfigBase implements Serializable {
         return Boolean.parseBoolean(this.getOptional("kylin.storage.hbase.coprocessor-spill-enabled", "true"));
     }
 
+    public long getQueryCoprocessorMaxScanBytes() {
+        long value = Long.parseLong(this.getOptional("kylin.storage.hbase.coprocessor-max-scan-bytes", String.valueOf(3L * 1024 * 1024 * 1024)));
+        return value > 0 ? value : Long.MAX_VALUE;
+    }
+
     public int getQueryCoprocessorTimeoutSeconds() {
         return Integer.parseInt(this.getOptional("kylin.storage.hbase.coprocessor-timeout-seconds", "0"));
     }
@@ -807,10 +812,16 @@ abstract public class KylinConfigBase implements Serializable {
         return Integer.parseInt(getOptional("kylin.query.max-limit-pushdown", "10000"));
     }
 
+    @Deprecated
     public int getScanThreshold() {
         return Integer.parseInt(getOptional("kylin.query.scan-threshold", "10000000"));
     }
 
+    public long getQueryMaxScanBytes() {
+        long value = Long.parseLong(getOptional("kylin.query.max-scan-bytes", "0"));
+        return value > 0 ? value : Long.MAX_VALUE;
+    }
+
     public int getLargeQueryThreshold() {
         return Integer.parseInt(getOptional("kylin.query.large-query-threshold", String.valueOf((int) (getScanThreshold() * 0.1))));
     }
@@ -851,10 +862,6 @@ abstract public class KylinConfigBase implements Serializable {
         return Long.parseLong(this.getOptional("kylin.query.cache-threshold-scan-count", String.valueOf(10 * 1024)));
     }
 
-    public long getQueryMemBudget() {
-        return Long.parseLong(this.getOptional("kylin.query.memory-budget-bytes", String.valueOf(3L * 1024 * 1024 * 1024)));
-    }
-
     public boolean isQuerySecureEnabled() {
         return Boolean.parseBoolean(this.getOptional("kylin.query.security-enabled", "true"));
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/core-common/src/main/java/org/apache/kylin/common/QueryContext.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/QueryContext.java b/core-common/src/main/java/org/apache/kylin/common/QueryContext.java
index 93b8556..3a73993 100644
--- a/core-common/src/main/java/org/apache/kylin/common/QueryContext.java
+++ b/core-common/src/main/java/org/apache/kylin/common/QueryContext.java
@@ -18,44 +18,48 @@
 
 package org.apache.kylin.common;
 
-import java.util.Map;
-
-import com.google.common.collect.Maps;
+import java.util.concurrent.atomic.AtomicLong;
 
 /**
  * checkout {@link org.apache.kylin.common.debug.BackdoorToggles} for comparison
  */
 public class QueryContext {
-    private static final ThreadLocal<Map<String, String>> _queryContext = new ThreadLocal<Map<String, String>>();
 
-    public final static String KEY_QUERY_ID = "QUERY_ID";
+    private static final ThreadLocal<QueryContext> contexts = new ThreadLocal<QueryContext>() {
+        @Override
+        protected QueryContext initialValue() {
+            return new QueryContext();
+        }
+    };
+
+    private String queryId;
+    private AtomicLong scanBytes = new AtomicLong();
 
-    public static String getQueryId() {
-        return getString(KEY_QUERY_ID);
+    private QueryContext() {
+        // use QueryContext.current() instead
     }
 
-    public static void setQueryId(String uuid) {
-        setString(KEY_QUERY_ID, uuid);
+    public static QueryContext current() {
+        return contexts.get();
     }
 
-    private static void setString(String key, String value) {
-        Map<String, String> context = _queryContext.get();
-        if (context == null) {
-            Map<String, String> newMap = Maps.newHashMap();
-            newMap.put(key, value);
-            _queryContext.set(newMap);
-        } else {
-            context.put(key, value);
-        }
+    public static void reset() {
+        contexts.remove();
     }
 
-    private static String getString(String key) {
-        Map<String, String> context = _queryContext.get();
-        if (context == null) {
-            return null;
-        } else {
-            return context.get(key);
-        }
+    public String getQueryId() {
+        return queryId;
     }
 
+    public void setQueryId(String queryId) {
+        this.queryId = queryId;
+    }
+
+    public long getScanBytes() {
+        return scanBytes.get();
+    }
+
+    public long addAndGetScanBytes(long delta) {
+        return scanBytes.addAndGet(delta);
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/core-common/src/main/resources/kylin-backward-compatibility.properties
----------------------------------------------------------------------
diff --git a/core-common/src/main/resources/kylin-backward-compatibility.properties b/core-common/src/main/resources/kylin-backward-compatibility.properties
index 16871d8..ab9bcb1 100644
--- a/core-common/src/main/resources/kylin-backward-compatibility.properties
+++ b/core-common/src/main/resources/kylin-backward-compatibility.properties
@@ -142,8 +142,6 @@ kylin.query.hbase.hconnection.threads.alive.seconds=kylin.storage.hbase.hconnect
 
 ### QUERY ###
 
-kylin.query.pushdown.limit.max=kylin.query.max-limit-pushdown
-kylin.query.scan.threshold=kylin.query.scan-threshold
 kylin.query.filter.derived_in.max=kylin.query.derived-filter-translation-threshold
 kylin.query.badquery.stacktrace.depth=kylin.query.badquery-stacktrace-depth
 kylin.query.badquery.history.num=kylin.query.badquery-history-number
@@ -154,13 +152,17 @@ kylin.query.transformers=kylin.query.transformers
 kylin.query.cache.enabled=kylin.query.cache-enabled
 kylin.query.cache.threshold.duration=kylin.query.cache-threshold-duration
 kylin.query.cache.threshold.scancount=kylin.query.cache-threshold-scan-count
-kylin.query.mem.budget=kylin.query.memory-budget-bytes
+kylin.query.mem.budget=kylin.storage.hbase.coprocessor-max-scan-bytes
 kylin.query.ignore_unknown_function=kylin.query.ignore-unknown-function
 kylin.query.dim.distinct.max=kylin.query.max-dimension-count-distinct
 kylin.query.security.enabled=kylin.query.security-enabled
 kylin.query.access.controller=kylin.query.access-controller
 kylin.query.udf.=kylin.query.udf.
 
+#deprecated
+kylin.query.pushdown.limit.max=kylin.query.max-limit-pushdown
+kylin.query.scan.threshold=kylin.query.scan-threshold
+
 
 ### SERVER ###
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java b/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
index 708dfde..0f52c53 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
@@ -20,7 +20,6 @@ package org.apache.kylin.storage;
 
 import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.metadata.realization.IRealization;
 import org.slf4j.Logger;
@@ -35,7 +34,6 @@ public class StorageContext {
     private static final Logger logger = LoggerFactory.getLogger(StorageContext.class);
 
     private String connUrl;
-    private int threshold;
     private int limit = Integer.MAX_VALUE;
     private int offset = 0;
     private int finalPushDownLimit = Integer.MAX_VALUE;
@@ -54,10 +52,6 @@ public class StorageContext {
     private Cuboid cuboid;
     private boolean partialResultReturned = false;
 
-    public StorageContext() {
-        this.threshold = KylinConfig.getInstanceFromEnv().getScanThreshold();
-    }
-
     private Range<Long> reusedPeriod;
 
     public String getConnUrl() {
@@ -68,14 +62,6 @@ public class StorageContext {
         this.connUrl = connUrl;
     }
 
-    public int getThreshold() {
-        return threshold;
-    }
-
-    public void setThreshold(int t) {
-        threshold = t;
-    }
-
     public int getLimit() {
         return limit;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/core-storage/src/main/java/org/apache/kylin/storage/exception/ScanOutOfLimitException.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/exception/ScanOutOfLimitException.java b/core-storage/src/main/java/org/apache/kylin/storage/exception/ScanOutOfLimitException.java
deleted file mode 100644
index f77cc35..0000000
--- a/core-storage/src/main/java/org/apache/kylin/storage/exception/ScanOutOfLimitException.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.storage.exception;
-
-/**
- * @author ysong1
- * 
- */
-public class ScanOutOfLimitException extends RuntimeException {
-    private static final long serialVersionUID = 2045169570038227895L;
-
-    public ScanOutOfLimitException(String message) {
-        super(message);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeScanRangePlanner.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeScanRangePlanner.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeScanRangePlanner.java
index b05a629..6911827 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeScanRangePlanner.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeScanRangePlanner.java
@@ -150,8 +150,8 @@ public class CubeScanRangePlanner extends ScanRangePlannerBase {
         if (scanRanges != null && scanRanges.size() != 0) {
             scanRequest = new GTScanRequestBuilder().setInfo(gtInfo).setRanges(scanRanges).setDimensions(gtDimensions).//
                     setAggrGroupBy(gtAggrGroups).setAggrMetrics(gtAggrMetrics).setAggrMetricsFuncs(gtAggrFuncs).setFilterPushDown(gtFilter).//
-                    setAllowStorageAggregation(context.isNeedStorageAggregation()).setAggCacheMemThreshold(cubeSegment.getCubeInstance().getConfig().getQueryCoprocessorMemGB()).//
-                    setStoragePushDownLimit(context.getFinalPushDownLimit()).setStorageScanRowNumThreshold(context.getThreshold()).createGTScanRequest();
+                    setAllowStorageAggregation(context.isNeedStorageAggregation()).setAggCacheMemThreshold(cubeSegment.getConfig().getQueryCoprocessorMemGB()).//
+                    setStoragePushDownLimit(context.getFinalPushDownLimit()).createGTScanRequest();
         } else {
             scanRequest = null;
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/GTCubeStorageQueryBase.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/GTCubeStorageQueryBase.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/GTCubeStorageQueryBase.java
index 4dbdf94..a72460c 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/GTCubeStorageQueryBase.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/GTCubeStorageQueryBase.java
@@ -122,8 +122,6 @@ public abstract class GTCubeStorageQueryBase implements IStorageQuery {
         enableStorageLimitIfPossible(cuboid, groups, derivedPostAggregation, groupsD, filter, loosenedColumnD, sqlDigest.aggregations, context);
         // set query deadline
         context.setDeadline(cubeInstance);
-        // set cautious threshold to prevent out of memory
-        setThresholdIfNecessary(dimensionsD, metrics, context);
 
         List<CubeSegmentScanner> scanners = Lists.newArrayList();
         for (CubeSegment cubeSeg : cubeInstance.getSegments(SegmentStatusEnum.READY)) {
@@ -323,32 +321,6 @@ public abstract class GTCubeStorageQueryBase implements IStorageQuery {
         }
     }
 
-    private void setThresholdIfNecessary(Collection<TblColRef> dimensions, Collection<FunctionDesc> metrics, StorageContext context) {
-        boolean hasMemHungryMeasure = false;
-        for (FunctionDesc func : metrics) {
-            hasMemHungryMeasure |= func.getMeasureType().isMemoryHungry();
-        }
-
-        // need to limit the memory usage for memory hungry measures
-        if (hasMemHungryMeasure == false) {
-            return;
-        }
-
-        int rowSizeEst = dimensions.size() * 3;
-        for (FunctionDesc func : metrics) {
-            // FIXME getStorageBytesEstimate() is not appropriate as here we want size in memory (not in storage)
-            rowSizeEst += func.getReturnDataType().getStorageBytesEstimate();
-        }
-
-        long rowEst = this.cubeInstance.getConfig().getQueryMemBudget() / rowSizeEst;
-        if (rowEst > 0) {
-            logger.info("Memory budget is set to " + rowEst + " rows");
-            context.setThreshold((int) rowEst);
-        } else {
-            logger.info("Memory budget is not set.");
-        }
-    }
-
     private void enableStorageLimitIfPossible(Cuboid cuboid, Collection<TblColRef> groups, Set<TblColRef> derivedPostAggregation, Collection<TblColRef> groupsD, TupleFilter filter, Set<TblColRef> loosenedColumnD, Collection<FunctionDesc> functionDescs, StorageContext context) {
         boolean possible = true;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java
index c6b2c6c..bb2d7f9 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java
@@ -26,7 +26,6 @@ import java.util.Set;
 
 import javax.annotation.Nullable;
 
-import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.gridtable.GTScanTimeoutException;
 import org.apache.kylin.metadata.model.FunctionDesc;
@@ -35,7 +34,6 @@ import org.apache.kylin.metadata.tuple.ITuple;
 import org.apache.kylin.metadata.tuple.ITupleIterator;
 import org.apache.kylin.metadata.tuple.TupleInfo;
 import org.apache.kylin.storage.StorageContext;
-import org.apache.kylin.storage.exception.ScanOutOfLimitException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -48,8 +46,6 @@ public class SequentialCubeTupleIterator implements ITupleIterator {
 
     private static final Logger logger = LoggerFactory.getLogger(SequentialCubeTupleIterator.class);
 
-    private final int SCAN_THRESHOLD = KylinConfig.getInstanceFromEnv().getScanThreshold();
-
     protected List<CubeSegmentScanner> scanners;
     protected List<SegmentCubeTupleIterator> segmentCubeTupleIterators;
     protected Iterator<ITuple> tupleIterator;
@@ -142,15 +138,10 @@ public class SequentialCubeTupleIterator implements ITupleIterator {
 
     @Override
     public ITuple next() {
-        if (scanCount % 100 == 1 && System.currentTimeMillis() > context.getDeadline()) {
+        if (scanCount++ % 100 == 1 && System.currentTimeMillis() > context.getDeadline()) {
             throw new GTScanTimeoutException("Query Timeout!");
         }
 
-        // prevent the big query to make the Query Server OOM
-        if (scanCount++ > SCAN_THRESHOLD) {
-            throw new ScanOutOfLimitException("Scan count exceed the scan threshold: " + SCAN_THRESHOLD);
-        }
-
         if (++scanCountDelta >= 1000)
             flushScanCountDelta();
 
@@ -181,10 +172,6 @@ public class SequentialCubeTupleIterator implements ITupleIterator {
         }
     }
 
-    public int getScanCount() {
-        return scanCount;
-    }
-
     private void flushScanCountDelta() {
         context.increaseTotalScanCount(scanCountDelta);
         scanCountDelta = 0;

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java b/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
index 6b9397d..d83ad75 100644
--- a/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
+++ b/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
@@ -39,7 +39,6 @@ import java.util.Arrays;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Properties;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.logging.LogManager;
@@ -642,8 +641,7 @@ public class KylinTestBase {
 
         //setup cube conn
         File olapTmp = OLAPSchemaFactory.createTempOLAPJson(ProjectInstance.DEFAULT_PROJECT_NAME, config);
-        Properties props = new Properties();
-        cubeConnection = DriverManager.getConnection("jdbc:calcite:model=" + olapTmp.getAbsolutePath(), props);
+        cubeConnection = DriverManager.getConnection("jdbc:calcite:model=" + olapTmp.getAbsolutePath());
 
         //setup h2
         h2Connection = DriverManager.getConnection("jdbc:h2:mem:db" + (h2InstanceCount++) + ";CACHE_SIZE=32072", "sa", "");

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITStorageTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITStorageTest.java b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITStorageTest.java
index aea8bef..733ca06 100644
--- a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITStorageTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITStorageTest.java
@@ -39,13 +39,11 @@ import org.apache.kylin.storage.IStorageQuery;
 import org.apache.kylin.storage.StorageContext;
 import org.apache.kylin.storage.StorageFactory;
 import org.apache.kylin.storage.StorageMockUtils;
-import org.apache.kylin.storage.exception.ScanOutOfLimitException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.collect.Sets;
@@ -84,16 +82,6 @@ public class ITStorageTest extends HBaseMetadataTestCase {
         this.cleanupTestMetadata();
     }
 
-    @Test(expected = ScanOutOfLimitException.class)
-    @Ignore
-    public void testScanOutOfLimit() {
-        context.setThreshold(1);
-        List<TblColRef> groups = mockup.buildGroups();
-        List<FunctionDesc> aggregations = mockup.buildAggregations();
-
-        search(groups, aggregations, null, context);
-    }
-
     @Test
     public void test01() {
         List<TblColRef> groups = mockup.buildGroups();

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
index f012638..56b82b9 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
@@ -20,12 +20,9 @@ package org.apache.kylin.query.enumerator;
 
 import java.util.Arrays;
 import java.util.Map;
-import java.util.Properties;
 
 import org.apache.calcite.DataContext;
-import org.apache.calcite.jdbc.CalciteConnection;
 import org.apache.calcite.linq4j.Enumerator;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter;
@@ -111,9 +108,6 @@ public class OLAPEnumerator implements Enumerator<Object[]> {
     private ITupleIterator queryStorage() {
         logger.debug("query storage...");
 
-        // set connection properties
-        setConnectionProperties();
-
         // bind dynamic variables
         bindVariable(olapContext.filter);
 
@@ -156,16 +150,4 @@ public class OLAPEnumerator implements Enumerator<Object[]> {
             }
         }
     }
-
-    private void setConnectionProperties() {
-        CalciteConnection conn = (CalciteConnection) optiqContext.getQueryProvider();
-        Properties connProps = conn.getProperties();
-
-        String propThreshold = connProps.getProperty(OLAPQuery.PROP_SCAN_THRESHOLD);
-        if (!StringUtils.isBlank(propThreshold)) {
-            int threshold = Integer.valueOf(propThreshold);
-            olapContext.storageContext.setThreshold(threshold);
-        }
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/query/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java
index 27d8c94..8318a07 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java
@@ -28,8 +28,6 @@ import org.apache.kylin.query.relnode.OLAPContext;
  */
 public class OLAPQuery extends AbstractEnumerable<Object[]> implements Enumerable<Object[]> {
 
-    public static final String PROP_SCAN_THRESHOLD = "scan_threshold";
-
     public enum EnumeratorTypeEnum {
         OLAP, //finish query with Cube or II, or a combination of both
         LOOKUP_TABLE, //using a snapshot of lookup table

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
index 0938e95..af680a5 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
@@ -21,7 +21,6 @@ package org.apache.kylin.rest.service;
 import java.io.File;
 import java.io.IOException;
 import java.nio.charset.Charset;
-import java.util.Properties;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
@@ -30,11 +29,9 @@ import javax.sql.DataSource;
 import org.apache.calcite.jdbc.Driver;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.metadata.cachesync.Broadcaster;
 import org.apache.kylin.metadata.cachesync.Broadcaster.Event;
 import org.apache.kylin.metadata.project.ProjectInstance;
-import org.apache.kylin.query.enumerator.OLAPQuery;
 import org.apache.kylin.query.schema.OLAPSchemaFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -166,9 +163,6 @@ public class CacheService extends BasicService {
             }
 
             DriverManagerDataSource ds = new DriverManagerDataSource();
-            Properties props = new Properties();
-            props.setProperty(OLAPQuery.PROP_SCAN_THRESHOLD, String.valueOf(KylinConfig.getInstanceFromEnv().getScanThreshold()));
-            ds.setConnectionProperties(props);
             ds.setDriverClassName(Driver.class.getName());
             ds.setUrl("jdbc:calcite:model=" + modelJson.getAbsolutePath());
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 9ccda03..7d9e24d 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -63,6 +63,7 @@ import org.apache.kylin.common.util.SetThreadName;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.gridtable.GTScanExceedThresholdException;
 import org.apache.kylin.metadata.project.RealizationEntry;
 import org.apache.kylin.metadata.realization.RealizationType;
 import org.apache.kylin.query.relnode.OLAPContext;
@@ -79,7 +80,6 @@ import org.apache.kylin.rest.response.SQLResponse;
 import org.apache.kylin.rest.util.QueryUtil;
 import org.apache.kylin.rest.util.Serializer;
 import org.apache.kylin.rest.util.TableauInterceptor;
-import org.apache.kylin.storage.exception.ScanOutOfLimitException;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.storage.hybrid.HybridInstance;
 import org.slf4j.Logger;
@@ -267,7 +267,7 @@ public class QueryService extends BasicService {
         StringBuilder stringBuilder = new StringBuilder();
         stringBuilder.append(newLine);
         stringBuilder.append("==========================[QUERY]===============================").append(newLine);
-        stringBuilder.append("Query Id: ").append(QueryContext.getQueryId()).append(newLine);
+        stringBuilder.append("Query Id: ").append(QueryContext.current().getQueryId()).append(newLine);
         stringBuilder.append("SQL: ").append(request.getSql()).append(newLine);
         stringBuilder.append("User: ").append(user).append(newLine);
         stringBuilder.append("Success: ").append((null == response.getExceptionMessage())).append(newLine);
@@ -331,7 +331,7 @@ public class QueryService extends BasicService {
         final String queryId = UUID.randomUUID().toString();
         if (sqlRequest.getBackdoorToggles() != null)
             BackdoorToggles.addToggles(sqlRequest.getBackdoorToggles());
-        QueryContext.setQueryId(queryId);
+        QueryContext.current().setQueryId(queryId);
 
         try (SetThreadName ignored = new SetThreadName("Query %s", queryId)) {
             String sql = sqlRequest.getSql();
@@ -383,7 +383,7 @@ public class QueryService extends BasicService {
                 sqlResponse = new SQLResponse(null, null, 0, true, errMsg);
 
                 // for exception queries, only cache ScanOutOfLimitException
-                if (queryCacheEnabled && e instanceof ScanOutOfLimitException) {
+                if (queryCacheEnabled && e instanceof GTScanExceedThresholdException) {
                     Cache exceptionCache = cacheManager.getCache(EXCEPTION_QUERY_CACHE);
                     exceptionCache.put(new Element(sqlRequest, sqlResponse));
                 }
@@ -400,6 +400,7 @@ public class QueryService extends BasicService {
 
         } finally {
             BackdoorToggles.cleanToggles();
+            QueryContext.reset();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/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 a2b2611..68a84c1 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.kylin.common.util.LoggableCachedThreadPool;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.gridtable.GTInfo;
+import org.apache.kylin.gridtable.GTScanExceedThresholdException;
 import org.apache.kylin.gridtable.GTScanRequest;
 import org.apache.kylin.gridtable.GTScanSelfTerminatedException;
 import org.apache.kylin.gridtable.IGTScanner;
@@ -106,6 +107,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
     @SuppressWarnings("checkstyle:methodlength")
     @Override
     public IGTScanner getGTScanner(final GTScanRequest scanRequest) throws IOException {
+        final QueryContext queryContext = QueryContext.current();
 
         Pair<Short, Short> shardNumAndBaseShard = getShardNumAndBaseShard();
         short shardNum = shardNumAndBaseShard.getFirst();
@@ -160,11 +162,12 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
         }
         builder.setRowkeyPreambleSize(cubeSeg.getRowKeyPreambleSize());
         builder.setKylinProperties(kylinConfig.getConfigAsString());
-        final String queryId = QueryContext.getQueryId();
+        final String queryId = queryContext.getQueryId();
         if (queryId != null) {
             builder.setQueryId(queryId);
         }
         builder.setSpillEnabled(cubeSeg.getConfig().getQueryCoprocessorSpillEnabled());
+        builder.setMaxScanBytes(cubeSeg.getConfig().getQueryCoprocessorMaxScanBytes());
 
         for (final Pair<byte[], byte[]> epRange : getEPKeyRanges(cuboidBaseShard, shardNum, totalShards)) {
             executorService.submit(new Runnable() {
@@ -199,10 +202,15 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
                                         if (region == null)
                                             return;
 
-                                        context.increaseTotalScanBytes(result.getStats().getScannedBytes());
+                                        final long scanBytes = result.getStats().getScannedBytes();
+                                        context.increaseTotalScanBytes(scanBytes);
                                         totalScannedCount.addAndGet(result.getStats().getScannedRowCount());
                                         logger.info(logHeader + getStatsString(region, result));
 
+                                        if (queryContext.addAndGetScanBytes(scanBytes) > cubeSeg.getConfig().getQueryMaxScanBytes()) {
+                                            throw new GTScanExceedThresholdException("Query scanned " + queryContext.getScanBytes() + " bytes exceeds threshold " + cubeSeg.getConfig().getQueryMaxScanBytes());
+                                        }
+
                                         if (result.getStats().getNormalComplete() != 1) {
                                             abnormalFinish[0] = true;
                                             return;

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
index 1f6425f..5fd9740 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
@@ -297,8 +297,8 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
 
             ResourceTrackingCellListIterator cellListIterator = new ResourceTrackingCellListIterator(
                     allCellLists,
-                    scanReq.getStorageScanRowNumThreshold(),
-                    Long.MAX_VALUE,
+                    scanReq.getStorageScanRowNumThreshold(), // for old client (scan threshold)
+                    request.getMaxScanBytes() == 0 ? Long.MAX_VALUE : request.getMaxScanBytes(), // for new client
                     scanReq.getTimeout());
 
             IGTStore store = new HBaseReadonlyStore(cellListIterator, scanReq, hbaseRawScans.get(0).hbaseColumns, hbaseColumnsToGT, request.getRowkeyPreambleSize(), behavior.delayToggledOn());

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
index 5a3aa5a..4b6fc95 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
@@ -117,6 +117,16 @@ public final class CubeVisitProtos {
      * <code>optional bool spillEnabled = 7 [default = true];</code>
      */
     boolean getSpillEnabled();
+
+    // optional int64 maxScanBytes = 8;
+    /**
+     * <code>optional int64 maxScanBytes = 8;</code>
+     */
+    boolean hasMaxScanBytes();
+    /**
+     * <code>optional int64 maxScanBytes = 8;</code>
+     */
+    long getMaxScanBytes();
   }
   /**
    * Protobuf type {@code CubeVisitRequest}
@@ -207,6 +217,11 @@ public final class CubeVisitProtos {
               spillEnabled_ = input.readBool();
               break;
             }
+            case 64: {
+              bitField0_ |= 0x00000040;
+              maxScanBytes_ = input.readInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -952,6 +967,22 @@ public final class CubeVisitProtos {
       return spillEnabled_;
     }
 
+    // optional int64 maxScanBytes = 8;
+    public static final int MAXSCANBYTES_FIELD_NUMBER = 8;
+    private long maxScanBytes_;
+    /**
+     * <code>optional int64 maxScanBytes = 8;</code>
+     */
+    public boolean hasMaxScanBytes() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>optional int64 maxScanBytes = 8;</code>
+     */
+    public long getMaxScanBytes() {
+      return maxScanBytes_;
+    }
+
     private void initFields() {
       gtScanRequest_ = com.google.protobuf.ByteString.EMPTY;
       hbaseRawScan_ = com.google.protobuf.ByteString.EMPTY;
@@ -960,6 +991,7 @@ public final class CubeVisitProtos {
       kylinProperties_ = "";
       queryId_ = "";
       spillEnabled_ = true;
+      maxScanBytes_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -1010,6 +1042,9 @@ public final class CubeVisitProtos {
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
         output.writeBool(7, spillEnabled_);
       }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeInt64(8, maxScanBytes_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -1047,6 +1082,10 @@ public final class CubeVisitProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(7, spillEnabled_);
       }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(8, maxScanBytes_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -1102,6 +1141,11 @@ public final class CubeVisitProtos {
         result = result && (getSpillEnabled()
             == other.getSpillEnabled());
       }
+      result = result && (hasMaxScanBytes() == other.hasMaxScanBytes());
+      if (hasMaxScanBytes()) {
+        result = result && (getMaxScanBytes()
+            == other.getMaxScanBytes());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -1143,6 +1187,10 @@ public final class CubeVisitProtos {
         hash = (37 * hash) + SPILLENABLED_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getSpillEnabled());
       }
+      if (hasMaxScanBytes()) {
+        hash = (37 * hash) + MAXSCANBYTES_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getMaxScanBytes());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -1271,6 +1319,8 @@ public final class CubeVisitProtos {
         bitField0_ = (bitField0_ & ~0x00000020);
         spillEnabled_ = true;
         bitField0_ = (bitField0_ & ~0x00000040);
+        maxScanBytes_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000080);
         return this;
       }
 
@@ -1332,6 +1382,10 @@ public final class CubeVisitProtos {
           to_bitField0_ |= 0x00000020;
         }
         result.spillEnabled_ = spillEnabled_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.maxScanBytes_ = maxScanBytes_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1396,6 +1450,9 @@ public final class CubeVisitProtos {
         if (other.hasSpillEnabled()) {
           setSpillEnabled(other.getSpillEnabled());
         }
+        if (other.hasMaxScanBytes()) {
+          setMaxScanBytes(other.getMaxScanBytes());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -1989,6 +2046,39 @@ public final class CubeVisitProtos {
         return this;
       }
 
+      // optional int64 maxScanBytes = 8;
+      private long maxScanBytes_ ;
+      /**
+       * <code>optional int64 maxScanBytes = 8;</code>
+       */
+      public boolean hasMaxScanBytes() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      /**
+       * <code>optional int64 maxScanBytes = 8;</code>
+       */
+      public long getMaxScanBytes() {
+        return maxScanBytes_;
+      }
+      /**
+       * <code>optional int64 maxScanBytes = 8;</code>
+       */
+      public Builder setMaxScanBytes(long value) {
+        bitField0_ |= 0x00000080;
+        maxScanBytes_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 maxScanBytes = 8;</code>
+       */
+      public Builder clearMaxScanBytes() {
+        bitField0_ = (bitField0_ & ~0x00000080);
+        maxScanBytes_ = 0L;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:CubeVisitRequest)
     }
 
@@ -4433,27 +4523,27 @@ public final class CubeVisitProtos {
     java.lang.String[] descriptorData = {
       "\npstorage-hbase/src/main/java/org/apache" +
       "/kylin/storage/hbase/cube/v2/coprocessor" +
-      "/endpoint/protobuf/CubeVisit.proto\"\357\001\n\020C" +
+      "/endpoint/protobuf/CubeVisit.proto\"\205\002\n\020C" +
       "ubeVisitRequest\022\025\n\rgtScanRequest\030\001 \002(\014\022\024" +
       "\n\014hbaseRawScan\030\002 \002(\014\022\032\n\022rowkeyPreambleSi" +
       "ze\030\003 \002(\005\0223\n\020hbaseColumnsToGT\030\004 \003(\0132\031.Cub" +
       "eVisitRequest.IntList\022\027\n\017kylinProperties" +
       "\030\005 \002(\t\022\017\n\007queryId\030\006 \001(\t\022\032\n\014spillEnabled\030" +
-      "\007 \001(\010:\004true\032\027\n\007IntList\022\014\n\004ints\030\001 \003(\005\"\347\002\n" +
-      "\021CubeVisitResponse\022\026\n\016compressedRows\030\001 \002",
-      "(\014\022\'\n\005stats\030\002 \002(\0132\030.CubeVisitResponse.St" +
-      "ats\032\220\002\n\005Stats\022\030\n\020serviceStartTime\030\001 \001(\003\022" +
-      "\026\n\016serviceEndTime\030\002 \001(\003\022\027\n\017scannedRowCou" +
-      "nt\030\003 \001(\003\022\032\n\022aggregatedRowCount\030\004 \001(\003\022\025\n\r" +
-      "systemCpuLoad\030\005 \001(\001\022\036\n\026freePhysicalMemor" +
-      "ySize\030\006 \001(\001\022\031\n\021freeSwapSpaceSize\030\007 \001(\001\022\020" +
-      "\n\010hostname\030\010 \001(\t\022\016\n\006etcMsg\030\t \001(\t\022\026\n\016norm" +
-      "alComplete\030\n \001(\005\022\024\n\014scannedBytes\030\013 \001(\0032F" +
-      "\n\020CubeVisitService\0222\n\tvisitCube\022\021.CubeVi" +
-      "sitRequest\032\022.CubeVisitResponseB`\nEorg.ap",
-      "ache.kylin.storage.hbase.cube.v2.coproce" +
-      "ssor.endpoint.generatedB\017CubeVisitProtos" +
-      "H\001\210\001\001\240\001\001"
+      "\007 \001(\010:\004true\022\024\n\014maxScanBytes\030\010 \001(\003\032\027\n\007Int" +
+      "List\022\014\n\004ints\030\001 \003(\005\"\347\002\n\021CubeVisitResponse",
+      "\022\026\n\016compressedRows\030\001 \002(\014\022\'\n\005stats\030\002 \002(\0132" +
+      "\030.CubeVisitResponse.Stats\032\220\002\n\005Stats\022\030\n\020s" +
+      "erviceStartTime\030\001 \001(\003\022\026\n\016serviceEndTime\030" +
+      "\002 \001(\003\022\027\n\017scannedRowCount\030\003 \001(\003\022\032\n\022aggreg" +
+      "atedRowCount\030\004 \001(\003\022\025\n\rsystemCpuLoad\030\005 \001(" +
+      "\001\022\036\n\026freePhysicalMemorySize\030\006 \001(\001\022\031\n\021fre" +
+      "eSwapSpaceSize\030\007 \001(\001\022\020\n\010hostname\030\010 \001(\t\022\016" +
+      "\n\006etcMsg\030\t \001(\t\022\026\n\016normalComplete\030\n \001(\005\022\024" +
+      "\n\014scannedBytes\030\013 \001(\0032F\n\020CubeVisitService" +
+      "\0222\n\tvisitCube\022\021.CubeVisitRequest\032\022.CubeV",
+      "isitResponseB`\nEorg.apache.kylin.storage" +
+      ".hbase.cube.v2.coprocessor.endpoint.gene" +
+      "ratedB\017CubeVisitProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -4465,7 +4555,7 @@ public final class CubeVisitProtos {
           internal_static_CubeVisitRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_CubeVisitRequest_descriptor,
-              new java.lang.String[] { "GtScanRequest", "HbaseRawScan", "RowkeyPreambleSize", "HbaseColumnsToGT", "KylinProperties", "QueryId", "SpillEnabled", });
+              new java.lang.String[] { "GtScanRequest", "HbaseRawScan", "RowkeyPreambleSize", "HbaseColumnsToGT", "KylinProperties", "QueryId", "SpillEnabled", "MaxScanBytes", });
           internal_static_CubeVisitRequest_IntList_descriptor =
             internal_static_CubeVisitRequest_descriptor.getNestedTypes().get(0);
           internal_static_CubeVisitRequest_IntList_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/kylin/blob/09a08668/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
index f416669..00015fc 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
@@ -37,6 +37,7 @@ message CubeVisitRequest {
     required string kylinProperties = 5; // kylin properties
     optional string queryId = 6;
     optional bool spillEnabled = 7 [default = true];
+    optional int64 maxScanBytes = 8; // 0 means no limit
     message IntList {
         repeated int32 ints = 1;
     }


[08/13] kylin git commit: minor, update smoke-test scripts

Posted by li...@apache.org.
minor, update smoke-test scripts


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

Branch: refs/heads/master-hbase0.98
Commit: 707b05598a1b65e0c1b85c2dffd57f7cf2ac8dfb
Parents: cd4e116
Author: lidongsjtu <li...@apache.org>
Authored: Sun Feb 12 09:11:24 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Sun Feb 12 09:11:24 2017 +0800

----------------------------------------------------------------------
 build/smoke-test/testQuery.py | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/707b0559/build/smoke-test/testQuery.py
----------------------------------------------------------------------
diff --git a/build/smoke-test/testQuery.py b/build/smoke-test/testQuery.py
index 72eb80d..87a2456 100644
--- a/build/smoke-test/testQuery.py
+++ b/build/smoke-test/testQuery.py
@@ -59,6 +59,7 @@ class testQuery(unittest.TestCase):
             del actual_result['duration']
             del actual_result['hitExceptionCache']
             del actual_result['storageCacheUsed']
+            del actual_result['totalScanBytes']
 
             expect_result = json.loads(open(sql_file[:-4] + '.json').read().strip())
             self.assertEqual(actual_result, expect_result, 'Query result does not equal.')


[02/13] kylin git commit: KYLIN-2436 add a configuration knob to disable spilling of aggregation cache

Posted by li...@apache.org.
KYLIN-2436 add a configuration knob to disable spilling of aggregation cache


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

Branch: refs/heads/master-hbase0.98
Commit: ecf6a69fece7cbda3a9bd8d678c928224ce677aa
Parents: 41332f3
Author: gaodayue <ga...@meituan.com>
Authored: Mon Feb 6 14:59:06 2017 +0800
Committer: gaodayue <ga...@meituan.com>
Committed: Fri Feb 10 12:43:34 2017 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/KylinConfigBase.java    |   4 +
 .../cube/inmemcubing/InMemCubeBuilder.java      |   2 +-
 .../kylin/gridtable/GTAggregateScanner.java     |  54 ++++----
 .../apache/kylin/gridtable/GTScanRequest.java   |   6 +-
 .../GTScanSelfTerminatedException.java          |   4 +
 .../gridtable/AggregationCacheSpillTest.java    |   4 +-
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java     |   1 +
 .../coprocessor/endpoint/CubeVisitService.java  |  12 +-
 .../endpoint/generated/CubeVisitProtos.java     | 123 ++++++++++++++++---
 .../endpoint/protobuf/CubeVisit.proto           |   1 +
 10 files changed, 162 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/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 dce4149..81f0187 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
@@ -609,6 +609,10 @@ abstract public class KylinConfigBase implements Serializable {
         return Double.parseDouble(this.getOptional("kylin.storage.hbase.coprocessor-mem-gb", "3.0"));
     }
 
+    public boolean getQueryCoprocessorSpillEnabled() {
+        return Boolean.parseBoolean(this.getOptional("kylin.storage.hbase.coprocessor-spill-enabled", "true"));
+    }
+
     public int getQueryCoprocessorTimeoutSeconds() {
         return Integer.parseInt(this.getOptional("kylin.storage.hbase.coprocessor-timeout-seconds", "0"));
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
index 398c594..a74f0c0 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
@@ -335,7 +335,7 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
 
         Pair<ImmutableBitSet, ImmutableBitSet> dimensionMetricsBitSet = InMemCubeBuilderUtils.getDimensionAndMetricColumnBitSet(baseCuboidId, measureCount);
         GTScanRequest req = new GTScanRequestBuilder().setInfo(baseCuboid.getInfo()).setRanges(null).setDimensions(null).setAggrGroupBy(dimensionMetricsBitSet.getFirst()).setAggrMetrics(dimensionMetricsBitSet.getSecond()).setAggrMetricsFuncs(metricsAggrFuncs).setFilterPushDown(null).createGTScanRequest();
-        GTAggregateScanner aggregationScanner = new GTAggregateScanner(baseInput, req, Long.MAX_VALUE);
+        GTAggregateScanner aggregationScanner = new GTAggregateScanner(baseInput, req);
         aggregationScanner.trackMemoryLevel(baseCuboidMemTracker);
 
         int count = 0;

http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
index 9158aa3..147dbc1 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
@@ -63,15 +63,20 @@ public class GTAggregateScanner implements IGTScanner {
     final String[] metricsAggrFuncs;
     final IGTScanner inputScanner;
     final AggregationCache aggrCache;
-    final long spillThreshold;
+    final long spillThreshold; // 0 means no memory control && no spill
     final int storagePushDownLimit;//default to be Int.MAX
     final long deadline;
+    final boolean spillEnabled;
 
     private int aggregatedRowCount = 0;
     private MemoryWaterLevel memTracker;
     private boolean[] aggrMask;
 
-    public GTAggregateScanner(IGTScanner inputScanner, GTScanRequest req, long deadline) {
+    public GTAggregateScanner(IGTScanner inputScanner, GTScanRequest req) {
+        this(inputScanner, req, Long.MAX_VALUE, true);
+    }
+
+    public GTAggregateScanner(IGTScanner inputScanner, GTScanRequest req, long deadline, boolean spillEnabled) {
         if (!req.hasAggregation())
             throw new IllegalStateException();
 
@@ -86,6 +91,7 @@ public class GTAggregateScanner implements IGTScanner {
         this.aggrMask = new boolean[metricsAggrFuncs.length];
         this.storagePushDownLimit = req.getStoragePushDownLimit();
         this.deadline = deadline;
+        this.spillEnabled = spillEnabled;
 
         Arrays.fill(aggrMask, true);
     }
@@ -276,10 +282,16 @@ public class GTAggregateScanner implements IGTScanner {
                 if (memTracker != null) {
                     memTracker.markHigh();
                 }
-                if (spillThreshold > 0) {
+
+                final long estMemSize = estimatedMemSize();
+                if (spillThreshold > 0 && estMemSize > spillThreshold) {
                     // spill to disk when aggBufMap used too large memory
-                    if (estimatedMemSize() > spillThreshold) {
-                        spillBuffMap();
+                    if (spillEnabled) {
+                        spillBuffMap(estMemSize);
+                        aggBufMap = createBuffMap();
+
+                    } else {
+                        throw new GTScanSelfTerminatedException("Aggregation using more than " + spillThreshold + " memory and spill is disabled");
                     }
                 }
             }
@@ -306,17 +318,13 @@ public class GTAggregateScanner implements IGTScanner {
             return true;
         }
 
-        private void spillBuffMap() throws RuntimeException {
-            if (aggBufMap.isEmpty())
-                return;
-
+        private void spillBuffMap(long estMemSize) throws RuntimeException {
             try {
-                Dump dump = new Dump(aggBufMap);
+                Dump dump = new Dump(aggBufMap, estMemSize);
                 dump.flush();
                 dumps.add(dump);
-                aggBufMap = createBuffMap();
             } catch (Exception e) {
-                throw new RuntimeException("AggregationCache spill failed: " + e.getMessage());
+                throw new RuntimeException("AggregationCache failed to spill", e);
             }
         }
 
@@ -372,9 +380,9 @@ public class GTAggregateScanner implements IGTScanner {
                 };
             } else {
                 // the spill case
-
-                logger.info("Last spill, current AggregationCache memory estimated size is: " + getEstimateSizeOfAggrCache());
-                this.spillBuffMap();
+                if (!aggBufMap.isEmpty()) {
+                    this.spillBuffMap(getEstimateSizeOfAggrCache()); // TODO allow merge in-mem map with spilled dumps
+                }
 
                 return new Iterator<GTRecord>() {
                     final DumpMerger merger = new DumpMerger(dumps);
@@ -430,12 +438,16 @@ public class GTAggregateScanner implements IGTScanner {
         }
 
         class Dump implements Iterable<Pair<byte[], byte[]>> {
-            File dumpedFile;
-            DataInputStream dis;
+            final File dumpedFile;
             SortedMap<byte[], MeasureAggregator[]> buffMap;
+            final long estMemSize;
 
-            public Dump(SortedMap<byte[], MeasureAggregator[]> buffMap) throws IOException {
+            DataInputStream dis;
+
+            public Dump(SortedMap<byte[], MeasureAggregator[]> buffMap, long estMemSize) throws IOException {
+                this.dumpedFile = File.createTempFile("KYLIN_SPILL_", ".tmp");
                 this.buffMap = buffMap;
+                this.estMemSize = estMemSize;
             }
 
             @Override
@@ -482,13 +494,13 @@ public class GTAggregateScanner implements IGTScanner {
             }
 
             public void flush() throws IOException {
+                logger.info("AggregationCache(size={} est_mem_size={} threshold={}) will spill to {}",
+                        buffMap.size(), estMemSize, spillThreshold, dumpedFile.getAbsolutePath());
+
                 if (buffMap != null) {
                     DataOutputStream dos = null;
                     Object[] aggrResult = null;
                     try {
-                        dumpedFile = File.createTempFile("KYLIN_AGGR_", ".tmp");
-
-                        logger.info("AggregationCache will dump to file: " + dumpedFile.getAbsolutePath());
                         dos = new DataOutputStream(new FileOutputStream(dumpedFile));
                         dos.writeInt(buffMap.size());
                         for (Entry<byte[], MeasureAggregator[]> entry : buffMap.entrySet()) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java
index 7976e3f..7c94f5a 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java
@@ -166,13 +166,13 @@ public class GTScanRequest {
      * Refer to CoprocessorBehavior for explanation
      */
     public IGTScanner decorateScanner(IGTScanner scanner, boolean filterToggledOn, boolean aggrToggledOn, long deadline) throws IOException {
-        return decorateScanner(scanner, filterToggledOn, aggrToggledOn, false, deadline);
+        return decorateScanner(scanner, filterToggledOn, aggrToggledOn, false, deadline, true);
     }
 
     /**
      * hasPreFiltered indicate the data has been filtered before scanning
      */
-    public IGTScanner decorateScanner(IGTScanner scanner, boolean filterToggledOn, boolean aggrToggledOn, boolean hasPreFiltered, long deadline) throws IOException {
+    public IGTScanner decorateScanner(IGTScanner scanner, boolean filterToggledOn, boolean aggrToggledOn, boolean hasPreFiltered, long deadline, boolean spillEnabled) throws IOException {
         IGTScanner result = scanner;
         if (!filterToggledOn) { //Skip reading this section if you're not profiling! 
             int scanned = lookAndForget(result);
@@ -194,7 +194,7 @@ public class GTScanRequest {
             } else if (this.hasAggregation()) {
                 logger.info("pre aggregating results before returning");
                 this.doingStorageAggregation = true;
-                result = new GTAggregateScanner(result, this, deadline);
+                result = new GTAggregateScanner(result, this, deadline, spillEnabled);
             } else {
                 logger.info("has no aggregation, skip it");
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java
index 4775ac6..30d3aaa 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java
@@ -18,6 +18,10 @@
 
 package org.apache.kylin.gridtable;
 
+/**
+ * Implementations of {@link IGTScanner} should throw {@link GTScanSelfTerminatedException} or its subclasses
+ * in cases where the scan runs out of resources (time, memory, etc) and can not be continued.
+ */
 public class GTScanSelfTerminatedException extends RuntimeException {
 
     public GTScanSelfTerminatedException(String s) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java
index 74c2659..7abb069 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java
@@ -84,7 +84,7 @@ public class AggregationCacheSpillTest extends LocalFileMetadataTestCase {
 
         GTScanRequest scanRequest = new GTScanRequestBuilder().setInfo(INFO).setRanges(null).setDimensions(new ImmutableBitSet(0, 3)).setAggrGroupBy(new ImmutableBitSet(0, 3)).setAggrMetrics(new ImmutableBitSet(3, 6)).setAggrMetricsFuncs(new String[] { "SUM", "SUM", "COUNT_DISTINCT" }).setFilterPushDown(null).setAggCacheMemThreshold(0.5).createGTScanRequest();
 
-        GTAggregateScanner scanner = new GTAggregateScanner(inputScanner, scanRequest, Long.MAX_VALUE);
+        GTAggregateScanner scanner = new GTAggregateScanner(inputScanner, scanRequest);
 
         int count = 0;
         for (GTRecord record : scanner) {
@@ -126,7 +126,7 @@ public class AggregationCacheSpillTest extends LocalFileMetadataTestCase {
         // all-in-mem testcase
         GTScanRequest scanRequest = new GTScanRequestBuilder().setInfo(INFO).setRanges(null).setDimensions(new ImmutableBitSet(0, 3)).setAggrGroupBy(new ImmutableBitSet(1, 3)).setAggrMetrics(new ImmutableBitSet(3, 6)).setAggrMetricsFuncs(new String[] { "SUM", "SUM", "COUNT_DISTINCT" }).setFilterPushDown(null).setAggCacheMemThreshold(0.5).createGTScanRequest();
 
-        GTAggregateScanner scanner = new GTAggregateScanner(inputScanner, scanRequest, Long.MAX_VALUE);
+        GTAggregateScanner scanner = new GTAggregateScanner(inputScanner, scanRequest);
 
         int count = 0;
         for (GTRecord record : scanner) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/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 cad5a3f..dd9f74c 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
@@ -163,6 +163,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
         if (queryId != null) {
             builder.setQueryId(queryId);
         }
+        builder.setSpillEnabled(cubeSeg.getConfig().getQueryCoprocessorSpillEnabled());
 
         for (final Pair<byte[], byte[]> epRange : getEPKeyRanges(cuboidBaseShard, shardNum, totalShards)) {
             executorService.submit(new Runnable() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
index 810747f..3e0a065 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
@@ -51,6 +51,7 @@ import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.gridtable.GTRecord;
 import org.apache.kylin.gridtable.GTScanExceedThresholdException;
 import org.apache.kylin.gridtable.GTScanRequest;
+import org.apache.kylin.gridtable.GTScanSelfTerminatedException;
 import org.apache.kylin.gridtable.GTScanTimeoutException;
 import org.apache.kylin.gridtable.IGTScanner;
 import org.apache.kylin.gridtable.IGTStore;
@@ -280,7 +281,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
             IGTStore store = new HBaseReadonlyStore(cellListIterator, scanReq, hbaseRawScans.get(0).hbaseColumns, hbaseColumnsToGT, request.getRowkeyPreambleSize(), behavior.delayToggledOn());
 
             IGTScanner rawScanner = store.scan(scanReq);
-            IGTScanner finalScanner = scanReq.decorateScanner(rawScanner, behavior.filterToggledOn(), behavior.aggrToggledOn(), deadline);
+            IGTScanner finalScanner = scanReq.decorateScanner(rawScanner, behavior.filterToggledOn(), behavior.aggrToggledOn(), false, deadline, request.getSpillEnabled());
 
             ByteBuffer buffer = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);
 
@@ -315,12 +316,11 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
                         break;
                     }
                 }
-            } catch (GTScanTimeoutException e) {
+            } catch (GTScanSelfTerminatedException e) {
+                // the query is using too much resource, we mark it as abnormal finish instead of
+                // throwing RuntimeException to avoid client retrying RPC.
                 scanNormalComplete.setValue(false);
-                logger.info("The cube visit did not finish normally because scan timeout", e);
-            } catch (GTScanExceedThresholdException e) {
-                scanNormalComplete.setValue(false);
-                logger.info("The cube visit did not finish normally because scan num exceeds threshold", e);
+                logger.warn("Abort scan: {}", e.getMessage());
             } finally {
                 finalScanner.close();
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
index 628d453..def0182 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
@@ -107,6 +107,16 @@ public final class CubeVisitProtos {
      */
     com.google.protobuf.ByteString
         getQueryIdBytes();
+
+    // optional bool spillEnabled = 7 [default = true];
+    /**
+     * <code>optional bool spillEnabled = 7 [default = true];</code>
+     */
+    boolean hasSpillEnabled();
+    /**
+     * <code>optional bool spillEnabled = 7 [default = true];</code>
+     */
+    boolean getSpillEnabled();
   }
   /**
    * Protobuf type {@code CubeVisitRequest}
@@ -192,6 +202,11 @@ public final class CubeVisitProtos {
               queryId_ = input.readBytes();
               break;
             }
+            case 56: {
+              bitField0_ |= 0x00000020;
+              spillEnabled_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -921,6 +936,22 @@ public final class CubeVisitProtos {
       }
     }
 
+    // optional bool spillEnabled = 7 [default = true];
+    public static final int SPILLENABLED_FIELD_NUMBER = 7;
+    private boolean spillEnabled_;
+    /**
+     * <code>optional bool spillEnabled = 7 [default = true];</code>
+     */
+    public boolean hasSpillEnabled() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    /**
+     * <code>optional bool spillEnabled = 7 [default = true];</code>
+     */
+    public boolean getSpillEnabled() {
+      return spillEnabled_;
+    }
+
     private void initFields() {
       gtScanRequest_ = com.google.protobuf.ByteString.EMPTY;
       hbaseRawScan_ = com.google.protobuf.ByteString.EMPTY;
@@ -928,6 +959,7 @@ public final class CubeVisitProtos {
       hbaseColumnsToGT_ = java.util.Collections.emptyList();
       kylinProperties_ = "";
       queryId_ = "";
+      spillEnabled_ = true;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -975,6 +1007,9 @@ public final class CubeVisitProtos {
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         output.writeBytes(6, getQueryIdBytes());
       }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeBool(7, spillEnabled_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -1008,6 +1043,10 @@ public final class CubeVisitProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBytesSize(6, getQueryIdBytes());
       }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(7, spillEnabled_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -1058,6 +1097,11 @@ public final class CubeVisitProtos {
         result = result && getQueryId()
             .equals(other.getQueryId());
       }
+      result = result && (hasSpillEnabled() == other.hasSpillEnabled());
+      if (hasSpillEnabled()) {
+        result = result && (getSpillEnabled()
+            == other.getSpillEnabled());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -1095,6 +1139,10 @@ public final class CubeVisitProtos {
         hash = (37 * hash) + QUERYID_FIELD_NUMBER;
         hash = (53 * hash) + getQueryId().hashCode();
       }
+      if (hasSpillEnabled()) {
+        hash = (37 * hash) + SPILLENABLED_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getSpillEnabled());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -1221,6 +1269,8 @@ public final class CubeVisitProtos {
         bitField0_ = (bitField0_ & ~0x00000010);
         queryId_ = "";
         bitField0_ = (bitField0_ & ~0x00000020);
+        spillEnabled_ = true;
+        bitField0_ = (bitField0_ & ~0x00000040);
         return this;
       }
 
@@ -1278,6 +1328,10 @@ public final class CubeVisitProtos {
           to_bitField0_ |= 0x00000010;
         }
         result.queryId_ = queryId_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.spillEnabled_ = spillEnabled_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1339,6 +1393,9 @@ public final class CubeVisitProtos {
           queryId_ = other.queryId_;
           onChanged();
         }
+        if (other.hasSpillEnabled()) {
+          setSpillEnabled(other.getSpillEnabled());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -1899,6 +1956,39 @@ public final class CubeVisitProtos {
         return this;
       }
 
+      // optional bool spillEnabled = 7 [default = true];
+      private boolean spillEnabled_ = true;
+      /**
+       * <code>optional bool spillEnabled = 7 [default = true];</code>
+       */
+      public boolean hasSpillEnabled() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      /**
+       * <code>optional bool spillEnabled = 7 [default = true];</code>
+       */
+      public boolean getSpillEnabled() {
+        return spillEnabled_;
+      }
+      /**
+       * <code>optional bool spillEnabled = 7 [default = true];</code>
+       */
+      public Builder setSpillEnabled(boolean value) {
+        bitField0_ |= 0x00000040;
+        spillEnabled_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool spillEnabled = 7 [default = true];</code>
+       */
+      public Builder clearSpillEnabled() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        spillEnabled_ = true;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:CubeVisitRequest)
     }
 
@@ -4253,25 +4343,26 @@ public final class CubeVisitProtos {
     java.lang.String[] descriptorData = {
       "\npstorage-hbase/src/main/java/org/apache" +
       "/kylin/storage/hbase/cube/v2/coprocessor" +
-      "/endpoint/protobuf/CubeVisit.proto\"\323\001\n\020C" +
+      "/endpoint/protobuf/CubeVisit.proto\"\357\001\n\020C" +
       "ubeVisitRequest\022\025\n\rgtScanRequest\030\001 \002(\014\022\024" +
       "\n\014hbaseRawScan\030\002 \002(\014\022\032\n\022rowkeyPreambleSi" +
       "ze\030\003 \002(\005\0223\n\020hbaseColumnsToGT\030\004 \003(\0132\031.Cub" +
       "eVisitRequest.IntList\022\027\n\017kylinProperties" +
-      "\030\005 \002(\t\022\017\n\007queryId\030\006 \001(\t\032\027\n\007IntList\022\014\n\004in" +
-      "ts\030\001 \003(\005\"\321\002\n\021CubeVisitResponse\022\026\n\016compre" +
-      "ssedRows\030\001 \002(\014\022\'\n\005stats\030\002 \002(\0132\030.CubeVisi",
-      "tResponse.Stats\032\372\001\n\005Stats\022\030\n\020serviceStar" +
-      "tTime\030\001 \001(\003\022\026\n\016serviceEndTime\030\002 \001(\003\022\027\n\017s" +
-      "cannedRowCount\030\003 \001(\003\022\032\n\022aggregatedRowCou" +
-      "nt\030\004 \001(\003\022\025\n\rsystemCpuLoad\030\005 \001(\001\022\036\n\026freeP" +
-      "hysicalMemorySize\030\006 \001(\001\022\031\n\021freeSwapSpace" +
-      "Size\030\007 \001(\001\022\020\n\010hostname\030\010 \001(\t\022\016\n\006etcMsg\030\t" +
-      " \001(\t\022\026\n\016normalComplete\030\n \001(\0052F\n\020CubeVisi" +
-      "tService\0222\n\tvisitCube\022\021.CubeVisitRequest" +
-      "\032\022.CubeVisitResponseB`\nEorg.apache.kylin" +
-      ".storage.hbase.cube.v2.coprocessor.endpo",
-      "int.generatedB\017CubeVisitProtosH\001\210\001\001\240\001\001"
+      "\030\005 \002(\t\022\017\n\007queryId\030\006 \001(\t\022\032\n\014spillEnabled\030" +
+      "\007 \001(\010:\004true\032\027\n\007IntList\022\014\n\004ints\030\001 \003(\005\"\321\002\n" +
+      "\021CubeVisitResponse\022\026\n\016compressedRows\030\001 \002",
+      "(\014\022\'\n\005stats\030\002 \002(\0132\030.CubeVisitResponse.St" +
+      "ats\032\372\001\n\005Stats\022\030\n\020serviceStartTime\030\001 \001(\003\022" +
+      "\026\n\016serviceEndTime\030\002 \001(\003\022\027\n\017scannedRowCou" +
+      "nt\030\003 \001(\003\022\032\n\022aggregatedRowCount\030\004 \001(\003\022\025\n\r" +
+      "systemCpuLoad\030\005 \001(\001\022\036\n\026freePhysicalMemor" +
+      "ySize\030\006 \001(\001\022\031\n\021freeSwapSpaceSize\030\007 \001(\001\022\020" +
+      "\n\010hostname\030\010 \001(\t\022\016\n\006etcMsg\030\t \001(\t\022\026\n\016norm" +
+      "alComplete\030\n \001(\0052F\n\020CubeVisitService\0222\n\t" +
+      "visitCube\022\021.CubeVisitRequest\032\022.CubeVisit" +
+      "ResponseB`\nEorg.apache.kylin.storage.hba",
+      "se.cube.v2.coprocessor.endpoint.generate" +
+      "dB\017CubeVisitProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -4283,7 +4374,7 @@ public final class CubeVisitProtos {
           internal_static_CubeVisitRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_CubeVisitRequest_descriptor,
-              new java.lang.String[] { "GtScanRequest", "HbaseRawScan", "RowkeyPreambleSize", "HbaseColumnsToGT", "KylinProperties", "QueryId", });
+              new java.lang.String[] { "GtScanRequest", "HbaseRawScan", "RowkeyPreambleSize", "HbaseColumnsToGT", "KylinProperties", "QueryId", "SpillEnabled", });
           internal_static_CubeVisitRequest_IntList_descriptor =
             internal_static_CubeVisitRequest_descriptor.getNestedTypes().get(0);
           internal_static_CubeVisitRequest_IntList_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
index 1a56719..c7c2954 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
@@ -36,6 +36,7 @@ message CubeVisitRequest {
     repeated IntList hbaseColumnsToGT = 4;
     required string kylinProperties = 5; // kylin properties
     optional string queryId = 6;
+    optional bool spillEnabled = 7 [default = true];
     message IntList {
         repeated int32 ints = 1;
     }


[12/13] kylin git commit: KYLIN-2307 Create a branch for master with HBase 0.98 API

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
index 74ab017..a44de4f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
@@ -30,15 +30,14 @@ import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
@@ -69,7 +68,7 @@ public class HBaseResourceStore extends ResourceStore {
     final String tableNameBase;
     final String hbaseUrl;
 
-    Connection getConnection() throws IOException {
+    HConnection getConnection() throws IOException {
         return HBaseConnection.get(hbaseUrl);
     }
 
@@ -121,7 +120,7 @@ public class HBaseResourceStore extends ResourceStore {
         byte[] endRow = Bytes.toBytes(lookForPrefix);
         endRow[endRow.length - 1]++;
 
-        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
+        HTableInterface table = getConnection().getTable(getAllInOneTableName());
         Scan scan = new Scan(startRow, endRow);
         if ((filter != null && filter instanceof KeyOnlyFilter) == false) {
             scan.addColumn(B_FAMILY, B_COLUMN_TS);
@@ -238,12 +237,13 @@ public class HBaseResourceStore extends ResourceStore {
         IOUtils.copy(content, bout);
         bout.close();
 
-        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
+        HTableInterface table = getConnection().getTable(getAllInOneTableName());
         try {
             byte[] row = Bytes.toBytes(resPath);
             Put put = buildPut(resPath, ts, row, bout.toByteArray(), table);
 
             table.put(put);
+            table.flushCommits();
         } finally {
             IOUtils.closeQuietly(table);
         }
@@ -251,7 +251,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     @Override
     protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException {
-        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
+        HTableInterface table = getConnection().getTable(getAllInOneTableName());
         try {
             byte[] row = Bytes.toBytes(resPath);
             byte[] bOldTS = oldTS == 0 ? null : Bytes.toBytes(oldTS);
@@ -264,6 +264,8 @@ public class HBaseResourceStore extends ResourceStore {
                 throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + oldTS + ", but it is " + real);
             }
 
+            table.flushCommits();
+
             return newTS;
         } finally {
             IOUtils.closeQuietly(table);
@@ -272,7 +274,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     @Override
     protected void deleteResourceImpl(String resPath) throws IOException {
-        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
+        HTableInterface table = getConnection().getTable(getAllInOneTableName());
         try {
             boolean hdfsResourceExist = false;
             Result result = internalGetFromHTable(table, resPath, true, false);
@@ -285,6 +287,7 @@ public class HBaseResourceStore extends ResourceStore {
 
             Delete del = new Delete(Bytes.toBytes(resPath));
             table.delete(del);
+            table.flushCommits();
 
             if (hdfsResourceExist) { // remove hdfs cell value
                 Path redirectPath = bigCellHDFSPath(resPath);
@@ -305,7 +308,7 @@ public class HBaseResourceStore extends ResourceStore {
     }
 
     private Result getFromHTable(String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
-        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
+        HTableInterface table = getConnection().getTable(getAllInOneTableName());
         try {
             return internalGetFromHTable(table, path, fetchContent, fetchTimestamp);
         } finally {
@@ -314,7 +317,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     }
 
-    private Result internalGetFromHTable(Table table, String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
+    private Result internalGetFromHTable(HTableInterface table, String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
         byte[] rowkey = Bytes.toBytes(path);
 
         Get get = new Get(rowkey);
@@ -333,7 +336,7 @@ public class HBaseResourceStore extends ResourceStore {
         return exists ? result : null;
     }
 
-    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, Table table) throws IOException {
+    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, HTableInterface table) throws IOException {
         Path redirectPath = bigCellHDFSPath(resPath);
         FileSystem fileSystem = HadoopUtil.getWorkingFileSystem();
 
@@ -358,7 +361,7 @@ public class HBaseResourceStore extends ResourceStore {
         return redirectPath;
     }
 
-    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, Table table) throws IOException {
+    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, HTableInterface table) throws IOException {
         int kvSizeLimit = Integer.parseInt(getConnection().getConfiguration().get("hbase.client.keyvalue.maxsize", "10485760"));
         if (content.length > kvSizeLimit) {
             writeLargeCellToHdfs(resPath, content, table);
@@ -366,8 +369,8 @@ public class HBaseResourceStore extends ResourceStore {
         }
 
         Put put = new Put(row);
-        put.addColumn(B_FAMILY, B_COLUMN, content);
-        put.addColumn(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
+        put.add(B_FAMILY, B_COLUMN, content);
+        put.add(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
 
         return put;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
index f63d9c2..b141190 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
@@ -26,13 +26,12 @@ import java.util.NoSuchElementException;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.BufferedMutator;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.kv.RowConstants;
@@ -87,13 +86,14 @@ public class SimpleHBaseStore implements IGTStore {
     }
 
     private class Writer implements IGTWriter {
-        final BufferedMutator table;
+        final HTableInterface table;
         final ByteBuffer rowkey = ByteBuffer.allocate(50);
         final ByteBuffer value = ByteBuffer.allocate(50);
 
         Writer() throws IOException {
-            Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
-            table = conn.getBufferedMutator(htableName);
+            HConnection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+            table = conn.getTable(htableName);
+            table.setAutoFlush(false, true);
         }
 
         @Override
@@ -113,24 +113,24 @@ public class SimpleHBaseStore implements IGTStore {
 
             Put put = new Put(rowkey);
             put.addImmutable(CF_B, ByteBuffer.wrap(COL_B), HConstants.LATEST_TIMESTAMP, value);
-            table.mutate(put);
+            table.put(put);
         }
 
         @Override
         public void close() throws IOException {
-            table.flush();
+            table.flushCommits();
             table.close();
         }
     }
 
     class Reader implements IGTScanner {
-        final Table table;
+        final HTableInterface table;
         final ResultScanner scanner;
 
         int count = 0;
 
         Reader() throws IOException {
-            Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+            HConnection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
             table = conn.getTable(htableName);
 
             Scan scan = new Scan();

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/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 3c01da2..6384b77 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
@@ -27,9 +27,8 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.zip.DataFormatException;
 
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.HConnection;
+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;
@@ -53,10 +52,10 @@ import org.apache.kylin.storage.gtrecord.StorageResponseGTScatter;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos;
 import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitRequest;
-import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitRequest.IntList;
 import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse;
-import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.Stats;
 import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitService;
+import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitRequest.IntList;
+import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.Stats;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -118,7 +117,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
         final ImmutableBitSet selectedColBlocks = scanRequest.getSelectedColBlocks().set(0);
 
         // globally shared connection, does not require close
-        final Connection conn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
+        final HConnection conn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
 
         final List<IntList> hbaseColumnsToGTIntList = Lists.newArrayList();
         List<List<Integer>> hbaseColumnsToGT = getHBaseColumnsGTMapping(selectedColBlocks);
@@ -175,7 +174,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
                     final AtomicReference<RuntimeException> regionErrorHolder = new AtomicReference<>();
 
                     try {
-                        Table table = conn.getTable(TableName.valueOf(cubeSeg.getStorageLocationIdentifier()), HBaseConnection.getCoprocessorPool());
+                        HTableInterface table = conn.getTable(cubeSeg.getStorageLocationIdentifier(), HBaseConnection.getCoprocessorPool());
 
                         final CubeVisitRequest request = builder.build();
                         final byte[] startKey = epRange.getFirst();

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
index 1698180..b32f6b1 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
@@ -25,12 +25,11 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.ShardingHash;
@@ -156,8 +155,8 @@ public class CubeHBaseScanRPC extends CubeHBaseRPC {
         // primary key (also the 0th column block) is always selected
         final ImmutableBitSet selectedColBlocks = scanRequest.getSelectedColBlocks().set(0);
         // globally shared connection, does not require close
-        Connection hbaseConn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
-        final Table hbaseTable = hbaseConn.getTable(TableName.valueOf(cubeSeg.getStorageLocationIdentifier()));
+        HConnection hbaseConn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
+        final HTableInterface hbaseTable = hbaseConn.getTable(cubeSeg.getStorageLocationIdentifier());
 
         List<RawScan> rawScans = preparedHBaseScans(scanRequest.getGTScanRanges(), selectedColBlocks);
         List<List<Integer>> hbaseColumnsToGT = getHBaseColumnsGTMapping(selectedColBlocks);

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
index e18ff0d..743206f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
@@ -197,7 +197,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
         if (shardLength == 0) {
             return;
         }
-        byte[] regionStartKey = ArrayUtils.isEmpty(region.getRegionInfo().getStartKey()) ? new byte[shardLength] : region.getRegionInfo().getStartKey();
+        byte[] regionStartKey = ArrayUtils.isEmpty(region.getStartKey()) ? new byte[shardLength] : region.getStartKey();
         Bytes.putBytes(rawScan.startKey, 0, regionStartKey, 0, shardLength);
         Bytes.putBytes(rawScan.endKey, 0, regionStartKey, 0, shardLength);
     }
@@ -236,7 +236,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
         try (SetThreadName ignored = new SetThreadName("Query %s", queryId)) {
             this.serviceStartTime = System.currentTimeMillis();
 
-            region = (HRegion)env.getRegion();
+            region = env.getRegion();
             region.startRegionOperation();
 
             // if user change kylin.properties on kylin server, need to manually redeploy coprocessor jar to update KylinConfig of Env.

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
index feb4842..2814ad6 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
@@ -26,8 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.regionserver.BloomType;
@@ -80,8 +79,7 @@ public class CubeHTableUtil {
         tableDesc.setValue(IRealizationConstants.HTableSegmentTag, cubeSegment.toString());
 
         Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
-        Admin admin = conn.getAdmin();
+        HBaseAdmin admin = new HBaseAdmin(conf);
 
         try {
             if (User.isHBaseSecurityEnabled(conf)) {
@@ -94,7 +92,7 @@ public class CubeHTableUtil {
                 tableDesc.addFamily(cf);
             }
 
-            if (admin.tableExists(TableName.valueOf(tableName))) {
+            if (admin.tableExists(tableName)) {
                 // admin.disableTable(tableName);
                 // admin.deleteTable(tableName);
                 throw new RuntimeException("HBase table " + tableName + " exists!");
@@ -103,7 +101,7 @@ public class CubeHTableUtil {
             DeployCoprocessorCLI.deployCoprocessor(tableDesc);
 
             admin.createTable(tableDesc, splitKeys);
-            Preconditions.checkArgument(admin.isTableAvailable(TableName.valueOf(tableName)), "table " + tableName + " created, but is not available due to some reasons");
+            Preconditions.checkArgument(admin.isTableAvailable(tableName), "table " + tableName + " created, but is not available due to some reasons");
             logger.info("create hbase table " + tableName + " done.");
         } finally {
             IOUtils.closeQuietly(admin);
@@ -112,7 +110,8 @@ public class CubeHTableUtil {
     }
 
     public static void deleteHTable(TableName tableName) throws IOException {
-        Admin admin = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getAdmin();
+        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
+        HBaseAdmin admin = new HBaseAdmin(conf);
         try {
             if (admin.tableExists(tableName)) {
                 logger.info("disabling hbase table " + tableName);
@@ -127,7 +126,8 @@ public class CubeHTableUtil {
 
     /** create a HTable that has the same performance settings as normal cube table, for benchmark purpose */
     public static void createBenchmarkHTable(TableName tableName, String cfName) throws IOException {
-        Admin admin = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getAdmin();
+        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
+        HBaseAdmin admin = new HBaseAdmin(conf);
         try {
             if (admin.tableExists(tableName)) {
                 logger.info("disabling hbase table " + tableName);

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
index df3cf08..eacff9f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
@@ -25,13 +25,13 @@ import java.util.List;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.common.util.HiveCmdBuilder;
 import org.apache.kylin.job.exception.ExecuteException;
@@ -100,21 +100,19 @@ public class DeprecatedGCStep extends AbstractExecutable {
         List<String> oldTables = getOldHTables();
         if (oldTables != null && oldTables.size() > 0) {
             String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-            Admin admin = null;
+            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
+            HBaseAdmin admin = null;
             try {
-
-                Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
-                admin = conn.getAdmin();
-
+                admin = new HBaseAdmin(conf);
                 for (String table : oldTables) {
-                    if (admin.tableExists(TableName.valueOf(table))) {
-                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(TableName.valueOf(table));
+                    if (admin.tableExists(table)) {
+                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
                         String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
                         if (metadataUrlPrefix.equalsIgnoreCase(host)) {
-                            if (admin.isTableEnabled(TableName.valueOf(table))) {
-                                admin.disableTable(TableName.valueOf(table));
+                            if (admin.isTableEnabled(table)) {
+                                admin.disableTable(table);
                             }
-                            admin.deleteTable(TableName.valueOf(table));
+                            admin.deleteTable(table);
                             logger.debug("Dropped HBase table " + table);
                             output.append("Dropped HBase table " + table + " \n");
                         } else {

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
index 6587d4e..d5b36df 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
@@ -23,8 +23,8 @@ import java.util.List;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
@@ -49,7 +49,7 @@ public class HBaseCuboidWriter implements ICuboidWriter {
 
     private final List<KeyValueCreator> keyValueCreators;
     private final int nColumns;
-    private final Table hTable;
+    private final HTableInterface hTable;
     private final CubeDesc cubeDesc;
     private final CubeSegment cubeSegment;
     private final Object[] measureValues;
@@ -58,7 +58,7 @@ public class HBaseCuboidWriter implements ICuboidWriter {
     private AbstractRowKeyEncoder rowKeyEncoder;
     private byte[] keybuf;
 
-    public HBaseCuboidWriter(CubeSegment segment, Table hTable) {
+    public HBaseCuboidWriter(CubeSegment segment, HTableInterface hTable) {
         this.keyValueCreators = Lists.newArrayList();
         this.cubeSegment = segment;
         this.cubeDesc = cubeSegment.getCubeDesc();
@@ -117,6 +117,7 @@ public class HBaseCuboidWriter implements ICuboidWriter {
             long t = System.currentTimeMillis();
             if (hTable != null) {
                 hTable.put(puts);
+                hTable.flushCommits();
             }
             logger.info("commit total " + puts.size() + " puts, totally cost:" + (System.currentTimeMillis() - t) + "ms");
             puts.clear();

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
index 2f7e164..5b2441c 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
@@ -24,11 +24,11 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.job.exception.ExecuteException;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableContext;
@@ -69,20 +69,19 @@ public class MergeGCStep extends AbstractExecutable {
         List<String> oldTables = getOldHTables();
         if (oldTables != null && oldTables.size() > 0) {
             String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-            Admin admin = null;
+            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
+            HBaseAdmin admin = null;
             try {
-                Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
-                admin = conn.getAdmin();
-
+                admin = new HBaseAdmin(conf);
                 for (String table : oldTables) {
-                    if (admin.tableExists(TableName.valueOf(table))) {
-                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(TableName.valueOf((table)));
+                    if (admin.tableExists(table)) {
+                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
                         String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
                         if (metadataUrlPrefix.equalsIgnoreCase(host)) {
-                            if (admin.isTableEnabled(TableName.valueOf(table))) {
-                                admin.disableTable(TableName.valueOf(table));
+                            if (admin.isTableEnabled(table)) {
+                                admin.disableTable(table);
                             }
-                            admin.deleteTable(TableName.valueOf(table));
+                            admin.deleteTable(table);
                             logger.debug("Dropped htable: " + table);
                             output.append("HBase table " + table + " is dropped. \n");
                         } else {

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
index 56f867a..a150607 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
@@ -21,11 +21,9 @@ package org.apache.kylin.storage.hbase.util;
 import java.io.IOException;
 
 import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.kylin.common.KylinConfig;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
@@ -40,8 +38,8 @@ public class CleanHtableCLI extends AbstractApplication {
     protected static final Logger logger = LoggerFactory.getLogger(CleanHtableCLI.class);
 
     private void clean() throws IOException {
-        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
-        Admin hbaseAdmin = conn.getAdmin();
+        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
+        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
 
         for (HTableDescriptor descriptor : hbaseAdmin.listTables()) {
             String name = descriptor.getNameAsString().toLowerCase();
@@ -52,7 +50,7 @@ public class CleanHtableCLI extends AbstractApplication {
                 System.out.println();
 
                 descriptor.setValue(IRealizationConstants.HTableOwner, "DL-eBay-Kylin@ebay.com");
-                hbaseAdmin.modifyTable(TableName.valueOf(descriptor.getNameAsString()), descriptor);
+                hbaseAdmin.modifyTable(descriptor.getNameAsString(), descriptor);
             }
         }
         hbaseAdmin.close();

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
index 581de38..68c0a39 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
@@ -26,19 +26,19 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.RawResource;
@@ -89,7 +89,7 @@ public class CubeMigrationCLI {
     private static ResourceStore srcStore;
     private static ResourceStore dstStore;
     private static FileSystem hdfsFS;
-    private static Admin hbaseAdmin;
+    private static HBaseAdmin hbaseAdmin;
 
     public static final String ACL_INFO_FAMILY = "i";
     private static final String ACL_TABLE_NAME = "_acl";
@@ -134,8 +134,8 @@ public class CubeMigrationCLI {
 
         checkAndGetHbaseUrl();
 
-        Connection conn = HBaseConnection.get(srcConfig.getStorageUrl());
-        hbaseAdmin = conn.getAdmin();
+        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
+        hbaseAdmin = new HBaseAdmin(conf);
 
         hdfsFS = HadoopUtil.getWorkingFileSystem();
 
@@ -233,7 +233,6 @@ public class CubeMigrationCLI {
             operations.add(new Opt(OptType.COPY_DICT_OR_SNAPSHOT, new Object[] { item, cube.getName() }));
         }
     }
-
     private static void addCubeAndModelIntoProject(CubeInstance srcCube, String cubeName, String projectName) throws IOException {
         String projectResPath = ProjectInstance.concatResourcePath(projectName);
         if (!dstStore.exists(projectResPath))
@@ -327,8 +326,8 @@ public class CubeMigrationCLI {
 
         switch (opt.type) {
         case CHANGE_HTABLE_HOST: {
-            TableName tableName = TableName.valueOf((String) opt.params[0]);
-            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(tableName);
+            String tableName = (String) opt.params[0];
+            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
             hbaseAdmin.disableTable(tableName);
             desc.setValue(IRealizationConstants.HTableTag, dstConfig.getMetadataUrlPrefix());
             hbaseAdmin.modifyTable(tableName, desc);
@@ -450,11 +449,11 @@ public class CubeMigrationCLI {
             Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
             ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
             String projUUID = project.getUuid();
-            Table srcAclHtable = null;
-            Table destAclHtable = null;
+            HTableInterface srcAclHtable = null;
+            HTableInterface destAclHtable = null;
             try {
-                srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(TableName.valueOf(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
-                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
+                srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
 
                 // cube acl
                 Result result = srcAclHtable.get(new Get(Bytes.toBytes(cubeId)));
@@ -474,6 +473,7 @@ public class CubeMigrationCLI {
                         destAclHtable.put(put);
                     }
                 }
+                destAclHtable.flushCommits();
             } finally {
                 IOUtils.closeQuietly(srcAclHtable);
                 IOUtils.closeQuietly(destAclHtable);
@@ -504,8 +504,8 @@ public class CubeMigrationCLI {
 
         switch (opt.type) {
         case CHANGE_HTABLE_HOST: {
-            TableName tableName = TableName.valueOf((String) opt.params[0]);
-            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(tableName);
+            String tableName = (String) opt.params[0];
+            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
             hbaseAdmin.disableTable(tableName);
             desc.setValue(IRealizationConstants.HTableTag, srcConfig.getMetadataUrlPrefix());
             hbaseAdmin.modifyTable(tableName, desc);
@@ -539,12 +539,13 @@ public class CubeMigrationCLI {
         case COPY_ACL: {
             String cubeId = (String) opt.params[0];
             String modelId = (String) opt.params[1];
-            Table destAclHtable = null;
+            HTableInterface destAclHtable = null;
             try {
-                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
+                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
 
                 destAclHtable.delete(new Delete(Bytes.toBytes(cubeId)));
                 destAclHtable.delete(new Delete(Bytes.toBytes(modelId)));
+                destAclHtable.flushCommits();
             } finally {
                 IOUtils.closeQuietly(destAclHtable);
             }
@@ -561,7 +562,7 @@ public class CubeMigrationCLI {
         }
     }
 
-    private static void updateMeta(KylinConfig config) {
+    private static void updateMeta(KylinConfig config){
         String[] nodes = config.getRestServers();
         for (String node : nodes) {
             RestClient restClient = new RestClient(node);

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
index 20d0f7d..8bd4abf 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
@@ -26,10 +26,10 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.cube.CubeInstance;
@@ -61,7 +61,7 @@ public class CubeMigrationCheckCLI {
     private static final Option OPTION_CUBE = OptionBuilder.withArgName("cube").hasArg().isRequired(false).withDescription("The name of cube migrated").create("cube");
 
     private KylinConfig dstCfg;
-    private Admin hbaseAdmin;
+    private HBaseAdmin hbaseAdmin;
 
     private List<String> issueExistHTables;
     private List<String> inconsistentHTables;
@@ -130,8 +130,9 @@ public class CubeMigrationCheckCLI {
         this.dstCfg = kylinConfig;
         this.ifFix = isFix;
 
-        Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
-        hbaseAdmin = conn.getAdmin();
+        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
+        hbaseAdmin = new HBaseAdmin(conf);
+
         issueExistHTables = Lists.newArrayList();
         inconsistentHTables = Lists.newArrayList();
     }
@@ -188,10 +189,10 @@ public class CubeMigrationCheckCLI {
                 String[] sepNameList = segFullName.split(",");
                 HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(sepNameList[0]));
                 logger.info("Change the host of htable " + sepNameList[0] + "belonging to cube " + sepNameList[1] + " from " + desc.getValue(IRealizationConstants.HTableTag) + " to " + dstCfg.getMetadataUrlPrefix());
-                hbaseAdmin.disableTable(TableName.valueOf(sepNameList[0]));
+                hbaseAdmin.disableTable(sepNameList[0]);
                 desc.setValue(IRealizationConstants.HTableTag, dstCfg.getMetadataUrlPrefix());
-                hbaseAdmin.modifyTable(TableName.valueOf(sepNameList[0]), desc);
-                hbaseAdmin.enableTable(TableName.valueOf(sepNameList[0]));
+                hbaseAdmin.modifyTable(sepNameList[0], desc);
+                hbaseAdmin.enableTable(sepNameList[0]);
             }
         } else {
             logger.info("------ Inconsistent HTables Needed To Be Fixed ------");

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
index e72859d..c8410f9 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
@@ -44,8 +44,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.KylinVersion;
@@ -82,8 +81,7 @@ public class DeployCoprocessorCLI {
         KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
         Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
         FileSystem fileSystem = FileSystem.get(hconf);
-        Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
-        Admin hbaseAdmin = conn.getAdmin();
+        HBaseAdmin hbaseAdmin = new HBaseAdmin(hconf);
 
         String localCoprocessorJar;
         if ("default".equals(args[0])) {
@@ -167,10 +165,10 @@ public class DeployCoprocessorCLI {
     public static void deployCoprocessor(HTableDescriptor tableDesc) {
         try {
             initHTableCoprocessor(tableDesc);
-            logger.info("hbase table " + tableDesc.getTableName() + " deployed with coprocessor.");
+            logger.info("hbase table " + tableDesc.getName() + " deployed with coprocessor.");
 
         } catch (Exception ex) {
-            logger.error("Error deploying coprocessor on " + tableDesc.getTableName(), ex);
+            logger.error("Error deploying coprocessor on " + tableDesc.getName(), ex);
             logger.error("Will try creating the table without coprocessor.");
         }
     }
@@ -191,7 +189,7 @@ public class DeployCoprocessorCLI {
         desc.addCoprocessor(CubeEndpointClass, hdfsCoprocessorJar, 1001, null);
     }
 
-    public static boolean resetCoprocessor(String tableName, Admin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
+    public static boolean resetCoprocessor(String tableName, HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
         KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
         HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
 
@@ -206,7 +204,7 @@ public class DeployCoprocessorCLI {
         logger.info("reset coprocessor on " + tableName);
 
         logger.info("Disable " + tableName);
-        hbaseAdmin.disableTable(TableName.valueOf(tableName));
+        hbaseAdmin.disableTable(tableName);
 
         while (desc.hasCoprocessor(CubeObserverClassOld2)) {
             desc.removeCoprocessor(CubeObserverClassOld2);
@@ -232,15 +230,16 @@ public class DeployCoprocessorCLI {
             desc.setValue(IRealizationConstants.HTableGitTag, commitInfo);
         }
 
-        hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+        hbaseAdmin.modifyTable(tableName, desc);
 
         logger.info("Enable " + tableName);
-        hbaseAdmin.enableTable(TableName.valueOf(tableName));
+        hbaseAdmin.enableTable(tableName);
 
         return true;
     }
 
-    private static List<String> resetCoprocessorOnHTables(final Admin hbaseAdmin, final Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
+
+    private static List<String> resetCoprocessorOnHTables(final HBaseAdmin hbaseAdmin, final Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
         List<String> processedTables = Collections.synchronizedList(new ArrayList<String>());
         ExecutorService coprocessorPool = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors() * 2);
         CountDownLatch countDownLatch = new CountDownLatch(tableNames.size());
@@ -261,12 +260,12 @@ public class DeployCoprocessorCLI {
 
     private static class ResetCoprocessorWorker implements Runnable {
         private final CountDownLatch countDownLatch;
-        private final Admin hbaseAdmin;
+        private final HBaseAdmin hbaseAdmin;
         private final Path hdfsCoprocessorJar;
         private final String tableName;
         private final List<String> processedTables;
 
-        public ResetCoprocessorWorker(CountDownLatch countDownLatch, Admin hbaseAdmin, Path hdfsCoprocessorJar, String tableName, List<String> processedTables) {
+        public ResetCoprocessorWorker(CountDownLatch countDownLatch, HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar, String tableName, List<String> processedTables) {
             this.countDownLatch = countDownLatch;
             this.hbaseAdmin = hbaseAdmin;
             this.hdfsCoprocessorJar = hdfsCoprocessorJar;
@@ -387,7 +386,7 @@ public class DeployCoprocessorCLI {
         return coprocessorDir;
     }
 
-    private static Set<String> getCoprocessorJarPaths(Admin hbaseAdmin, List<String> tableNames) throws IOException {
+    private static Set<String> getCoprocessorJarPaths(HBaseAdmin hbaseAdmin, List<String> tableNames) throws IOException {
         HashSet<String> result = new HashSet<String>();
 
         for (String tableName : tableNames) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
index 1cdb2f8..61c73d5 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
@@ -25,11 +25,10 @@ import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
@@ -236,9 +235,9 @@ public class ExtendCubeToHybridCLI {
         Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
         ProjectInstance project = store.getResource(projectResPath, ProjectInstance.class, projectSerializer);
         String projUUID = project.getUuid();
-        Table aclHtable = null;
+        HTableInterface aclHtable = null;
         try {
-            aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(TableName.valueOf(kylinConfig.getMetadataUrlPrefix() + "_acl"));
+            aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(kylinConfig.getMetadataUrlPrefix() + "_acl");
 
             // cube acl
             Result result = aclHtable.get(new Get(Bytes.toBytes(origCubeId)));
@@ -258,6 +257,7 @@ public class ExtendCubeToHybridCLI {
                     aclHtable.put(put);
                 }
             }
+            aclHtable.flushCommits();
         } finally {
             IOUtils.closeQuietly(aclHtable);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
index dd5f8fa..86ba22f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
@@ -28,13 +28,13 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.Pair;
@@ -75,7 +75,7 @@ public class GridTableHBaseBenchmark {
         System.out.println("Testing grid table scanning, hit ratio " + hitRatio + ", index ratio " + indexRatio);
         String hbaseUrl = "hbase"; // use hbase-site.xml on classpath
 
-        Connection conn = HBaseConnection.get(hbaseUrl);
+        HConnection conn = HBaseConnection.get(hbaseUrl);
         createHTableIfNeeded(conn, TEST_TABLE);
         prepareData(conn);
 
@@ -91,10 +91,10 @@ public class GridTableHBaseBenchmark {
 
     }
 
-    private static void testColumnScan(Connection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
+    private static void testColumnScan(HConnection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
         Stats stats = new Stats("COLUMN_SCAN");
 
-        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
+        HTableInterface table = conn.getTable(TEST_TABLE);
         try {
             stats.markStart();
 
@@ -122,20 +122,20 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void testRowScanNoIndexFullScan(Connection conn, boolean[] hits) throws IOException {
+    private static void testRowScanNoIndexFullScan(HConnection conn, boolean[] hits) throws IOException {
         fullScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_FULL"));
     }
 
-    private static void testRowScanNoIndexSkipScan(Connection conn, boolean[] hits) throws IOException {
+    private static void testRowScanNoIndexSkipScan(HConnection conn, boolean[] hits) throws IOException {
         jumpScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_SKIP"));
     }
 
-    private static void testRowScanWithIndex(Connection conn, boolean[] hits) throws IOException {
+    private static void testRowScanWithIndex(HConnection conn, boolean[] hits) throws IOException {
         jumpScan(conn, hits, new Stats("ROW_SCAN_IDX"));
     }
 
-    private static void fullScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
-        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
+    private static void fullScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
+        HTableInterface table = conn.getTable(TEST_TABLE);
         try {
             stats.markStart();
 
@@ -156,11 +156,11 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void jumpScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
+    private static void jumpScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
 
         final int jumpThreshold = 6; // compensate for Scan() overhead, totally by experience
 
-        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
+        HTableInterface table = conn.getTable(TEST_TABLE);
         try {
 
             stats.markStart();
@@ -204,8 +204,8 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void prepareData(Connection conn) throws IOException {
-        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
+    private static void prepareData(HConnection conn) throws IOException {
+        HTableInterface table = conn.getTable(TEST_TABLE);
 
         try {
             // check how many rows existing
@@ -258,8 +258,8 @@ public class GridTableHBaseBenchmark {
         return bytes;
     }
 
-    private static void createHTableIfNeeded(Connection conn, String tableName) throws IOException {
-        Admin hbase = conn.getAdmin();
+    private static void createHTableIfNeeded(HConnection conn, String tableName) throws IOException {
+        HBaseAdmin hbase = new HBaseAdmin(conn);
 
         try {
             boolean tableExist = false;

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
index 940d64a..6749d6c 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
@@ -24,11 +24,9 @@ import java.util.List;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.kylin.common.KylinConfig;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
@@ -57,8 +55,8 @@ public class HBaseClean extends AbstractApplication {
     private void cleanUp() {
         try {
             // get all kylin hbase tables
-            Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
-            Admin hbaseAdmin = conn.getAdmin();
+            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
+            HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
             String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
             HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
             List<String> allTablesNeedToBeDropped = Lists.newArrayList();
@@ -73,12 +71,12 @@ public class HBaseClean extends AbstractApplication {
                 // drop tables
                 for (String htableName : allTablesNeedToBeDropped) {
                     logger.info("Deleting HBase table " + htableName);
-                    if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
-                        if (hbaseAdmin.isTableEnabled(TableName.valueOf(htableName))) {
-                            hbaseAdmin.disableTable(TableName.valueOf(htableName));
+                    if (hbaseAdmin.tableExists(htableName)) {
+                        if (hbaseAdmin.isTableEnabled(htableName)) {
+                            hbaseAdmin.disableTable(htableName);
                         }
 
-                        hbaseAdmin.deleteTable(TableName.valueOf(htableName));
+                        hbaseAdmin.deleteTable(htableName);
                         logger.info("Deleted HBase table " + htableName);
                     } else {
                         logger.info("HBase table" + htableName + " does not exist");

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
index 1daca0a..937b65f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
@@ -32,15 +31,12 @@ import java.util.TreeSet;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.kylin.common.util.Pair;
 import org.slf4j.Logger;
@@ -62,31 +58,30 @@ public class HBaseRegionSizeCalculator {
     /**
      * Computes size of each region for table and given column families.
      * */
-    public HBaseRegionSizeCalculator(String tableName, Connection hbaseConnection) throws IOException {
+    public HBaseRegionSizeCalculator(HTable table) throws IOException {
+        this(table, new HBaseAdmin(table.getConfiguration()));
+    }
 
-        Table table = null;
-        Admin admin = null;
-        try {
-            table = hbaseConnection.getTable(TableName.valueOf(tableName));
-            admin = hbaseConnection.getAdmin();
+    /** Constructor for unit testing */
+    HBaseRegionSizeCalculator(HTable table, HBaseAdmin hBaseAdmin) throws IOException {
 
+        try {
             if (!enabled(table.getConfiguration())) {
                 logger.info("Region size calculation disabled.");
                 return;
             }
 
-            logger.info("Calculating region sizes for table \"" + table.getName() + "\".");
+            logger.info("Calculating region sizes for table \"" + new String(table.getTableName()) + "\".");
 
             // Get regions for table.
-            RegionLocator regionLocator = hbaseConnection.getRegionLocator(table.getName());
-            List<HRegionLocation> regionLocationList = regionLocator.getAllRegionLocations();
+            Set<HRegionInfo> tableRegionInfos = table.getRegionLocations().keySet();
             Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
 
-            for (HRegionLocation hRegionLocation : regionLocationList) {
-                tableRegions.add(hRegionLocation.getRegionInfo().getRegionName());
+            for (HRegionInfo regionInfo : tableRegionInfos) {
+                tableRegions.add(regionInfo.getRegionName());
             }
 
-            ClusterStatus clusterStatus = admin.getClusterStatus();
+            ClusterStatus clusterStatus = hBaseAdmin.getClusterStatus();
             Collection<ServerName> servers = clusterStatus.getServers();
             final long megaByte = 1024L * 1024L;
 
@@ -110,7 +105,7 @@ public class HBaseRegionSizeCalculator {
                 }
             }
         } finally {
-            IOUtils.closeQuietly(admin);
+            IOUtils.closeQuietly(hBaseAdmin);
         }
 
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
index a2f60d4..266f7e7 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
@@ -23,10 +23,9 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.kylin.common.KylinConfig;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 
@@ -43,8 +42,8 @@ public class HBaseUsage {
         Map<String, List<String>> envs = Maps.newHashMap();
 
         // get all kylin hbase tables
-        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
-        Admin hbaseAdmin = conn.getAdmin();
+        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
+        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         for (HTableDescriptor desc : tableDescriptors) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
index 8dd2164..1db60fb 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
@@ -32,15 +32,15 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
-import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
@@ -58,11 +58,11 @@ public class HbaseStreamingInput {
     private static final byte[] QN = "C".getBytes();
 
     public static void createTable(String tableName) throws IOException {
-        Connection conn = getConnection();
-        Admin hadmin = conn.getAdmin();
+        HConnection conn = getConnection();
+        HBaseAdmin hadmin = new HBaseAdmin(conn);
 
         try {
-            boolean tableExist = hadmin.tableExists(TableName.valueOf(tableName));
+            boolean tableExist = hadmin.tableExists(tableName);
             if (tableExist) {
                 logger.info("HTable '" + tableName + "' already exists");
                 return;
@@ -120,8 +120,8 @@ public class HbaseStreamingInput {
                 e.printStackTrace();
             }
 
-            Connection conn = getConnection();
-            Table table = conn.getTable(TableName.valueOf(tableName));
+            HConnection conn = getConnection();
+            HTableInterface table = conn.getTable(tableName);
 
             byte[] key = new byte[8 + 4];//time + id
 
@@ -136,7 +136,7 @@ public class HbaseStreamingInput {
                 Bytes.putInt(key, 8, i);
                 Put put = new Put(key);
                 byte[] cell = randomBytes(CELL_SIZE);
-                put.addColumn(CF, QN, cell);
+                put.add(CF, QN, cell);
                 buffer.add(put);
             }
             table.put(buffer);
@@ -172,8 +172,8 @@ public class HbaseStreamingInput {
             }
 
             Random r = new Random();
-            Connection conn = getConnection();
-            Table table = conn.getTable(TableName.valueOf(tableName));
+            HConnection conn = getConnection();
+            HTableInterface table = conn.getTable(tableName);
 
             long leftBound = getFirstKeyTime(table);
             long rightBound = System.currentTimeMillis();
@@ -208,7 +208,7 @@ public class HbaseStreamingInput {
         }
     }
 
-    private static long getFirstKeyTime(Table table) throws IOException {
+    private static long getFirstKeyTime(HTableInterface table) throws IOException {
         long startTime = 0;
 
         Scan scan = new Scan();
@@ -226,8 +226,8 @@ public class HbaseStreamingInput {
 
     }
 
-    private static Connection getConnection() throws IOException {
-        return HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+    private static HConnection getConnection() throws IOException {
+        return HConnectionManager.createConnection(HBaseConnection.getCurrentHBaseConfiguration());
     }
 
     private static String formatTime(long time) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
index ea05ab2..ca1a060 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
@@ -23,11 +23,10 @@ import java.io.IOException;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.kylin.common.KylinConfig;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.engine.mr.common.BatchConstants;
@@ -51,8 +50,8 @@ public class HtableAlterMetadataCLI extends AbstractApplication {
     String metadataValue;
 
     private void alter() throws IOException {
-        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
-        Admin hbaseAdmin = conn.getAdmin();
+        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
+        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
         HTableDescriptor table = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
 
         hbaseAdmin.disableTable(table.getTableName());

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
index df4e912..8ff5b0f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
@@ -30,14 +30,10 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.kylin.common.KylinConfig;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
-import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -56,9 +52,9 @@ public class OrphanHBaseCleanJob extends AbstractApplication {
     Set<String> metastoreWhitelistSet = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
 
     private void cleanUnusedHBaseTables(Configuration conf) throws IOException {
-        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+
         // get all kylin hbase tables
-        Admin hbaseAdmin = conn.getAdmin();
+        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         List<String> allTablesNeedToBeDropped = new ArrayList<String>();
@@ -77,13 +73,12 @@ public class OrphanHBaseCleanJob extends AbstractApplication {
             // drop tables
             for (String htableName : allTablesNeedToBeDropped) {
                 logger.info("Deleting HBase table " + htableName);
-                TableName tableName = TableName.valueOf(htableName);
-                if (hbaseAdmin.tableExists(tableName)) {
-                    if (hbaseAdmin.isTableEnabled(tableName)) {
-                        hbaseAdmin.disableTable(tableName);
+                if (hbaseAdmin.tableExists(htableName)) {
+                    if (hbaseAdmin.isTableEnabled(htableName)) {
+                        hbaseAdmin.disableTable(htableName);
                     }
 
-                    hbaseAdmin.deleteTable(tableName);
+                    hbaseAdmin.deleteTable(htableName);
                     logger.info("Deleted HBase table " + htableName);
                 } else {
                     logger.info("HBase table" + htableName + " does not exist");

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
index bba6745..1ea8e8d 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
@@ -22,13 +22,12 @@ import java.io.IOException;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.token.TokenUtil;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -60,12 +59,12 @@ public class PingHBaseCLI {
         Scan scan = new Scan();
         int limit = 20;
 
-        Connection conn = null;
-        Table table = null;
+        HConnection conn = null;
+        HTableInterface table = null;
         ResultScanner scanner = null;
         try {
-            conn = ConnectionFactory.createConnection(hconf);
-            table = conn.getTable(TableName.valueOf(hbaseTable));
+            conn = HConnectionManager.createConnection(hconf);
+            table = conn.getTable(hbaseTable);
             scanner = table.getScanner(scan);
             int count = 0;
             for (Result r : scanner) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
index db516bb..01edb1f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
@@ -22,12 +22,11 @@ import java.io.IOException;
 import java.util.Iterator;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.storage.hbase.HBaseConnection;
@@ -71,8 +70,8 @@ public class RowCounterCLI {
 
         logger.info("My Scan " + scan.toString());
 
-        Connection conn = ConnectionFactory.createConnection(conf);
-        Table tableInterface = conn.getTable(TableName.valueOf(htableName));
+        HConnection conn = HConnectionManager.createConnection(conf);
+        HTableInterface tableInterface = conn.getTable(htableName);
 
         Iterator<Result> iterator = tableInterface.getScanner(scan).iterator();
         int counter = 0;

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
index f6b65ab..23e7e10 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
@@ -40,9 +40,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.CliCommandExecutor;
@@ -59,7 +57,6 @@ import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableManager;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
-import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -80,8 +77,7 @@ public class StorageCleanupJob extends AbstractApplication {
     private void cleanUnusedHBaseTables(Configuration conf) throws IOException {
         CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
         // get all kylin hbase tables
-        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
-        Admin hbaseAdmin = conn.getAdmin();
+        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         List<String> allTablesNeedToBeDropped = new ArrayList<String>();
@@ -157,22 +153,22 @@ public class StorageCleanupJob extends AbstractApplication {
     }
 
     class DeleteHTableRunnable implements Callable {
-        Admin hbaseAdmin;
+        HBaseAdmin hbaseAdmin;
         String htableName;
 
-        DeleteHTableRunnable(Admin hbaseAdmin, String htableName) {
+        DeleteHTableRunnable(HBaseAdmin hbaseAdmin, String htableName) {
             this.hbaseAdmin = hbaseAdmin;
             this.htableName = htableName;
         }
 
         public Object call() throws Exception {
             logger.info("Deleting HBase table " + htableName);
-            if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
-                if (hbaseAdmin.isTableEnabled(TableName.valueOf(htableName))) {
-                    hbaseAdmin.disableTable(TableName.valueOf(htableName));
+            if (hbaseAdmin.tableExists(htableName)) {
+                if (hbaseAdmin.isTableEnabled(htableName)) {
+                    hbaseAdmin.disableTable(htableName);
                 }
 
-                hbaseAdmin.deleteTable(TableName.valueOf(htableName));
+                hbaseAdmin.deleteTable(htableName);
                 logger.info("Deleted HBase table " + htableName);
             } else {
                 logger.info("HBase table" + htableName + " does not exist");

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
index 42a54c8..e36f662 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
@@ -24,18 +24,16 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,15 +49,14 @@ public class UpdateHTableHostCLI {
     private List<String> errorMsgs = Lists.newArrayList();
 
     private List<String> htables;
-    private Admin hbaseAdmin;
+    private HBaseAdmin hbaseAdmin;
     private KylinConfig kylinConfig;
     private String oldHostValue;
 
     public UpdateHTableHostCLI(List<String> htables, String oldHostValue) throws IOException {
         this.htables = htables;
         this.oldHostValue = oldHostValue;
-        Connection conn = ConnectionFactory.createConnection(HBaseConfiguration.create());
-        hbaseAdmin = conn.getAdmin();
+        this.hbaseAdmin = new HBaseAdmin(HBaseConnection.getCurrentHBaseConfiguration());
         this.kylinConfig = KylinConfig.getInstanceFromEnv();
     }
 
@@ -169,9 +166,9 @@ public class UpdateHTableHostCLI {
         HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
         if (oldHostValue.equals(desc.getValue(IRealizationConstants.HTableTag))) {
             desc.setValue(IRealizationConstants.HTableTag, kylinConfig.getMetadataUrlPrefix());
-            hbaseAdmin.disableTable(TableName.valueOf(tableName));
-            hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
-            hbaseAdmin.enableTable(TableName.valueOf(tableName));
+            hbaseAdmin.disableTable(tableName);
+            hbaseAdmin.modifyTable(tableName, desc);
+            hbaseAdmin.enableTable(tableName);
 
             updatedResources.add(tableName);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/tool/pom.xml
----------------------------------------------------------------------
diff --git a/tool/pom.xml b/tool/pom.xml
index 91040d4..919a903 100644
--- a/tool/pom.xml
+++ b/tool/pom.xml
@@ -60,16 +60,6 @@
             <artifactId>hbase-client</artifactId>
             <scope>provided</scope>
         </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-yarn-api</artifactId>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-yarn-common</artifactId>
-            <scope>provided</scope>
-        </dependency>
 
         <!-- Env & Test -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java b/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
index c0042f3..c8bff89 100644
--- a/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
+++ b/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
@@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.RawResource;
@@ -231,7 +231,6 @@ public class CubeMigrationCLI {
             operations.add(new Opt(OptType.COPY_DICT_OR_SNAPSHOT, new Object[] { item, cube.getName() }));
         }
     }
-
     private static void addCubeAndModelIntoProject(CubeInstance srcCube, String cubeName, String projectName) throws IOException {
         String projectResPath = ProjectInstance.concatResourcePath(projectName);
         if (!dstStore.exists(projectResPath))
@@ -448,11 +447,11 @@ public class CubeMigrationCLI {
             Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
             ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
             String projUUID = project.getUuid();
-            Table srcAclHtable = null;
-            Table destAclHtable = null;
+            HTableInterface srcAclHtable = null;
+            HTableInterface destAclHtable = null;
             try {
-                srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(TableName.valueOf(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
-                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
+                srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
 
                 // cube acl
                 Result result = srcAclHtable.get(new Get(Bytes.toBytes(cubeId)));
@@ -472,6 +471,7 @@ public class CubeMigrationCLI {
                         destAclHtable.put(put);
                     }
                 }
+                destAclHtable.flushCommits();
             } finally {
                 IOUtils.closeQuietly(srcAclHtable);
                 IOUtils.closeQuietly(destAclHtable);
@@ -537,12 +537,13 @@ public class CubeMigrationCLI {
         case COPY_ACL: {
             String cubeId = (String) opt.params[0];
             String modelId = (String) opt.params[1];
-            Table destAclHtable = null;
+            HTableInterface destAclHtable = null;
             try {
-                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
+                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
 
                 destAclHtable.delete(new Delete(Bytes.toBytes(cubeId)));
                 destAclHtable.delete(new Delete(Bytes.toBytes(modelId)));
+                destAclHtable.flushCommits();
             } finally {
                 IOUtils.closeQuietly(destAclHtable);
             }
@@ -559,7 +560,7 @@ public class CubeMigrationCLI {
         }
     }
 
-    private static void updateMeta(KylinConfig config) {
+    private static void updateMeta(KylinConfig config){
         String[] nodes = config.getRestServers();
         for (String node : nodes) {
             RestClient restClient = new RestClient(node);


[06/13] kylin git commit: KYLIN 1875 UI replace table name with alias

Posted by li...@apache.org.
KYLIN 1875 UI replace table name with alias

Signed-off-by: lidongsjtu <li...@apache.org>


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

Branch: refs/heads/master-hbase0.98
Commit: 3c49c9e0c261ad0893c8911cc0f19804afea5516
Parents: 09a0866
Author: chenzhx <34...@qq.com>
Authored: Fri Feb 10 14:48:47 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Fri Feb 10 20:59:55 2017 +0800

----------------------------------------------------------------------
 .../js/controllers/modelConditionsSettings.js   | 42 ++++++++++++++------
 webapp/app/js/controllers/modelDataModel.js     | 10 ++++-
 webapp/app/js/controllers/modelEdit.js          | 13 ------
 webapp/app/js/controllers/modelSchema.js        | 11 -----
 .../app/partials/cubeDesigner/dimensions.html   |  2 +-
 .../modelDesigner/conditions_settings.html      | 32 +++++++++++++--
 .../app/partials/modelDesigner/data_model.html  | 37 +++++++++++------
 .../modelDesigner/model_dimensions.html         |  4 +-
 .../partials/modelDesigner/model_measures.html  |  2 +-
 webapp/app/partials/models/model_edit.html      |  7 +---
 10 files changed, 96 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/3c49c9e0/webapp/app/js/controllers/modelConditionsSettings.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/modelConditionsSettings.js b/webapp/app/js/controllers/modelConditionsSettings.js
index b3e4998..004d5bc 100644
--- a/webapp/app/js/controllers/modelConditionsSettings.js
+++ b/webapp/app/js/controllers/modelConditionsSettings.js
@@ -25,14 +25,18 @@
 KylinApp.controller('ModelConditionsSettingsCtrl', function ($scope, $modal,MetaModel,modelsManager,VdmUtil) {
   $scope.modelsManager = modelsManager;
   $scope.availableFactTables = [];
-  $scope.selectedTables={fact:VdmUtil.getNameSpaceAliasName($scope.modelsManager.selectedModel.partition_desc.partition_date_column)}
-  $scope.availableFactTables.push(VdmUtil.removeNameSpace($scope.modelsManager.selectedModel.fact_table));
-  var joinTable = $scope.modelsManager.selectedModel.lookups;
-  for (var j = 0; j < joinTable.length; j++) {
-    if(joinTable[j].kind=='FACT'){
-      $scope.availableFactTables.push(joinTable[j].alias);
+  $scope.initSetting = function (){
+    $scope.selectedTables={fact:VdmUtil.getNameSpaceAliasName($scope.modelsManager.selectedModel.partition_desc.partition_date_column)}
+    $scope.selectedTablesForPartitionTime={fact:VdmUtil.getNameSpaceAliasName($scope.modelsManager.selectedModel.partition_desc.partition_time_column)}
+    $scope.availableFactTables.push(VdmUtil.removeNameSpace($scope.modelsManager.selectedModel.fact_table));
+    var joinTable = $scope.modelsManager.selectedModel.lookups;
+    for (var j = 0; j < joinTable.length; j++) {
+      if(joinTable[j].kind=='FACT'){
+        $scope.availableFactTables.push(joinTable[j].alias);
+      }
     }
   }
+
   $scope.isFormatEdit = {editable:false};
   var judgeFormatEditable = function(dateColumn){
     if(dateColumn == null){
@@ -61,6 +65,21 @@ KylinApp.controller('ModelConditionsSettingsCtrl', function ($scope, $modal,Meta
     return;
 
   };
+
+  $scope.getPartitonColumns = function(alias){
+    var columns = _.filter($scope.getColumnsByAlias(alias),function(column){
+      return column.datatype==="date"||column.datatype==="timestamp"||column.datatype==="string"||column.datatype.startsWith("varchar")||column.datatype==="bigint"||column.datatype==="int"||column.datatype==="integer";
+    });
+    return columns;
+  };
+
+  $scope.getPartitonTimeColumns = function(tableName,filterColumn){
+    var columns = _.filter($scope.getColumnsByAlias(tableName),function(column){
+      return (column.datatype==="time"||column.datatype==="timestamp"||column.datatype==="string"||column.datatype.startsWith("varchar"))&&(tableName+'.'+column.name!=filterColumn);
+    });
+    return columns;
+  };
+
   $scope.partitionChange = function (dateColumn) {
     judgeFormatEditable(dateColumn);
   };
@@ -68,14 +87,13 @@ KylinApp.controller('ModelConditionsSettingsCtrl', function ($scope, $modal,Meta
       "hasSeparateTimeColumn" : false
   }
 
-  if (($scope.state.mode=='edit')&&($scope.isEdit = !!$scope.route.params)) {
-    if($scope.modelsManager.selectedModel.partition_desc.partition_time_column){
-      $scope.partitionColumn.hasSeparateTimeColumn = true;
-    }
+  if ($scope.state.mode=='edit'){
+    $scope.initSetting();
     judgeFormatEditable($scope.modelsManager.selectedModel.partition_desc.partition_date_column);
   }
-
-
+  if($scope.modelsManager.selectedModel.partition_desc.partition_time_column){
+    $scope.partitionColumn.hasSeparateTimeColumn = true;
+  }
   $scope.toggleHasSeparateColumn = function(){
     if($scope.partitionColumn.hasSeparateTimeColumn == false){
       $scope.modelsManager.selectedModel.partition_desc.partition_time_column = null;

http://git-wip-us.apache.org/repos/asf/kylin/blob/3c49c9e0/webapp/app/js/controllers/modelDataModel.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/modelDataModel.js b/webapp/app/js/controllers/modelDataModel.js
index 07009a2..02c44ea 100644
--- a/webapp/app/js/controllers/modelDataModel.js
+++ b/webapp/app/js/controllers/modelDataModel.js
@@ -269,8 +269,14 @@ KylinApp.controller('ModelDataModelCtrl', function ($location,$scope, $modal,cub
           break;
         }
       }
-      if($scope.aliasName.indexOf($scope.newLookup.alias)!=-1&&$scope.aliasName[$scope.lookupState.editingIndex+1] != $scope.newLookup.alias){
-        errors.push("Table Alias ["+$scope.newLookup.alias+"] already exist!");
+      if($scope.aliasName.indexOf($scope.newLookup.alias)!=-1){
+        if($scope.lookupState.editingIndex==-1){
+           errors.push("Table Alias ["+$scope.newLookup.alias+"] already exist!");
+        }else{
+          if($scope.aliasName[$scope.lookupState.editingIndex+1] != $scope.newLookup.alias){
+            errors.push("Table Alias ["+$scope.newLookup.alias+"] already exist!");
+          }
+        }
       }
       var errorInfo = "";
       angular.forEach(errors,function(item){

http://git-wip-us.apache.org/repos/asf/kylin/blob/3c49c9e0/webapp/app/js/controllers/modelEdit.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/modelEdit.js b/webapp/app/js/controllers/modelEdit.js
index 351e779..ff85f4f 100644
--- a/webapp/app/js/controllers/modelEdit.js
+++ b/webapp/app/js/controllers/modelEdit.js
@@ -37,19 +37,6 @@ KylinApp.controller('ModelEditCtrl', function ($scope, $q, $routeParams, $locati
     $scope.modelsManager = modelsManager;
     $scope.cubeConfig = cubeConfig;
 
-    $scope.getPartitonColumns = function(alias){
-        var columns = _.filter($scope.getColumnsByAlias(alias),function(column){
-            return column.datatype==="date"||column.datatype==="timestamp"||column.datatype==="string"||column.datatype.startsWith("varchar")||column.datatype==="bigint"||column.datatype==="int"||column.datatype==="integer";
-        });
-        return columns;
-    };
-
-    $scope.getPartitonTimeColumns = function(tableName){
-        var columns = _.filter($scope.getColumnsByTable(tableName),function(column){
-            return column.datatype==="time"||column.datatype==="timestamp"||column.datatype==="string"||column.datatype.startsWith("varchar");
-        });
-        return columns;
-    };
 
     $scope.getColumnsByTable = function (tableName) {
         var temp = [];

http://git-wip-us.apache.org/repos/asf/kylin/blob/3c49c9e0/webapp/app/js/controllers/modelSchema.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/modelSchema.js b/webapp/app/js/controllers/modelSchema.js
index f917f1e..41a26bb 100644
--- a/webapp/app/js/controllers/modelSchema.js
+++ b/webapp/app/js/controllers/modelSchema.js
@@ -33,17 +33,6 @@ KylinApp.controller('ModelSchemaCtrl', function ($scope, QueryService, UserServi
     $scope.state = {mode: "view"};
   }
 
-  if(!$scope.partitionColumn){
-    $scope.partitionColumn ={
-      "hasSeparateTimeColumn" : false
-    }
-  }
-
-  if($scope.state.mode !== "edit" && $scope.modelsManager.selectedModel.partition_desc.partition_time_column){
-    $scope.partitionColumn.hasSeparateTimeColumn = true;
-  }
-
-
 
   $scope.wizardSteps = [
     {title: 'Model Info', src: 'partials/modelDesigner/model_info.html', isComplete: false, form: 'model_info_form'},

http://git-wip-us.apache.org/repos/asf/kylin/blob/3c49c9e0/webapp/app/partials/cubeDesigner/dimensions.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubeDesigner/dimensions.html b/webapp/app/partials/cubeDesigner/dimensions.html
index d8a8fce..b8bd05f 100644
--- a/webapp/app/partials/cubeDesigner/dimensions.html
+++ b/webapp/app/partials/cubeDesigner/dimensions.html
@@ -39,7 +39,7 @@
                 <tr>
                     <th>ID</th>
                     <th>Name</th>
-                    <th>Table Name</th>
+                    <th>Table Alias</th>
                     <th>Type</th>
                     <th>Column</th>
                     <th ng-if="state.mode=='edit'">Actions</th>

http://git-wip-us.apache.org/repos/asf/kylin/blob/3c49c9e0/webapp/app/partials/modelDesigner/conditions_settings.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/modelDesigner/conditions_settings.html b/webapp/app/partials/modelDesigner/conditions_settings.html
index 30fcb67..a39c8b3 100644
--- a/webapp/app/partials/modelDesigner/conditions_settings.html
+++ b/webapp/app/partials/modelDesigner/conditions_settings.html
@@ -79,7 +79,7 @@
               </div>
             </div>
           </div>
-          <!--Partition Time Column-->
+<!--          &lt;!&ndash;Partition Time Column&ndash;&gt;
           <div class="form-group" ng-if="partitionColumn.hasSeparateTimeColumn == true">
             <div class="row middle-popover">
               <label class="control-label col-xs-12 col-sm-3 no-padding-right font-color-default"><b>Partition Time Column</b>  <i kylinpopover placement="right" title="Partition Time Column" template="partitionTimeTip.html" class="fa fa-info-circle"></i></label>
@@ -90,16 +90,40 @@
                         ng-if="state.mode=='edit'"
                         data-placement=""
                         ng-options="modelsManager.selectedModel.fact_table+'.'+columns.name as modelsManager.selectedModel.fact_table+'.'+columns.name for columns in getPartitonTimeColumns(modelsManager.selectedModel.fact_table)" >
-                  <option value="">--Select Partition Column--</option>
+                  <option value="">&#45;&#45;Select Partition Column&#45;&#45;</option>
                 </select>
-                <!--<small class="text-info" ng-show="state.mode=='edit'">(Column Type should be TIME Type)</small>-->
+                &lt;!&ndash;<small class="text-info" ng-show="state.mode=='edit'">(Column Type should be TIME Type)</small>&ndash;&gt;
 
                       <span ng-if="state.mode=='view'">
                           {{!!(modelsManager.selectedModel.partition_desc.partition_time_column)?modelsManager.selectedModel.partition_desc.partition_time_column: ''}}</span>
               </div>
             </div>
-          </div>
+          </div>-->
 
+          <!--Partition Time Column-->
+          <div class="form-group" ng-if="partitionColumn.hasSeparateTimeColumn == true">
+            <div class="row middle-popover">
+              <label class="control-label col-xs-12 col-sm-3 no-padding-right font-color-default align-right"><b>Partition Time Column </b> <i kylinpopover placement="right" ng-title="Partition Time Column" template="partitionTimeTip.html" class="fa fa-info-circle"></i> </label>
+              <div class="col-xs-12 col-sm-6">
+                <select style="width: 49%" chosen
+                        ng-model="selectedTablesForPartitionTime.fact"
+                        ng-if="state.mode=='edit'"
+                        data-placement=""
+                        ng-options="alias as alias for alias in availableFactTables" >
+                  <option value="">--Select Partition Table--</option>
+                </select>
+                <select style="width: 49%" chosen
+                        ng-model="modelsManager.selectedModel.partition_desc.partition_time_column"
+                        ng-if="state.mode=='edit'"
+                        data-placement=""
+                        ng-options="selectedTablesForPartitionTime.fact+'.'+columns.name as columns.name for columns in getPartitonTimeColumns(selectedTablesForPartitionTime.fact,modelsManager.selectedModel.partition_desc.partition_date_column)">
+                  <option value="">--Select Partition Column--</option>
+                </select>
+                <span ng-if="state.mode=='view'">
+                      {{!!(modelsManager.selectedModel.partition_desc.partition_time_column)?modelsManager.selectedModel.partition_desc.partition_time_column: ''}}</span>
+              </div>
+            </div>
+          </div>
           <!--Time Format-->
           <div class="form-group" ng-if="partitionColumn.hasSeparateTimeColumn == true">
             <div class="row">

http://git-wip-us.apache.org/repos/asf/kylin/blob/3c49c9e0/webapp/app/partials/modelDesigner/data_model.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/modelDesigner/data_model.html b/webapp/app/partials/modelDesigner/data_model.html
index 167c088..b55935e 100644
--- a/webapp/app/partials/modelDesigner/data_model.html
+++ b/webapp/app/partials/modelDesigner/data_model.html
@@ -23,7 +23,7 @@
     <div class="form-group" ng-class="{'required':state.mode=='edit'}">
         <div class="row">
             <label class="col-xs-12 col-sm-2 control-label concube.detailtrol-label no-padding-right font-color-default">
-                <b>Root Fact Table</b>
+                <b>Fact Table</b>
             </label>
             <div class="col-xs-12 col-sm-6" ng-class="{'has-error':forms.data_model_form.table_name.$invalid && (forms.data_model_form.table_name.$dirty||forms.data_model_form.$submitted)}">
               <select chosen ng-model="FactTable.root" ng-if="state.mode=='edit'"
@@ -46,7 +46,7 @@
             <div class="col-xs-6" ng-if="state.mode=='edit'">
                 <button type="button" class="btn btn-primary" ng-disabled="!modelsManager.selectedModel.fact_table.length"
                         ng-click="openLookupModal()">
-                    <i class="fa fa-plus"></i> Add Join Table
+                    <i class="fa fa-plus"></i> Add Lookup Table
                 </button>
             </div>
             <div class="col-xs-6" ng-if="state.mode!='edit'">
@@ -63,6 +63,7 @@
             <thead>
             <tr>
                 <th>ID</th>
+                <th>Table Alias</th>
                 <th>Table Name</th>
                 <th>Table Kind</th>
                 <th>Join Type</th>
@@ -75,13 +76,18 @@
                 <td>
                     <b>{{$index + 1}}</b>
                 </td>
-                <!-- Table Name -->
+                <!-- Table Alias -->
                 <td>
                     <span>{{lookup.alias}}</span>
                 </td>
+                <!-- Table Alias -->
+                <td>
+                    <span>{{lookup.table}}</span>
+                </td>
                 <!-- Table Kind -->
                 <td>
-                    <span>{{lookup.kind}}</span>
+                    <span ng-if="lookup.kind=='FACT'">Limited</span>
+                    <span ng-if="lookup.kind=='LOOKUP'">Normal</span>
                 </td>
                 <!-- Join Type -->
                 <td>
@@ -113,10 +119,10 @@
     </div>
     </ng-form>
 
-    <!-- Add Join Table Form -->
+    <!-- Add Lookup Table Form -->
     <script type="text/ng-template" id="dataModelLookupTable.html">
         <div class="modal-header">
-            <h4 class="box-title lighter">{{lookupState.editing ? 'Edit' : 'Add'}} Join Table</h4>
+            <h4 class="box-title lighter">{{lookupState.editing ? 'Edit' : 'Add'}} Lookup Table</h4>
         </div>
         <div class="modal-body">
             <div class="row">
@@ -132,10 +138,10 @@
                                       ng-required="true"
                                       data-placeholder="Join Table Name"
                                       class="chosen-select">
-                                <option value=""> &#45;&#45; Select Join Table &#45;&#45; </option>
+                                <option value=""> &#45;&#45; From Table &#45;&#45; </option>
                               </select>
                               <!--Join Type-->
-                              <select class="form-control"  chosen ng-model="newLookup.join.type" style="width:120px;"
+                              <select class="form-control"  chosen ng-init="newLookup.join.type='inner'"  ng-model="newLookup.join.type" style="width:120px;"
                                             ng-options="joinType.value as joinType.name for joinType in cubeConfig.joinTypes">
                                 <option value=""></option>
                               </select>
@@ -144,9 +150,9 @@
                                       ng-options="table.name as VdmUtil.removeNameSpace(table.name) for table in tableModel.selectProjectTables"
                                       name="table_name"  ng-disabled="lookupState.editing"
                                       ng-required="true"  ng-change="changeJoinTable()"
-                                      data-placeholder="Join Table Name"
+                                      data-placeholder="Lookup Table Name"
                                       class="chosen-select">
-                                <option value=""> &#45;&#45; Select Join Table &#45;&#45; </option>
+                                <option value=""> &#45;&#45; Select Lookup Table &#45;&#45; </option>
                               </select>
                             </div>
                           </div>
@@ -165,10 +171,12 @@
                     <!--Table Type-->
                     <div class="form-group">
                         <div class="row">
-                          <label class="col-sm-3 control-label font-color-default"><b>Table Type</b></label>
+                          <label class="col-sm-3"></label>
                           <div class="col-sm-6">
-                            <label> <input type="radio" ng-model="newLookup.kind" value="FACT" > Fact Table </label>
-                            <label> <input type="radio" ng-model="newLookup.kind" value="LOOKUP"> Lookup Table </label>
+                            <label>
+                              <input type="checkbox" ng-model="newLookup.kind" ng-true-value="FACT" ng-false-value="LOOKUP" >&nbsp;Is Limited
+                            </label>
+                            <i class="fa fa-info-circle" kylinpopover="" placement="right" title="" template="isLimited.html"></i>
                           </div>
                         </div>
                     </div>
@@ -243,3 +251,6 @@
 <script type="text/ng-template" id="tableInUse.html">
   <p>This table is in use in Cube.</p>
 </script>
+<script type="text/ng-template" id="isLimited.html">
+  <p>Some lookup table is too big (e.g. >300 MB) for snapshot, thus must be marked as limited. Limited lookup table cannot be queried directly and does not support derived dimensions.</p>
+</script>

http://git-wip-us.apache.org/repos/asf/kylin/blob/3c49c9e0/webapp/app/partials/modelDesigner/model_dimensions.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/modelDesigner/model_dimensions.html b/webapp/app/partials/modelDesigner/model_dimensions.html
index 6249521..cfa422f 100644
--- a/webapp/app/partials/modelDesigner/model_dimensions.html
+++ b/webapp/app/partials/modelDesigner/model_dimensions.html
@@ -27,7 +27,7 @@
                 <thead>
                     <tr>
                         <th>ID</th>
-                        <th>Table Name</th>
+                        <th>Table Alias</th>
                         <th>Columns</th>
                     </tr>
                 </thead>
@@ -56,7 +56,7 @@
                    class="table table-hover list">
                 <tr class="row">
                     <th class="col-xs-1">ID</th>
-                    <th class="col-xs-2">Table Name</th>
+                    <th class="col-xs-2">Table Alias</th>
                     <th class="col-xs-9">Columns</th>
                 </tr>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/3c49c9e0/webapp/app/partials/modelDesigner/model_measures.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/modelDesigner/model_measures.html b/webapp/app/partials/modelDesigner/model_measures.html
index 1bb71b5..bf27914 100644
--- a/webapp/app/partials/modelDesigner/model_measures.html
+++ b/webapp/app/partials/modelDesigner/model_measures.html
@@ -47,7 +47,7 @@
                class="table table-hover list">
           <tr class="row">
             <th class="col-xs-1">ID</th>
-            <th class="col-xs-2">Table Name</th>
+            <th class="col-xs-2">Table Alias</th>
             <th class="col-xs-9">Columns</th>
           </tr>
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/3c49c9e0/webapp/app/partials/models/model_edit.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/models/model_edit.html b/webapp/app/partials/models/model_edit.html
index d0c9814..5dbc3cf 100644
--- a/webapp/app/partials/models/model_edit.html
+++ b/webapp/app/partials/models/model_edit.html
@@ -17,10 +17,7 @@
 -->
 
 <div class="row">
-    <div class="col-xs-3">
-        <div ng-include src="'partials/tables/source_table_tree.html'" ng-controller="SourceMetaCtrl"></div>
-    </div>
-    <div class="col-xs-9">
+    <div class="col-xs-12">
         <form role="form" name="cube_form" novalidate>
             <!-- This margin in order to align with table tree in left part -->
             <div style="margin-top: 20px;">
@@ -30,4 +27,4 @@
         </form>
     </div>
 </div>
-</div>
\ No newline at end of file
+</div>


[10/13] kylin git commit: KYLIN-2443 Report coprocessor error information back to client

Posted by li...@apache.org.
KYLIN-2443 Report coprocessor error information back to client


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

Branch: refs/heads/master-hbase0.98
Commit: 43c0566728092d537201d751d3e8f6e3c0d5f051
Parents: 707b055
Author: gaodayue <ga...@meituan.com>
Authored: Sat Feb 11 19:13:24 2017 +0800
Committer: gaodayue <ga...@meituan.com>
Committed: Sun Feb 12 09:34:49 2017 +0800

----------------------------------------------------------------------
 build/smoke-test/sql/sql1.json                  |    1 -
 build/smoke-test/testQuery.py                   |    1 +
 .../org/apache/kylin/common/QueryContext.java   |   21 +-
 .../exceptions/KylinTimeoutException.java       |   26 +
 .../ResourceLimitExceededException.java         |   30 +
 .../kylin/gridtable/GTAggregateScanner.java     |   12 +-
 .../GTScanExceedThresholdException.java         |   26 -
 .../GTScanSelfTerminatedException.java          |   30 -
 .../kylin/gridtable/GTScanTimeoutException.java |   26 -
 .../apache/kylin/storage/StorageContext.java    |   19 +-
 .../storage/gtrecord/CubeSegmentScanner.java    |    2 +-
 .../kylin/storage/gtrecord/ScannerWorker.java   |    5 +-
 .../gtrecord/SequentialCubeTupleIterator.java   |    6 +-
 .../apache/kylin/query/ITKylinQueryTest.java    |    6 +-
 .../apache/kylin/rest/service/QueryService.java |   28 +-
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java     |   69 +-
 .../storage/hbase/cube/v2/CubeHBaseRPC.java     |    8 +-
 .../storage/hbase/cube/v2/CubeHBaseScanRPC.java |   15 +-
 .../coprocessor/endpoint/CubeVisitService.java  |   42 +-
 .../endpoint/generated/CubeVisitProtos.java     | 1254 ++++++++++++++++--
 .../endpoint/protobuf/CubeVisit.proto           |   12 +-
 21 files changed, 1331 insertions(+), 308 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/build/smoke-test/sql/sql1.json
----------------------------------------------------------------------
diff --git a/build/smoke-test/sql/sql1.json b/build/smoke-test/sql/sql1.json
index 7cb3258..21e4c01 100644
--- a/build/smoke-test/sql/sql1.json
+++ b/build/smoke-test/sql/sql1.json
@@ -9,7 +9,6 @@
     ]
   ],
   "exceptionMessage": null,
-  "totalScanCount": 1,
   "columnMetas": [
     {
       "scale": 0,

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/build/smoke-test/testQuery.py
----------------------------------------------------------------------
diff --git a/build/smoke-test/testQuery.py b/build/smoke-test/testQuery.py
index 87a2456..99c09d3 100644
--- a/build/smoke-test/testQuery.py
+++ b/build/smoke-test/testQuery.py
@@ -59,6 +59,7 @@ class testQuery(unittest.TestCase):
             del actual_result['duration']
             del actual_result['hitExceptionCache']
             del actual_result['storageCacheUsed']
+            del actual_result['totalScanCount']
             del actual_result['totalScanBytes']
 
             expect_result = json.loads(open(sql_file[:-4] + '.json').read().strip())

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/core-common/src/main/java/org/apache/kylin/common/QueryContext.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/QueryContext.java b/core-common/src/main/java/org/apache/kylin/common/QueryContext.java
index 3a73993..67925b6 100644
--- a/core-common/src/main/java/org/apache/kylin/common/QueryContext.java
+++ b/core-common/src/main/java/org/apache/kylin/common/QueryContext.java
@@ -21,7 +21,7 @@ package org.apache.kylin.common;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
- * checkout {@link org.apache.kylin.common.debug.BackdoorToggles} for comparison
+ * Holds per query information and statistics.
  */
 public class QueryContext {
 
@@ -33,7 +33,8 @@ public class QueryContext {
     };
 
     private String queryId;
-    private AtomicLong scanBytes = new AtomicLong();
+    private AtomicLong scannedRows = new AtomicLong();
+    private AtomicLong scannedBytes = new AtomicLong();
 
     private QueryContext() {
         // use QueryContext.current() instead
@@ -55,11 +56,19 @@ public class QueryContext {
         this.queryId = queryId;
     }
 
-    public long getScanBytes() {
-        return scanBytes.get();
+    public long getScannedRows() {
+        return scannedRows.get();
     }
 
-    public long addAndGetScanBytes(long delta) {
-        return scanBytes.addAndGet(delta);
+    public long addAndGetScannedRows(long deltaRows) {
+        return scannedRows.addAndGet(deltaRows);
+    }
+
+    public long getScannedBytes() {
+        return scannedBytes.get();
+    }
+
+    public long addAndGetScannedBytes(long deltaBytes) {
+        return scannedBytes.addAndGet(deltaBytes);
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/core-common/src/main/java/org/apache/kylin/common/exceptions/KylinTimeoutException.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/exceptions/KylinTimeoutException.java b/core-common/src/main/java/org/apache/kylin/common/exceptions/KylinTimeoutException.java
new file mode 100644
index 0000000..75d981f
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/exceptions/KylinTimeoutException.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.common.exceptions;
+
+public class KylinTimeoutException extends RuntimeException {
+
+    public KylinTimeoutException(String message) {
+        super(message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/core-common/src/main/java/org/apache/kylin/common/exceptions/ResourceLimitExceededException.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/exceptions/ResourceLimitExceededException.java b/core-common/src/main/java/org/apache/kylin/common/exceptions/ResourceLimitExceededException.java
new file mode 100644
index 0000000..df5d88e
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/exceptions/ResourceLimitExceededException.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.common.exceptions;
+
+public class ResourceLimitExceededException extends RuntimeException {
+
+    public ResourceLimitExceededException(String message) {
+        super(message);
+    }
+
+    public ResourceLimitExceededException(String message, Throwable cause) {
+        super(message, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
index dd359f8..8b0efcc 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
@@ -35,6 +35,7 @@ import java.util.SortedMap;
 import java.util.Map.Entry;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.kylin.common.exceptions.ResourceLimitExceededException;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.ImmutableBitSet;
@@ -278,14 +279,11 @@ public class GTAggregateScanner implements IGTScanner {
 
                 final long estMemSize = estimatedMemSize();
                 if (spillThreshold > 0 && estMemSize > spillThreshold) {
-                    // spill to disk when aggBufMap used too large memory
-                    if (spillEnabled) {
-                        spillBuffMap(estMemSize);
-                        aggBufMap = createBuffMap();
-
-                    } else {
-                        throw new GTScanSelfTerminatedException("Aggregation using more than " + spillThreshold + " memory and spill is disabled");
+                    if (!spillEnabled) {
+                        throw new ResourceLimitExceededException("aggregation's memory consumption " + estMemSize + " exceeds threshold " + spillThreshold);
                     }
+                    spillBuffMap(estMemSize); // spill to disk
+                    aggBufMap = createBuffMap();
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanExceedThresholdException.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanExceedThresholdException.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanExceedThresholdException.java
deleted file mode 100644
index ba75962..0000000
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanExceedThresholdException.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *  
- *     http://www.apache.org/licenses/LICENSE-2.0
- *  
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.gridtable;
-
-public class GTScanExceedThresholdException extends GTScanSelfTerminatedException {
-
-    public GTScanExceedThresholdException(String message) {
-        super(message);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java
deleted file mode 100644
index 30d3aaa..0000000
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *  
- *     http://www.apache.org/licenses/LICENSE-2.0
- *  
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.gridtable;
-
-/**
- * Implementations of {@link IGTScanner} should throw {@link GTScanSelfTerminatedException} or its subclasses
- * in cases where the scan runs out of resources (time, memory, etc) and can not be continued.
- */
-public class GTScanSelfTerminatedException extends RuntimeException {
-
-    public GTScanSelfTerminatedException(String s) {
-        super(s);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanTimeoutException.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanTimeoutException.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanTimeoutException.java
deleted file mode 100644
index 17a8d02..0000000
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanTimeoutException.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *  
- *     http://www.apache.org/licenses/LICENSE-2.0
- *  
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.gridtable;
-
-public class GTScanTimeoutException extends GTScanSelfTerminatedException {
-
-    public GTScanTimeoutException(String message) {
-        super(message);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java b/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
index 0f52c53..4713d71 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
@@ -47,8 +47,7 @@ public class StorageContext {
     private boolean enableCoprocessor = false;
 
     private IStorageQuery storageQuery;
-    private AtomicLong totalScanCount = new AtomicLong();
-    private AtomicLong totalScanBytes = new AtomicLong();
+    private AtomicLong processedRowCount = new AtomicLong();
     private Cuboid cuboid;
     private boolean partialResultReturned = false;
 
@@ -140,20 +139,12 @@ public class StorageContext {
         return cuboid;
     }
 
-    public long getTotalScanCount() {
-        return totalScanCount.get();
+    public long getProcessedRowCount() {
+        return processedRowCount.get();
     }
 
-    public long increaseTotalScanCount(long count) {
-        return this.totalScanCount.addAndGet(count);
-    }
-
-    public long getTotalScanBytes() {
-        return totalScanBytes.get();
-    }
-
-    public long increaseTotalScanBytes(long bytes) {
-        return totalScanBytes.addAndGet(bytes);
+    public long increaseProcessedRowCount(long count) {
+        return processedRowCount.addAndGet(count);
     }
 
     public boolean isAcceptPartialResult() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
index 974b8ea..029502c 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
@@ -78,7 +78,7 @@ public class CubeSegmentScanner implements IGTScanner {
         }
         scanRequest = scanRangePlanner.planScanRequest();
         String gtStorage = ((GTCubeStorageQueryBase) context.getStorageQuery()).getGTStorage();
-        scanner = new ScannerWorker(cubeSeg, cuboid, scanRequest, gtStorage, context);
+        scanner = new ScannerWorker(cubeSeg, cuboid, scanRequest, gtStorage);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/ScannerWorker.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/ScannerWorker.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/ScannerWorker.java
index 2a2a86a..fd50c54 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/ScannerWorker.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/ScannerWorker.java
@@ -26,7 +26,6 @@ import org.apache.kylin.gridtable.GTScanRequest;
 import org.apache.kylin.gridtable.IGTScanner;
 import org.apache.kylin.gridtable.IGTStorage;
 import org.apache.kylin.metadata.model.ISegment;
-import org.apache.kylin.storage.StorageContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,7 +38,7 @@ public class ScannerWorker {
     private static final Logger logger = LoggerFactory.getLogger(ScannerWorker.class);
     private IGTScanner internal = null;
 
-    public ScannerWorker(ISegment segment, Cuboid cuboid, GTScanRequest scanRequest, String gtStorage, StorageContext context) {
+    public ScannerWorker(ISegment segment, Cuboid cuboid, GTScanRequest scanRequest, String gtStorage) {
         if (scanRequest == null) {
             logger.info("Segment {} will be skipped", segment);
             internal = new EmptyGTScanner(0);
@@ -49,7 +48,7 @@ public class ScannerWorker {
         final GTInfo info = scanRequest.getInfo();
 
         try {
-            IGTStorage rpc = (IGTStorage) Class.forName(gtStorage).getConstructor(ISegment.class, Cuboid.class, GTInfo.class, StorageContext.class).newInstance(segment, cuboid, info, context); // default behavior
+            IGTStorage rpc = (IGTStorage) Class.forName(gtStorage).getConstructor(ISegment.class, Cuboid.class, GTInfo.class).newInstance(segment, cuboid, info); // default behavior
             internal = rpc.getGTScanner(scanRequest);
         } catch (IOException | InstantiationException | InvocationTargetException | IllegalAccessException | ClassNotFoundException | NoSuchMethodException e) {
             throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java
index bb2d7f9..14b6394 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java
@@ -26,8 +26,8 @@ import java.util.Set;
 
 import javax.annotation.Nullable;
 
+import org.apache.kylin.common.exceptions.KylinTimeoutException;
 import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.gridtable.GTScanTimeoutException;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.tuple.ITuple;
@@ -139,7 +139,7 @@ public class SequentialCubeTupleIterator implements ITupleIterator {
     @Override
     public ITuple next() {
         if (scanCount++ % 100 == 1 && System.currentTimeMillis() > context.getDeadline()) {
-            throw new GTScanTimeoutException("Query Timeout!");
+            throw new KylinTimeoutException("Query timeout after \"kylin.query.timeout-seconds\" seconds");
         }
 
         if (++scanCountDelta >= 1000)
@@ -173,7 +173,7 @@ public class SequentialCubeTupleIterator implements ITupleIterator {
     }
 
     private void flushScanCountDelta() {
-        context.increaseTotalScanCount(scanCountDelta);
+        context.increaseProcessedRowCount(scanCountDelta);
         scanCountDelta = 0;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
index 87ddcb8..4590e60 100644
--- a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
@@ -29,7 +29,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.KylinVersion;
 import org.apache.kylin.common.debug.BackdoorToggles;
-import org.apache.kylin.gridtable.GTScanSelfTerminatedException;
+import org.apache.kylin.common.exceptions.KylinTimeoutException;
 import org.apache.kylin.gridtable.StorageSideBehavior;
 import org.apache.kylin.metadata.realization.RealizationType;
 import org.apache.kylin.query.routing.Candidate;
@@ -124,12 +124,12 @@ public class ITKylinQueryTest extends KylinTestBase {
 
                 System.out.println(e.getMessage());
 
-                if (findRoot(e) instanceof GTScanSelfTerminatedException) {
+                if (findRoot(e) instanceof KylinTimeoutException) {
                     //expected
                     continue;
                 }
             }
-            throw new RuntimeException("Expecting GTScanTimeoutException");
+            throw new RuntimeException("Expecting KylinTimeoutException");
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 7d9e24d..4c02aa4 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -57,13 +57,13 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.QueryContext;
 import org.apache.kylin.common.debug.BackdoorToggles;
+import org.apache.kylin.common.exceptions.ResourceLimitExceededException;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.DBUtils;
 import org.apache.kylin.common.util.SetThreadName;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.gridtable.GTScanExceedThresholdException;
 import org.apache.kylin.metadata.project.RealizationEntry;
 import org.apache.kylin.metadata.realization.RealizationType;
 import org.apache.kylin.query.relnode.OLAPContext;
@@ -328,10 +328,12 @@ public class QueryService extends BasicService {
             throw new InternalErrorException("Project cannot be empty. Please select a project.");
         }
 
-        final String queryId = UUID.randomUUID().toString();
         if (sqlRequest.getBackdoorToggles() != null)
             BackdoorToggles.addToggles(sqlRequest.getBackdoorToggles());
-        QueryContext.current().setQueryId(queryId);
+
+        final QueryContext queryContext = QueryContext.current();
+        final String queryId = UUID.randomUUID().toString();
+        queryContext.setQueryId(queryId);
 
         try (SetThreadName ignored = new SetThreadName("Query %s", queryId)) {
             String sql = sqlRequest.getSql();
@@ -372,6 +374,8 @@ public class QueryService extends BasicService {
 
                 } else {
                     sqlResponse.setDuration(System.currentTimeMillis() - startTime);
+                    sqlResponse.setTotalScanCount(0);
+                    sqlResponse.setTotalScanBytes(0);
                 }
 
                 checkQueryAuth(sqlResponse);
@@ -381,9 +385,10 @@ public class QueryService extends BasicService {
                 String errMsg = QueryUtil.makeErrorMsgUserFriendly(e);
 
                 sqlResponse = new SQLResponse(null, null, 0, true, errMsg);
+                sqlResponse.setTotalScanCount(queryContext.getScannedRows());
+                sqlResponse.setTotalScanBytes(queryContext.getScannedBytes());
 
-                // for exception queries, only cache ScanOutOfLimitException
-                if (queryCacheEnabled && e instanceof GTScanExceedThresholdException) {
+                if (queryCacheEnabled && e.getCause() != null && e.getCause() instanceof ResourceLimitExceededException) {
                     Cache exceptionCache = cacheManager.getCache(EXCEPTION_QUERY_CACHE);
                     exceptionCache.put(new Element(sqlRequest, sqlResponse));
                 }
@@ -582,26 +587,21 @@ public class QueryService extends BasicService {
 
         boolean isPartialResult = false;
         String cube = "";
-        StringBuilder sb = new StringBuilder("Scan stats for each storageContext: ");
-        long totalScanCount = 0;
-        long totalScanBytes = 0;
+        StringBuilder sb = new StringBuilder("Processed rows for each storageContext: ");
         if (OLAPContext.getThreadLocalContexts() != null) { // contexts can be null in case of 'explain plan for'
             for (OLAPContext ctx : OLAPContext.getThreadLocalContexts()) {
                 if (ctx.realization != null) {
                     isPartialResult |= ctx.storageContext.isPartialResultReturned();
                     cube = ctx.realization.getName();
-                    totalScanCount += ctx.storageContext.getTotalScanCount();
-                    totalScanBytes += ctx.storageContext.getTotalScanBytes();
-                    sb.append("{rows=").append(ctx.storageContext.getTotalScanCount()).
-                       append(" bytes=").append(ctx.storageContext.getTotalScanBytes()).append("} ");
+                    sb.append(ctx.storageContext.getProcessedRowCount()).append(" ");
                 }
             }
         }
         logger.info(sb.toString());
 
         SQLResponse response = new SQLResponse(columnMetas, results, cube, 0, false, null, isPartialResult);
-        response.setTotalScanCount(totalScanCount);
-        response.setTotalScanBytes(totalScanBytes);
+        response.setTotalScanCount(QueryContext.current().getScannedRows());
+        response.setTotalScanBytes(QueryContext.current().getScannedBytes());
 
         return response;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/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 68a84c1..3c01da2 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
@@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.zip.DataFormatException;
 
 import org.apache.hadoop.hbase.TableName;
@@ -33,7 +34,8 @@ 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.QueryContext;
+import org.apache.kylin.common.exceptions.KylinTimeoutException;
+import org.apache.kylin.common.exceptions.ResourceLimitExceededException;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesSerializer;
 import org.apache.kylin.common.util.BytesUtil;
@@ -43,12 +45,9 @@ import org.apache.kylin.common.util.LoggableCachedThreadPool;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.gridtable.GTInfo;
-import org.apache.kylin.gridtable.GTScanExceedThresholdException;
 import org.apache.kylin.gridtable.GTScanRequest;
-import org.apache.kylin.gridtable.GTScanSelfTerminatedException;
 import org.apache.kylin.gridtable.IGTScanner;
 import org.apache.kylin.metadata.model.ISegment;
-import org.apache.kylin.storage.StorageContext;
 import org.apache.kylin.storage.gtrecord.DummyPartitionStreamer;
 import org.apache.kylin.storage.gtrecord.StorageResponseGTScatter;
 import org.apache.kylin.storage.hbase.HBaseConnection;
@@ -71,8 +70,8 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
 
     private static ExecutorService executorService = new LoggableCachedThreadPool();
 
-    public CubeHBaseEndpointRPC(ISegment segment, Cuboid cuboid, GTInfo fullGTInfo, StorageContext context) {
-        super(segment, cuboid, fullGTInfo, context);
+    public CubeHBaseEndpointRPC(ISegment segment, Cuboid cuboid, GTInfo fullGTInfo) {
+        super(segment, cuboid, fullGTInfo);
     }
 
     private byte[] getByteArrayForShort(short v) {
@@ -107,8 +106,6 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
     @SuppressWarnings("checkstyle:methodlength")
     @Override
     public IGTScanner getGTScanner(final GTScanRequest scanRequest) throws IOException {
-        final QueryContext queryContext = QueryContext.current();
-
         Pair<Short, Short> shardNumAndBaseShard = getShardNumAndBaseShard();
         short shardNum = shardNumAndBaseShard.getFirst();
         short cuboidBaseShard = shardNumAndBaseShard.getSecond();
@@ -175,7 +172,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
                 public void run() {
 
                     final String logHeader = String.format("<sub-thread for Query %s GTScanRequest %s>", queryId, Integer.toHexString(System.identityHashCode(scanRequest)));
-                    final boolean[] abnormalFinish = new boolean[1];
+                    final AtomicReference<RuntimeException> regionErrorHolder = new AtomicReference<>();
 
                     try {
                         Table table = conn.getTable(TableName.valueOf(cubeSeg.getStorageLocationIdentifier()), HBaseConnection.getCoprocessorPool());
@@ -199,22 +196,32 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
                                 }, new Batch.Callback<CubeVisitResponse>() {
                                     @Override
                                     public void update(byte[] region, byte[] row, CubeVisitResponse result) {
-                                        if (region == null)
+                                        if (region == null) {
                                             return;
+                                        }
 
-                                        final long scanBytes = result.getStats().getScannedBytes();
-                                        context.increaseTotalScanBytes(scanBytes);
-                                        totalScannedCount.addAndGet(result.getStats().getScannedRowCount());
                                         logger.info(logHeader + getStatsString(region, result));
 
-                                        if (queryContext.addAndGetScanBytes(scanBytes) > cubeSeg.getConfig().getQueryMaxScanBytes()) {
-                                            throw new GTScanExceedThresholdException("Query scanned " + queryContext.getScanBytes() + " bytes exceeds threshold " + cubeSeg.getConfig().getQueryMaxScanBytes());
+                                        Stats stats = result.getStats();
+                                        queryContext.addAndGetScannedRows(stats.getScannedRowCount());
+                                        queryContext.addAndGetScannedBytes(stats.getScannedBytes());
+                                        totalScannedCount.addAndGet(stats.getScannedRowCount());
+
+                                        // if any other region has responded with error, skip further processing
+                                        if (regionErrorHolder.get() != null) {
+                                            return;
                                         }
 
+                                        // record coprocessor error if happened
                                         if (result.getStats().getNormalComplete() != 1) {
-                                            abnormalFinish[0] = true;
+                                            regionErrorHolder.compareAndSet(null, getCoprocessorException(result));
                                             return;
                                         }
+
+                                        if (queryContext.getScannedBytes() > cubeSeg.getConfig().getQueryMaxScanBytes()) {
+                                            throw new ResourceLimitExceededException("Query scanned " + queryContext.getScannedBytes() + " bytes exceeds threshold " + cubeSeg.getConfig().getQueryMaxScanBytes());
+                                        }
+
                                         try {
                                             if (compressionResult) {
                                                 epResultItr.append(CompressionUtils.decompress(HBaseZeroCopyByteString.zeroCopyGetBytes(result.getCompressedRows())));
@@ -233,11 +240,10 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
                         return;
                     }
 
-                    if (abnormalFinish[0]) {
-                        Throwable ex = new GTScanSelfTerminatedException(logHeader + "The coprocessor thread stopped itself due to scan timeout or scan threshold(check region server log), failing current query...");
-                        logger.error(logHeader + "Error when visiting cubes by endpoint", ex); // double log coz the query thread may already timeout
-                        epResultItr.notifyCoprocException(ex);
-                        return;
+                    if (regionErrorHolder.get() != null) {
+                        RuntimeException exception = regionErrorHolder.get();
+                        logger.error(logHeader + "Error when visiting cubes by endpoint", exception); // double log coz the query thread may already timeout
+                        epResultItr.notifyCoprocException(exception);
                     }
                 }
             });
@@ -288,6 +294,8 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
     private String getStatsString(byte[] region, CubeVisitResponse result) {
         StringBuilder sb = new StringBuilder();
         Stats stats = result.getStats();
+        byte[] compressedRows = HBaseZeroCopyByteString.zeroCopyGetBytes(result.getCompressedRows());
+
         sb.append("Endpoint RPC returned from HTable ").append(cubeSeg.getStorageLocationIdentifier()).append(" Shard ").append(BytesUtil.toHex(region)).append(" on host: ").append(stats.getHostname()).append(".");
         sb.append("Total scanned row: ").append(stats.getScannedRowCount()).append(". ");
         sb.append("Total scanned bytes: ").append(stats.getScannedBytes()).append(". ");
@@ -296,8 +304,27 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
         sb.append("Server CPU usage: ").append(stats.getSystemCpuLoad()).append(", server physical mem left: ").append(stats.getFreePhysicalMemorySize()).append(", server swap mem left:").append(stats.getFreeSwapSpaceSize()).append(".");
         sb.append("Etc message: ").append(stats.getEtcMsg()).append(".");
         sb.append("Normal Complete: ").append(stats.getNormalComplete() == 1).append(".");
+        sb.append("Compressed row size: ").append(compressedRows.length);
         return sb.toString();
 
     }
 
+    private RuntimeException getCoprocessorException(CubeVisitResponse response) {
+        if (!response.hasErrorInfo()) {
+            return new RuntimeException("Coprocessor aborts due to scan timeout or other reasons, please re-deploy coprocessor to see concrete error message");
+        }
+
+        CubeVisitResponse.ErrorInfo errorInfo = response.getErrorInfo();
+
+        switch (errorInfo.getType()) {
+            case UNKNOWN_TYPE:
+                return new RuntimeException("Coprocessor aborts: " + errorInfo.getMessage());
+            case TIMEOUT:
+                return new KylinTimeoutException(errorInfo.getMessage());
+            case RESOURCE_LIMIT_EXCEEDED:
+                return new ResourceLimitExceededException("Coprocessor resource limit exceeded: " + errorInfo.getMessage());
+            default:
+                throw new AssertionError("Unknown error type: " + errorInfo.getType());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/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 11fbb19..f24290c 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FuzzyRowFilter;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.QueryContext;
 import org.apache.kylin.common.debug.BackdoorToggles;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.ImmutableBitSet;
@@ -48,7 +49,6 @@ 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.storage.StorageContext;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -63,18 +63,18 @@ public abstract class CubeHBaseRPC implements IGTStorage {
     final protected CubeSegment cubeSeg;
     final protected Cuboid cuboid;
     final protected GTInfo fullGTInfo;
-    final protected StorageContext context;
+    final protected QueryContext queryContext;
 
     final private RowKeyEncoder fuzzyKeyEncoder;
     final private RowKeyEncoder fuzzyMaskEncoder;
 
-    public CubeHBaseRPC(ISegment segment, Cuboid cuboid, GTInfo fullGTInfo, StorageContext context) {
+    public CubeHBaseRPC(ISegment segment, Cuboid cuboid, GTInfo fullGTInfo) {
         Preconditions.checkArgument(segment instanceof CubeSegment, "segment must be CubeSegment");
         
         this.cubeSeg = (CubeSegment) segment;
         this.cuboid = cuboid;
         this.fullGTInfo = fullGTInfo;
-        this.context = context;
+        this.queryContext = QueryContext.current();
 
         this.fuzzyKeyEncoder = new FuzzyKeyEncoder(cubeSeg, cuboid);
         this.fuzzyMaskEncoder = new FuzzyMaskEncoder(cubeSeg, cuboid);

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
index b94346c..1698180 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
@@ -42,7 +42,6 @@ import org.apache.kylin.gridtable.GTScanRequest;
 import org.apache.kylin.gridtable.IGTScanner;
 import org.apache.kylin.gridtable.IGTStore;
 import org.apache.kylin.metadata.model.ISegment;
-import org.apache.kylin.storage.StorageContext;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -88,8 +87,8 @@ public class CubeHBaseScanRPC extends CubeHBaseRPC {
         }
     }
 
-    public CubeHBaseScanRPC(ISegment segment, Cuboid cuboid, final GTInfo fullGTInfo, StorageContext context) {
-        super(segment, cuboid, fullGTInfo, context);
+    public CubeHBaseScanRPC(ISegment segment, Cuboid cuboid, final GTInfo fullGTInfo) {
+        super(segment, cuboid, fullGTInfo);
     }
 
     @Override
@@ -182,15 +181,18 @@ public class CubeHBaseScanRPC extends CubeHBaseRPC {
         final Iterator<Result> allResultsIterator = Iterators.concat(resultIterators.iterator());
 
         CellListIterator cellListIterator = new CellListIterator() {
-            long scanBytes = 0;
+            long scannedRows = 0;
+            long scannedBytes = 0;
 
             @Override
             public void close() throws IOException {
+                queryContext.addAndGetScannedRows(scannedRows);
+                queryContext.addAndGetScannedBytes(scannedBytes);
+
                 for (ResultScanner scanner : scanners) {
                     scanner.close();
                 }
                 hbaseTable.close();
-                context.increaseTotalScanBytes(scanBytes);
             }
 
             @Override
@@ -202,8 +204,9 @@ public class CubeHBaseScanRPC extends CubeHBaseRPC {
             public List<Cell> next() {
                 List<Cell> result = allResultsIterator.next().listCells();
                 for (Cell cell : result) {
-                    scanBytes += CellUtil.estimatedSizeOf(cell);
+                    scannedBytes += CellUtil.estimatedSizeOf(cell);
                 }
+                scannedRows++;
                 return result;
             }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
index 5fd9740..e18ff0d 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
@@ -31,7 +31,6 @@ import java.util.Properties;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.ArrayUtils;
-import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.Coprocessor;
@@ -44,16 +43,15 @@ import org.apache.hadoop.hbase.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.exceptions.KylinTimeoutException;
+import org.apache.kylin.common.exceptions.ResourceLimitExceededException;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.CompressionUtils;
 import org.apache.kylin.common.util.SetThreadName;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.gridtable.GTRecord;
-import org.apache.kylin.gridtable.GTScanExceedThresholdException;
 import org.apache.kylin.gridtable.GTScanRequest;
-import org.apache.kylin.gridtable.GTScanSelfTerminatedException;
-import org.apache.kylin.gridtable.GTScanTimeoutException;
 import org.apache.kylin.gridtable.IGTScanner;
 import org.apache.kylin.gridtable.IGTStore;
 import org.apache.kylin.gridtable.StorageSideBehavior;
@@ -165,13 +163,13 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
         @Override
         public boolean hasNext() {
             if (rowCount > rowCountLimit) {
-                throw new GTScanExceedThresholdException("Number of rows scanned exceeds threshold " + rowCountLimit);
+                throw new ResourceLimitExceededException("scanned row count exceeds threshold " + rowCountLimit);
             }
             if (rowBytes > bytesLimit) {
-                throw new GTScanExceedThresholdException("Scanned " + rowBytes + " bytes exceeds threshold " + bytesLimit);
+                throw new ResourceLimitExceededException("scanned bytes " + rowBytes + " exceeds threshold " + bytesLimit);
             }
             if ((rowCount % GTScanRequest.terminateCheckInterval == 1) && System.currentTimeMillis() > deadline) {
-                throw new GTScanTimeoutException("Scan timeout after " + timeout + " ms");
+                throw new KylinTimeoutException("coprocessor timeout after " + timeout + " ms");
             }
             return delegate.hasNext();
         }
@@ -232,6 +230,8 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
         byte[] allRows;
         String debugGitTag = "";
 
+        CubeVisitProtos.CubeVisitResponse.ErrorInfo errorInfo = null;
+
         String queryId = request.hasQueryId() ? request.getQueryId() : "UnknownId";
         try (SetThreadName ignored = new SetThreadName("Query %s", queryId)) {
             this.serviceStartTime = System.currentTimeMillis();
@@ -292,7 +292,6 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
                 scanReq.disableAggCacheMemCheck(); // disable mem check if so told
             }
 
-            final MutableBoolean scanNormalComplete = new MutableBoolean(true);
             final long storagePushDownLimit = scanReq.getStoragePushDownLimit();
 
             ResourceTrackingCellListIterator cellListIterator = new ResourceTrackingCellListIterator(
@@ -332,11 +331,18 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
                         break;
                     }
                 }
-            } catch (GTScanSelfTerminatedException e) {
-                // the query is using too much resource, we mark it as abnormal finish instead of
-                // throwing RuntimeException to avoid client retrying RPC.
-                scanNormalComplete.setValue(false);
-                logger.warn("Abort scan: {}", e.getMessage());
+            } catch (KylinTimeoutException e) {
+                logger.info("Abort scan: {}", e.getMessage());
+                errorInfo = CubeVisitProtos.CubeVisitResponse.ErrorInfo.newBuilder()
+                        .setType(CubeVisitProtos.CubeVisitResponse.ErrorType.TIMEOUT)
+                        .setMessage(e.getMessage())
+                        .build();
+            } catch (ResourceLimitExceededException e) {
+                logger.info("Abort scan: {}", e.getMessage());
+                errorInfo = CubeVisitProtos.CubeVisitResponse.ErrorInfo.newBuilder()
+                        .setType(CubeVisitProtos.CubeVisitResponse.ErrorType.RESOURCE_LIMIT_EXCEEDED)
+                        .setMessage(e.getMessage())
+                        .build();
             } finally {
                 finalScanner.close();
             }
@@ -347,7 +353,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
 
             //outputStream.close() is not necessary
             byte[] compressedAllRows;
-            if (scanNormalComplete.booleanValue()) {
+            if (errorInfo == null) {
                 allRows = outputStream.toByteArray();
             } else {
                 allRows = new byte[0];
@@ -370,6 +376,9 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
             sb.append(" debugGitTag:" + debugGitTag);
 
             CubeVisitProtos.CubeVisitResponse.Builder responseBuilder = CubeVisitProtos.CubeVisitResponse.newBuilder();
+            if (errorInfo != null) {
+                responseBuilder.setErrorInfo(errorInfo);
+            }
             done.run(responseBuilder.//
                     setCompressedRows(HBaseZeroCopyByteString.wrap(compressedAllRows)).//too many array copies 
                     setStats(CubeVisitProtos.CubeVisitResponse.Stats.newBuilder().
@@ -383,9 +392,8 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
                             setFreeSwapSpaceSize(freeSwapSpaceSize).
                             setHostname(InetAddress.getLocalHost().getHostName()).
                             setEtcMsg(sb.toString()).
-                            setNormalComplete(scanNormalComplete.booleanValue() ? 1 : 0).build())
-                    .//
-                    build());
+                            setNormalComplete(errorInfo == null ? 1 : 0).build())
+                    .build());
 
         } catch (IOException ioe) {
             logger.error(ioe.toString(), ioe);


[11/13] kylin git commit: KYLIN-2307 Create a branch for master with HBase 0.98 API

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/tool/src/main/java/org/apache/kylin/tool/ExtendCubeToHybridCLI.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/ExtendCubeToHybridCLI.java b/tool/src/main/java/org/apache/kylin/tool/ExtendCubeToHybridCLI.java
index f52fc3e..19e5db0 100644
--- a/tool/src/main/java/org/apache/kylin/tool/ExtendCubeToHybridCLI.java
+++ b/tool/src/main/java/org/apache/kylin/tool/ExtendCubeToHybridCLI.java
@@ -25,11 +25,10 @@ import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
@@ -232,9 +231,9 @@ public class ExtendCubeToHybridCLI {
         Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
         ProjectInstance project = store.getResource(projectResPath, ProjectInstance.class, projectSerializer);
         String projUUID = project.getUuid();
-        Table aclHtable = null;
+        HTableInterface aclHtable = null;
         try {
-            aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(TableName.valueOf(kylinConfig.getMetadataUrlPrefix() + "_acl"));
+            aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(kylinConfig.getMetadataUrlPrefix() + "_acl");
 
             // cube acl
             Result result = aclHtable.get(new Get(Bytes.toBytes(origCubeId)));
@@ -254,6 +253,7 @@ public class ExtendCubeToHybridCLI {
                     aclHtable.put(put);
                 }
             }
+            aclHtable.flushCommits();
         } finally {
             IOUtils.closeQuietly(aclHtable);
         }


[07/13] kylin git commit: minor, make FileResourceStore synchronized

Posted by li...@apache.org.
minor, make FileResourceStore synchronized


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

Branch: refs/heads/master-hbase0.98
Commit: cd4e116ac3391d4da975a29c21a0c69f053463a6
Parents: 3c49c9e
Author: Yang Li <li...@apache.org>
Authored: Fri Feb 10 21:41:50 2017 +0800
Committer: Yang Li <li...@apache.org>
Committed: Fri Feb 10 21:41:50 2017 +0800

----------------------------------------------------------------------
 .../common/persistence/FileResourceStore.java     | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/cd4e116a/core-common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java b/core-common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
index 3e012f5..d84e587 100644
--- a/core-common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
+++ b/core-common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
@@ -50,7 +50,7 @@ public class FileResourceStore extends ResourceStore {
     }
 
     @Override
-    protected NavigableSet<String> listResourcesImpl(String folderPath) throws IOException {
+    synchronized protected NavigableSet<String> listResourcesImpl(String folderPath) throws IOException {
         String[] names = file(folderPath).list();
         if (names == null) // not a directory
             return null;
@@ -64,13 +64,13 @@ public class FileResourceStore extends ResourceStore {
     }
 
     @Override
-    protected boolean existsImpl(String resPath) throws IOException {
+    synchronized protected boolean existsImpl(String resPath) throws IOException {
         File f = file(resPath);
         return f.exists() && f.isFile(); // directory is not considered a resource
     }
 
     @Override
-    protected List<RawResource> getAllResourcesImpl(String folderPath, long timeStart, long timeEndExclusive) throws IOException {
+    synchronized protected List<RawResource> getAllResourcesImpl(String folderPath, long timeStart, long timeEndExclusive) throws IOException {
         NavigableSet<String> resources = listResources(folderPath);
         if (resources == null)
             return Collections.emptyList();
@@ -95,7 +95,7 @@ public class FileResourceStore extends ResourceStore {
     }
 
     @Override
-    protected RawResource getResourceImpl(String resPath) throws IOException {
+    synchronized protected RawResource getResourceImpl(String resPath) throws IOException {
         File f = file(resPath);
         if (f.exists() && f.isFile()) {
             if (f.length() == 0) {
@@ -108,7 +108,7 @@ public class FileResourceStore extends ResourceStore {
     }
 
     @Override
-    protected long getResourceTimestampImpl(String resPath) throws IOException {
+    synchronized protected long getResourceTimestampImpl(String resPath) throws IOException {
         File f = file(resPath);
         if (f.exists() && f.isFile())
             return f.lastModified();
@@ -117,7 +117,7 @@ public class FileResourceStore extends ResourceStore {
     }
 
     @Override
-    protected void putResourceImpl(String resPath, InputStream content, long ts) throws IOException {
+    synchronized protected void putResourceImpl(String resPath, InputStream content, long ts) throws IOException {
         File f = file(resPath);
         f.getParentFile().mkdirs();
         FileOutputStream out = new FileOutputStream(f);
@@ -131,7 +131,7 @@ public class FileResourceStore extends ResourceStore {
     }
 
     @Override
-    protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException {
+    synchronized protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException {
         File f = file(resPath);
         if ((f.exists() && f.lastModified() != oldTS) || (f.exists() == false && oldTS != 0))
             throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + oldTS + ", but found " + f.lastModified());
@@ -143,13 +143,13 @@ public class FileResourceStore extends ResourceStore {
     }
 
     @Override
-    protected void deleteResourceImpl(String resPath) throws IOException {
+    synchronized protected void deleteResourceImpl(String resPath) throws IOException {
         File f = file(resPath);
         f.delete();
     }
 
     @Override
-    protected String getReadableResourcePathImpl(String resPath) {
+    synchronized protected String getReadableResourcePathImpl(String resPath) {
         return file(resPath).toString();
     }
 


[04/13] kylin git commit: minor, UHCReducerCount change back to 3 by default to make IT cover KYLIN-2135

Posted by li...@apache.org.
minor, UHCReducerCount change back to 3 by default to make IT cover KYLIN-2135


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

Branch: refs/heads/master-hbase0.98
Commit: edf6cef15753eae8675e9f2f4cf4ba5c7fb4e513
Parents: e09338b
Author: kangkaisen <ka...@163.com>
Authored: Fri Feb 10 17:36:51 2017 +0800
Committer: kangkaisen <ka...@163.com>
Committed: Fri Feb 10 17:36:51 2017 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/kylin/common/KylinConfigBase.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/edf6cef1/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 81f0187..894e28a 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
@@ -746,7 +746,7 @@ abstract public class KylinConfigBase implements Serializable {
 
     //UHC: ultra high cardinality columns, contain the ShardByColumns and the GlobalDictionaryColumns
     public int getUHCReducerCount() {
-        return Integer.parseInt(getOptional("kylin.engine.mr.uhc-reducer-count", "1"));
+        return Integer.parseInt(getOptional("kylin.engine.mr.uhc-reducer-count", "3"));
     }
 
     public boolean isBuildDictInReducerEnabled() {


[09/13] kylin git commit: KYLIN-2443 Report coprocessor error information back to client

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
index 4b6fc95..b9f2771 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java
@@ -121,10 +121,18 @@ public final class CubeVisitProtos {
     // optional int64 maxScanBytes = 8;
     /**
      * <code>optional int64 maxScanBytes = 8;</code>
+     *
+     * <pre>
+     * 0 means no limit
+     * </pre>
      */
     boolean hasMaxScanBytes();
     /**
      * <code>optional int64 maxScanBytes = 8;</code>
+     *
+     * <pre>
+     * 0 means no limit
+     * </pre>
      */
     long getMaxScanBytes();
   }
@@ -972,12 +980,20 @@ public final class CubeVisitProtos {
     private long maxScanBytes_;
     /**
      * <code>optional int64 maxScanBytes = 8;</code>
+     *
+     * <pre>
+     * 0 means no limit
+     * </pre>
      */
     public boolean hasMaxScanBytes() {
       return ((bitField0_ & 0x00000040) == 0x00000040);
     }
     /**
      * <code>optional int64 maxScanBytes = 8;</code>
+     *
+     * <pre>
+     * 0 means no limit
+     * </pre>
      */
     public long getMaxScanBytes() {
       return maxScanBytes_;
@@ -2050,18 +2066,30 @@ public final class CubeVisitProtos {
       private long maxScanBytes_ ;
       /**
        * <code>optional int64 maxScanBytes = 8;</code>
+       *
+       * <pre>
+       * 0 means no limit
+       * </pre>
        */
       public boolean hasMaxScanBytes() {
         return ((bitField0_ & 0x00000080) == 0x00000080);
       }
       /**
        * <code>optional int64 maxScanBytes = 8;</code>
+       *
+       * <pre>
+       * 0 means no limit
+       * </pre>
        */
       public long getMaxScanBytes() {
         return maxScanBytes_;
       }
       /**
        * <code>optional int64 maxScanBytes = 8;</code>
+       *
+       * <pre>
+       * 0 means no limit
+       * </pre>
        */
       public Builder setMaxScanBytes(long value) {
         bitField0_ |= 0x00000080;
@@ -2071,6 +2099,10 @@ public final class CubeVisitProtos {
       }
       /**
        * <code>optional int64 maxScanBytes = 8;</code>
+       *
+       * <pre>
+       * 0 means no limit
+       * </pre>
        */
       public Builder clearMaxScanBytes() {
         bitField0_ = (bitField0_ & ~0x00000080);
@@ -2116,6 +2148,32 @@ public final class CubeVisitProtos {
      * <code>required .CubeVisitResponse.Stats stats = 2;</code>
      */
     org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.StatsOrBuilder getStatsOrBuilder();
+
+    // optional .CubeVisitResponse.ErrorInfo errorInfo = 3;
+    /**
+     * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+     *
+     * <pre>
+     * should be set when stats.normalComplete == false
+     * </pre>
+     */
+    boolean hasErrorInfo();
+    /**
+     * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+     *
+     * <pre>
+     * should be set when stats.normalComplete == false
+     * </pre>
+     */
+    org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo getErrorInfo();
+    /**
+     * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+     *
+     * <pre>
+     * should be set when stats.normalComplete == false
+     * </pre>
+     */
+    org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfoOrBuilder getErrorInfoOrBuilder();
   }
   /**
    * Protobuf type {@code CubeVisitResponse}
@@ -2186,6 +2244,19 @@ public final class CubeVisitProtos {
               bitField0_ |= 0x00000002;
               break;
             }
+            case 26: {
+              org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                subBuilder = errorInfo_.toBuilder();
+              }
+              errorInfo_ = input.readMessage(org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(errorInfo_);
+                errorInfo_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000004;
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -2225,6 +2296,97 @@ public final class CubeVisitProtos {
       return PARSER;
     }
 
+    /**
+     * Protobuf enum {@code CubeVisitResponse.ErrorType}
+     */
+    public enum ErrorType
+        implements com.google.protobuf.ProtocolMessageEnum {
+      /**
+       * <code>UNKNOWN_TYPE = 0;</code>
+       */
+      UNKNOWN_TYPE(0, 0),
+      /**
+       * <code>TIMEOUT = 1;</code>
+       */
+      TIMEOUT(1, 1),
+      /**
+       * <code>RESOURCE_LIMIT_EXCEEDED = 2;</code>
+       */
+      RESOURCE_LIMIT_EXCEEDED(2, 2),
+      ;
+
+      /**
+       * <code>UNKNOWN_TYPE = 0;</code>
+       */
+      public static final int UNKNOWN_TYPE_VALUE = 0;
+      /**
+       * <code>TIMEOUT = 1;</code>
+       */
+      public static final int TIMEOUT_VALUE = 1;
+      /**
+       * <code>RESOURCE_LIMIT_EXCEEDED = 2;</code>
+       */
+      public static final int RESOURCE_LIMIT_EXCEEDED_VALUE = 2;
+
+
+      public final int getNumber() { return value; }
+
+      public static ErrorType valueOf(int value) {
+        switch (value) {
+          case 0: return UNKNOWN_TYPE;
+          case 1: return TIMEOUT;
+          case 2: return RESOURCE_LIMIT_EXCEEDED;
+          default: return null;
+        }
+      }
+
+      public static com.google.protobuf.Internal.EnumLiteMap<ErrorType>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<ErrorType>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<ErrorType>() {
+              public ErrorType findValueByNumber(int number) {
+                return ErrorType.valueOf(number);
+              }
+            };
+
+      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+          getValueDescriptor() {
+        return getDescriptor().getValues().get(index);
+      }
+      public final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptorForType() {
+        return getDescriptor();
+      }
+      public static final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptor() {
+        return org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.getDescriptor().getEnumTypes().get(0);
+      }
+
+      private static final ErrorType[] VALUES = values();
+
+      public static ErrorType valueOf(
+          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        if (desc.getType() != getDescriptor()) {
+          throw new java.lang.IllegalArgumentException(
+            "EnumValueDescriptor is not for this type.");
+        }
+        return VALUES[desc.getIndex()];
+      }
+
+      private final int index;
+      private final int value;
+
+      private ErrorType(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+
+      // @@protoc_insertion_point(enum_scope:CubeVisitResponse.ErrorType)
+    }
+
     public interface StatsOrBuilder
         extends com.google.protobuf.MessageOrBuilder {
 
@@ -2331,18 +2493,10 @@ public final class CubeVisitProtos {
       // optional int32 normalComplete = 10;
       /**
        * <code>optional int32 normalComplete = 10;</code>
-       *
-       * <pre>
-       *when time outs, normalComplete will be false
-       * </pre>
        */
       boolean hasNormalComplete();
       /**
        * <code>optional int32 normalComplete = 10;</code>
-       *
-       * <pre>
-       *when time outs, normalComplete will be false
-       * </pre>
        */
       int getNormalComplete();
 
@@ -2705,20 +2859,12 @@ public final class CubeVisitProtos {
       private int normalComplete_;
       /**
        * <code>optional int32 normalComplete = 10;</code>
-       *
-       * <pre>
-       *when time outs, normalComplete will be false
-       * </pre>
        */
       public boolean hasNormalComplete() {
         return ((bitField0_ & 0x00000200) == 0x00000200);
       }
       /**
        * <code>optional int32 normalComplete = 10;</code>
-       *
-       * <pre>
-       *when time outs, normalComplete will be false
-       * </pre>
        */
       public int getNormalComplete() {
         return normalComplete_;
@@ -3652,30 +3798,18 @@ public final class CubeVisitProtos {
         private int normalComplete_ ;
         /**
          * <code>optional int32 normalComplete = 10;</code>
-         *
-         * <pre>
-         *when time outs, normalComplete will be false
-         * </pre>
          */
         public boolean hasNormalComplete() {
           return ((bitField0_ & 0x00000200) == 0x00000200);
         }
         /**
          * <code>optional int32 normalComplete = 10;</code>
-         *
-         * <pre>
-         *when time outs, normalComplete will be false
-         * </pre>
          */
         public int getNormalComplete() {
           return normalComplete_;
         }
         /**
          * <code>optional int32 normalComplete = 10;</code>
-         *
-         * <pre>
-         *when time outs, normalComplete will be false
-         * </pre>
          */
         public Builder setNormalComplete(int value) {
           bitField0_ |= 0x00000200;
@@ -3685,10 +3819,6 @@ public final class CubeVisitProtos {
         }
         /**
          * <code>optional int32 normalComplete = 10;</code>
-         *
-         * <pre>
-         *when time outs, normalComplete will be false
-         * </pre>
          */
         public Builder clearNormalComplete() {
           bitField0_ = (bitField0_ & ~0x00000200);
@@ -3741,99 +3871,771 @@ public final class CubeVisitProtos {
       // @@protoc_insertion_point(class_scope:CubeVisitResponse.Stats)
     }
 
-    private int bitField0_;
-    // required bytes compressedRows = 1;
-    public static final int COMPRESSEDROWS_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString compressedRows_;
-    /**
-     * <code>required bytes compressedRows = 1;</code>
-     */
-    public boolean hasCompressedRows() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required bytes compressedRows = 1;</code>
-     */
-    public com.google.protobuf.ByteString getCompressedRows() {
-      return compressedRows_;
-    }
+    public interface ErrorInfoOrBuilder
+        extends com.google.protobuf.MessageOrBuilder {
 
-    // required .CubeVisitResponse.Stats stats = 2;
-    public static final int STATS_FIELD_NUMBER = 2;
-    private org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.Stats stats_;
-    /**
-     * <code>required .CubeVisitResponse.Stats stats = 2;</code>
-     */
-    public boolean hasStats() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>required .CubeVisitResponse.Stats stats = 2;</code>
-     */
-    public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.Stats getStats() {
-      return stats_;
+      // required .CubeVisitResponse.ErrorType type = 1;
+      /**
+       * <code>required .CubeVisitResponse.ErrorType type = 1;</code>
+       */
+      boolean hasType();
+      /**
+       * <code>required .CubeVisitResponse.ErrorType type = 1;</code>
+       */
+      org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorType getType();
+
+      // required string message = 2;
+      /**
+       * <code>required string message = 2;</code>
+       */
+      boolean hasMessage();
+      /**
+       * <code>required string message = 2;</code>
+       */
+      java.lang.String getMessage();
+      /**
+       * <code>required string message = 2;</code>
+       */
+      com.google.protobuf.ByteString
+          getMessageBytes();
     }
     /**
-     * <code>required .CubeVisitResponse.Stats stats = 2;</code>
+     * Protobuf type {@code CubeVisitResponse.ErrorInfo}
      */
-    public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.StatsOrBuilder getStatsOrBuilder() {
-      return stats_;
-    }
-
-    private void initFields() {
-      compressedRows_ = com.google.protobuf.ByteString.EMPTY;
-      stats_ = org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.Stats.getDefaultInstance();
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
+    public static final class ErrorInfo extends
+        com.google.protobuf.GeneratedMessage
+        implements ErrorInfoOrBuilder {
+      // Use ErrorInfo.newBuilder() to construct.
+      private ErrorInfo(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+        super(builder);
+        this.unknownFields = builder.getUnknownFields();
+      }
+      private ErrorInfo(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-      if (!hasCompressedRows()) {
-        memoizedIsInitialized = 0;
-        return false;
+      private static final ErrorInfo defaultInstance;
+      public static ErrorInfo getDefaultInstance() {
+        return defaultInstance;
       }
-      if (!hasStats()) {
-        memoizedIsInitialized = 0;
-        return false;
+
+      public ErrorInfo getDefaultInstanceForType() {
+        return defaultInstance;
       }
-      memoizedIsInitialized = 1;
-      return true;
-    }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBytes(1, compressedRows_);
+      private final com.google.protobuf.UnknownFieldSet unknownFields;
+      @java.lang.Override
+      public final com.google.protobuf.UnknownFieldSet
+          getUnknownFields() {
+        return this.unknownFields;
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeMessage(2, stats_);
+      private ErrorInfo(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        initFields();
+        int mutable_bitField0_ = 0;
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 8: {
+                int rawValue = input.readEnum();
+                org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorType value = org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorType.valueOf(rawValue);
+                if (value == null) {
+                  unknownFields.mergeVarintField(1, rawValue);
+                } else {
+                  bitField0_ |= 0x00000001;
+                  type_ = value;
+                }
+                break;
+              }
+              case 18: {
+                bitField0_ |= 0x00000002;
+                message_ = input.readBytes();
+                break;
+              }
+            }
+          }
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new com.google.protobuf.InvalidProtocolBufferException(
+              e.getMessage()).setUnfinishedMessage(this);
+        } finally {
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.internal_static_CubeVisitResponse_ErrorInfo_descriptor;
       }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
 
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(1, compressedRows_);
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.internal_static_CubeVisitResponse_ErrorInfo_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.class, org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.Builder.class);
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, stats_);
+
+      public static com.google.protobuf.Parser<ErrorInfo> PARSER =
+          new com.google.protobuf.AbstractParser<ErrorInfo>() {
+        public ErrorInfo parsePartialFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return new ErrorInfo(input, extensionRegistry);
+        }
+      };
+
+      @java.lang.Override
+      public com.google.protobuf.Parser<ErrorInfo> getParserForType() {
+        return PARSER;
       }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
 
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
+      private int bitField0_;
+      // required .CubeVisitResponse.ErrorType type = 1;
+      public static final int TYPE_FIELD_NUMBER = 1;
+      private org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorType type_;
+      /**
+       * <code>required .CubeVisitResponse.ErrorType type = 1;</code>
+       */
+      public boolean hasType() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .CubeVisitResponse.ErrorType type = 1;</code>
+       */
+      public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorType getType() {
+        return type_;
+      }
+
+      // required string message = 2;
+      public static final int MESSAGE_FIELD_NUMBER = 2;
+      private java.lang.Object message_;
+      /**
+       * <code>required string message = 2;</code>
+       */
+      public boolean hasMessage() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required string message = 2;</code>
+       */
+      public java.lang.String getMessage() {
+        java.lang.Object ref = message_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            message_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string message = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getMessageBytes() {
+        java.lang.Object ref = message_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          message_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      private void initFields() {
+        type_ = org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorType.UNKNOWN_TYPE;
+        message_ = "";
+      }
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized != -1) return isInitialized == 1;
+
+        if (!hasType()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        if (!hasMessage()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        memoizedIsInitialized = 1;
+        return true;
+      }
+
+      public void writeTo(com.google.protobuf.CodedOutputStream output)
+                          throws java.io.IOException {
+        getSerializedSize();
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          output.writeEnum(1, type_.getNumber());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeBytes(2, getMessageBytes());
+        }
+        getUnknownFields().writeTo(output);
+      }
+
+      private int memoizedSerializedSize = -1;
+      public int getSerializedSize() {
+        int size = memoizedSerializedSize;
+        if (size != -1) return size;
+
+        size = 0;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeEnumSize(1, type_.getNumber());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(2, getMessageBytes());
+        }
+        size += getUnknownFields().getSerializedSize();
+        memoizedSerializedSize = size;
+        return size;
+      }
+
+      private static final long serialVersionUID = 0L;
+      @java.lang.Override
+      protected java.lang.Object writeReplace()
+          throws java.io.ObjectStreamException {
+        return super.writeReplace();
+      }
+
+      @java.lang.Override
+      public boolean equals(final java.lang.Object obj) {
+        if (obj == this) {
+         return true;
+        }
+        if (!(obj instanceof org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo)) {
+          return super.equals(obj);
+        }
+        org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo other = (org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo) obj;
+
+        boolean result = true;
+        result = result && (hasType() == other.hasType());
+        if (hasType()) {
+          result = result &&
+              (getType() == other.getType());
+        }
+        result = result && (hasMessage() == other.hasMessage());
+        if (hasMessage()) {
+          result = result && getMessage()
+              .equals(other.getMessage());
+        }
+        result = result &&
+            getUnknownFields().equals(other.getUnknownFields());
+        return result;
+      }
+
+      private int memoizedHashCode = 0;
+      @java.lang.Override
+      public int hashCode() {
+        if (memoizedHashCode != 0) {
+          return memoizedHashCode;
+        }
+        int hash = 41;
+        hash = (19 * hash) + getDescriptorForType().hashCode();
+        if (hasType()) {
+          hash = (37 * hash) + TYPE_FIELD_NUMBER;
+          hash = (53 * hash) + hashEnum(getType());
+        }
+        if (hasMessage()) {
+          hash = (37 * hash) + MESSAGE_FIELD_NUMBER;
+          hash = (53 * hash) + getMessage().hashCode();
+        }
+        hash = (29 * hash) + getUnknownFields().hashCode();
+        memoizedHashCode = hash;
+        return hash;
+      }
+
+      public static org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo parseFrom(
+          com.google.protobuf.ByteString data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo parseFrom(
+          com.google.protobuf.ByteString data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo parseFrom(byte[] data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo parseFrom(
+          byte[] data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo parseFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+      public static org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input);
+      }
+      public static org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo parseDelimitedFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input, extensionRegistry);
+      }
+      public static org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo parseFrom(
+          com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo parseFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+
+      public static Builder newBuilder() { return Builder.create(); }
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder(org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo prototype) {
+        return newBuilder().mergeFrom(prototype);
+      }
+      public Builder toBuilder() { return newBuilder(this); }
+
+      @java.lang.Override
+      protected Builder newBuilderForType(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        Builder builder = new Builder(parent);
+        return builder;
+      }
+      /**
+       * Protobuf type {@code CubeVisitResponse.ErrorInfo}
+       */
+      public static final class Builder extends
+          com.google.protobuf.GeneratedMessage.Builder<Builder>
+         implements org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfoOrBuilder {
+        public static final com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.internal_static_CubeVisitResponse_ErrorInfo_descriptor;
+        }
+
+        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.internal_static_CubeVisitResponse_ErrorInfo_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.class, org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.Builder.class);
+        }
+
+        // Construct using org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.newBuilder()
+        private Builder() {
+          maybeForceBuilderInitialization();
+        }
+
+        private Builder(
+            com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+          super(parent);
+          maybeForceBuilderInitialization();
+        }
+        private void maybeForceBuilderInitialization() {
+          if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          }
+        }
+        private static Builder create() {
+          return new Builder();
+        }
+
+        public Builder clear() {
+          super.clear();
+          type_ = org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorType.UNKNOWN_TYPE;
+          bitField0_ = (bitField0_ & ~0x00000001);
+          message_ = "";
+          bitField0_ = (bitField0_ & ~0x00000002);
+          return this;
+        }
+
+        public Builder clone() {
+          return create().mergeFrom(buildPartial());
+        }
+
+        public com.google.protobuf.Descriptors.Descriptor
+            getDescriptorForType() {
+          return org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.internal_static_CubeVisitResponse_ErrorInfo_descriptor;
+        }
+
+        public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo getDefaultInstanceForType() {
+          return org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.getDefaultInstance();
+        }
+
+        public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo build() {
+          org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo result = buildPartial();
+          if (!result.isInitialized()) {
+            throw newUninitializedMessageException(result);
+          }
+          return result;
+        }
+
+        public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo buildPartial() {
+          org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo result = new org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo(this);
+          int from_bitField0_ = bitField0_;
+          int to_bitField0_ = 0;
+          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+            to_bitField0_ |= 0x00000001;
+          }
+          result.type_ = type_;
+          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+            to_bitField0_ |= 0x00000002;
+          }
+          result.message_ = message_;
+          result.bitField0_ = to_bitField0_;
+          onBuilt();
+          return result;
+        }
+
+        public Builder mergeFrom(com.google.protobuf.Message other) {
+          if (other instanceof org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo) {
+            return mergeFrom((org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo)other);
+          } else {
+            super.mergeFrom(other);
+            return this;
+          }
+        }
+
+        public Builder mergeFrom(org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo other) {
+          if (other == org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.getDefaultInstance()) return this;
+          if (other.hasType()) {
+            setType(other.getType());
+          }
+          if (other.hasMessage()) {
+            bitField0_ |= 0x00000002;
+            message_ = other.message_;
+            onChanged();
+          }
+          this.mergeUnknownFields(other.getUnknownFields());
+          return this;
+        }
+
+        public final boolean isInitialized() {
+          if (!hasType()) {
+            
+            return false;
+          }
+          if (!hasMessage()) {
+            
+            return false;
+          }
+          return true;
+        }
+
+        public Builder mergeFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo parsedMessage = null;
+          try {
+            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+            parsedMessage = (org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo) e.getUnfinishedMessage();
+            throw e;
+          } finally {
+            if (parsedMessage != null) {
+              mergeFrom(parsedMessage);
+            }
+          }
+          return this;
+        }
+        private int bitField0_;
+
+        // required .CubeVisitResponse.ErrorType type = 1;
+        private org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorType type_ = org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorType.UNKNOWN_TYPE;
+        /**
+         * <code>required .CubeVisitResponse.ErrorType type = 1;</code>
+         */
+        public boolean hasType() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>required .CubeVisitResponse.ErrorType type = 1;</code>
+         */
+        public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorType getType() {
+          return type_;
+        }
+        /**
+         * <code>required .CubeVisitResponse.ErrorType type = 1;</code>
+         */
+        public Builder setType(org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorType value) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          bitField0_ |= 0x00000001;
+          type_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required .CubeVisitResponse.ErrorType type = 1;</code>
+         */
+        public Builder clearType() {
+          bitField0_ = (bitField0_ & ~0x00000001);
+          type_ = org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorType.UNKNOWN_TYPE;
+          onChanged();
+          return this;
+        }
+
+        // required string message = 2;
+        private java.lang.Object message_ = "";
+        /**
+         * <code>required string message = 2;</code>
+         */
+        public boolean hasMessage() {
+          return ((bitField0_ & 0x00000002) == 0x00000002);
+        }
+        /**
+         * <code>required string message = 2;</code>
+         */
+        public java.lang.String getMessage() {
+          java.lang.Object ref = message_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            message_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string message = 2;</code>
+         */
+        public com.google.protobuf.ByteString
+            getMessageBytes() {
+          java.lang.Object ref = message_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            message_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string message = 2;</code>
+         */
+        public Builder setMessage(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+          message_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string message = 2;</code>
+         */
+        public Builder clearMessage() {
+          bitField0_ = (bitField0_ & ~0x00000002);
+          message_ = getDefaultInstance().getMessage();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string message = 2;</code>
+         */
+        public Builder setMessageBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+          message_ = value;
+          onChanged();
+          return this;
+        }
+
+        // @@protoc_insertion_point(builder_scope:CubeVisitResponse.ErrorInfo)
+      }
+
+      static {
+        defaultInstance = new ErrorInfo(true);
+        defaultInstance.initFields();
+      }
+
+      // @@protoc_insertion_point(class_scope:CubeVisitResponse.ErrorInfo)
+    }
+
+    private int bitField0_;
+    // required bytes compressedRows = 1;
+    public static final int COMPRESSEDROWS_FIELD_NUMBER = 1;
+    private com.google.protobuf.ByteString compressedRows_;
+    /**
+     * <code>required bytes compressedRows = 1;</code>
+     */
+    public boolean hasCompressedRows() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required bytes compressedRows = 1;</code>
+     */
+    public com.google.protobuf.ByteString getCompressedRows() {
+      return compressedRows_;
+    }
+
+    // required .CubeVisitResponse.Stats stats = 2;
+    public static final int STATS_FIELD_NUMBER = 2;
+    private org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.Stats stats_;
+    /**
+     * <code>required .CubeVisitResponse.Stats stats = 2;</code>
+     */
+    public boolean hasStats() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .CubeVisitResponse.Stats stats = 2;</code>
+     */
+    public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.Stats getStats() {
+      return stats_;
+    }
+    /**
+     * <code>required .CubeVisitResponse.Stats stats = 2;</code>
+     */
+    public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.StatsOrBuilder getStatsOrBuilder() {
+      return stats_;
+    }
+
+    // optional .CubeVisitResponse.ErrorInfo errorInfo = 3;
+    public static final int ERRORINFO_FIELD_NUMBER = 3;
+    private org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo errorInfo_;
+    /**
+     * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+     *
+     * <pre>
+     * should be set when stats.normalComplete == false
+     * </pre>
+     */
+    public boolean hasErrorInfo() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+     *
+     * <pre>
+     * should be set when stats.normalComplete == false
+     * </pre>
+     */
+    public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo getErrorInfo() {
+      return errorInfo_;
+    }
+    /**
+     * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+     *
+     * <pre>
+     * should be set when stats.normalComplete == false
+     * </pre>
+     */
+    public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfoOrBuilder getErrorInfoOrBuilder() {
+      return errorInfo_;
+    }
+
+    private void initFields() {
+      compressedRows_ = com.google.protobuf.ByteString.EMPTY;
+      stats_ = org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.Stats.getDefaultInstance();
+      errorInfo_ = org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasCompressedRows()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasStats()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (hasErrorInfo()) {
+        if (!getErrorInfo().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, compressedRows_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, stats_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, errorInfo_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, compressedRows_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, stats_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, errorInfo_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
     protected java.lang.Object writeReplace()
         throws java.io.ObjectStreamException {
       return super.writeReplace();
@@ -3860,6 +4662,11 @@ public final class CubeVisitProtos {
         result = result && getStats()
             .equals(other.getStats());
       }
+      result = result && (hasErrorInfo() == other.hasErrorInfo());
+      if (hasErrorInfo()) {
+        result = result && getErrorInfo()
+            .equals(other.getErrorInfo());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -3881,6 +4688,10 @@ public final class CubeVisitProtos {
         hash = (37 * hash) + STATS_FIELD_NUMBER;
         hash = (53 * hash) + getStats().hashCode();
       }
+      if (hasErrorInfo()) {
+        hash = (37 * hash) + ERRORINFO_FIELD_NUMBER;
+        hash = (53 * hash) + getErrorInfo().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -3983,6 +4794,7 @@ public final class CubeVisitProtos {
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getStatsFieldBuilder();
+          getErrorInfoFieldBuilder();
         }
       }
       private static Builder create() {
@@ -3999,6 +4811,12 @@ public final class CubeVisitProtos {
           statsBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000002);
+        if (errorInfoBuilder_ == null) {
+          errorInfo_ = org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.getDefaultInstance();
+        } else {
+          errorInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -4039,6 +4857,14 @@ public final class CubeVisitProtos {
         } else {
           result.stats_ = statsBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (errorInfoBuilder_ == null) {
+          result.errorInfo_ = errorInfo_;
+        } else {
+          result.errorInfo_ = errorInfoBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -4061,6 +4887,9 @@ public final class CubeVisitProtos {
         if (other.hasStats()) {
           mergeStats(other.getStats());
         }
+        if (other.hasErrorInfo()) {
+          mergeErrorInfo(other.getErrorInfo());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -4074,6 +4903,12 @@ public final class CubeVisitProtos {
           
           return false;
         }
+        if (hasErrorInfo()) {
+          if (!getErrorInfo().isInitialized()) {
+            
+            return false;
+          }
+        }
         return true;
       }
 
@@ -4249,6 +5084,159 @@ public final class CubeVisitProtos {
         return statsBuilder_;
       }
 
+      // optional .CubeVisitResponse.ErrorInfo errorInfo = 3;
+      private org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo errorInfo_ = org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo, org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.Builder, org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfoOrBuilder> errorInfoBuilder_;
+      /**
+       * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+       *
+       * <pre>
+       * should be set when stats.normalComplete == false
+       * </pre>
+       */
+      public boolean hasErrorInfo() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+       *
+       * <pre>
+       * should be set when stats.normalComplete == false
+       * </pre>
+       */
+      public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo getErrorInfo() {
+        if (errorInfoBuilder_ == null) {
+          return errorInfo_;
+        } else {
+          return errorInfoBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+       *
+       * <pre>
+       * should be set when stats.normalComplete == false
+       * </pre>
+       */
+      public Builder setErrorInfo(org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo value) {
+        if (errorInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          errorInfo_ = value;
+          onChanged();
+        } else {
+          errorInfoBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+       *
+       * <pre>
+       * should be set when stats.normalComplete == false
+       * </pre>
+       */
+      public Builder setErrorInfo(
+          org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.Builder builderForValue) {
+        if (errorInfoBuilder_ == null) {
+          errorInfo_ = builderForValue.build();
+          onChanged();
+        } else {
+          errorInfoBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+       *
+       * <pre>
+       * should be set when stats.normalComplete == false
+       * </pre>
+       */
+      public Builder mergeErrorInfo(org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo value) {
+        if (errorInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              errorInfo_ != org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.getDefaultInstance()) {
+            errorInfo_ =
+              org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.newBuilder(errorInfo_).mergeFrom(value).buildPartial();
+          } else {
+            errorInfo_ = value;
+          }
+          onChanged();
+        } else {
+          errorInfoBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+       *
+       * <pre>
+       * should be set when stats.normalComplete == false
+       * </pre>
+       */
+      public Builder clearErrorInfo() {
+        if (errorInfoBuilder_ == null) {
+          errorInfo_ = org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.getDefaultInstance();
+          onChanged();
+        } else {
+          errorInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      /**
+       * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+       *
+       * <pre>
+       * should be set when stats.normalComplete == false
+       * </pre>
+       */
+      public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.Builder getErrorInfoBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getErrorInfoFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+       *
+       * <pre>
+       * should be set when stats.normalComplete == false
+       * </pre>
+       */
+      public org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfoOrBuilder getErrorInfoOrBuilder() {
+        if (errorInfoBuilder_ != null) {
+          return errorInfoBuilder_.getMessageOrBuilder();
+        } else {
+          return errorInfo_;
+        }
+      }
+      /**
+       * <code>optional .CubeVisitResponse.ErrorInfo errorInfo = 3;</code>
+       *
+       * <pre>
+       * should be set when stats.normalComplete == false
+       * </pre>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo, org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.Builder, org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfoOrBuilder> 
+          getErrorInfoFieldBuilder() {
+        if (errorInfoBuilder_ == null) {
+          errorInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo, org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfo.Builder, org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.ErrorInfoOrBuilder>(
+                  errorInfo_,
+                  getParentForChildren(),
+                  isClean());
+          errorInfo_ = null;
+        }
+        return errorInfoBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:CubeVisitResponse)
     }
 
@@ -4512,6 +5500,11 @@ public final class CubeVisitProtos {
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_CubeVisitResponse_Stats_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_CubeVisitResponse_ErrorInfo_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_CubeVisitResponse_ErrorInfo_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -4530,20 +5523,25 @@ public final class CubeVisitProtos {
       "eVisitRequest.IntList\022\027\n\017kylinProperties" +
       "\030\005 \002(\t\022\017\n\007queryId\030\006 \001(\t\022\032\n\014spillEnabled\030" +
       "\007 \001(\010:\004true\022\024\n\014maxScanBytes\030\010 \001(\003\032\027\n\007Int" +
-      "List\022\014\n\004ints\030\001 \003(\005\"\347\002\n\021CubeVisitResponse",
+      "List\022\014\n\004ints\030\001 \003(\005\"\253\004\n\021CubeVisitResponse",
       "\022\026\n\016compressedRows\030\001 \002(\014\022\'\n\005stats\030\002 \002(\0132" +
-      "\030.CubeVisitResponse.Stats\032\220\002\n\005Stats\022\030\n\020s" +
-      "erviceStartTime\030\001 \001(\003\022\026\n\016serviceEndTime\030" +
-      "\002 \001(\003\022\027\n\017scannedRowCount\030\003 \001(\003\022\032\n\022aggreg" +
-      "atedRowCount\030\004 \001(\003\022\025\n\rsystemCpuLoad\030\005 \001(" +
-      "\001\022\036\n\026freePhysicalMemorySize\030\006 \001(\001\022\031\n\021fre" +
-      "eSwapSpaceSize\030\007 \001(\001\022\020\n\010hostname\030\010 \001(\t\022\016" +
-      "\n\006etcMsg\030\t \001(\t\022\026\n\016normalComplete\030\n \001(\005\022\024" +
-      "\n\014scannedBytes\030\013 \001(\0032F\n\020CubeVisitService" +
-      "\0222\n\tvisitCube\022\021.CubeVisitRequest\032\022.CubeV",
-      "isitResponseB`\nEorg.apache.kylin.storage" +
-      ".hbase.cube.v2.coprocessor.endpoint.gene" +
-      "ratedB\017CubeVisitProtosH\001\210\001\001\240\001\001"
+      "\030.CubeVisitResponse.Stats\022/\n\terrorInfo\030\003" +
+      " \001(\0132\034.CubeVisitResponse.ErrorInfo\032\220\002\n\005S" +
+      "tats\022\030\n\020serviceStartTime\030\001 \001(\003\022\026\n\016servic" +
+      "eEndTime\030\002 \001(\003\022\027\n\017scannedRowCount\030\003 \001(\003\022" +
+      "\032\n\022aggregatedRowCount\030\004 \001(\003\022\025\n\rsystemCpu" +
+      "Load\030\005 \001(\001\022\036\n\026freePhysicalMemorySize\030\006 \001" +
+      "(\001\022\031\n\021freeSwapSpaceSize\030\007 \001(\001\022\020\n\010hostnam" +
+      "e\030\010 \001(\t\022\016\n\006etcMsg\030\t \001(\t\022\026\n\016normalComplet" +
+      "e\030\n \001(\005\022\024\n\014scannedBytes\030\013 \001(\003\032H\n\tErrorIn",
+      "fo\022*\n\004type\030\001 \002(\0162\034.CubeVisitResponse.Err" +
+      "orType\022\017\n\007message\030\002 \002(\t\"G\n\tErrorType\022\020\n\014" +
+      "UNKNOWN_TYPE\020\000\022\013\n\007TIMEOUT\020\001\022\033\n\027RESOURCE_" +
+      "LIMIT_EXCEEDED\020\0022F\n\020CubeVisitService\0222\n\t" +
+      "visitCube\022\021.CubeVisitRequest\032\022.CubeVisit" +
+      "ResponseB`\nEorg.apache.kylin.storage.hba" +
+      "se.cube.v2.coprocessor.endpoint.generate" +
+      "dB\017CubeVisitProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -4567,13 +5565,19 @@ public final class CubeVisitProtos {
           internal_static_CubeVisitResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_CubeVisitResponse_descriptor,
-              new java.lang.String[] { "CompressedRows", "Stats", });
+              new java.lang.String[] { "CompressedRows", "Stats", "ErrorInfo", });
           internal_static_CubeVisitResponse_Stats_descriptor =
             internal_static_CubeVisitResponse_descriptor.getNestedTypes().get(0);
           internal_static_CubeVisitResponse_Stats_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_CubeVisitResponse_Stats_descriptor,
               new java.lang.String[] { "ServiceStartTime", "ServiceEndTime", "ScannedRowCount", "AggregatedRowCount", "SystemCpuLoad", "FreePhysicalMemorySize", "FreeSwapSpaceSize", "Hostname", "EtcMsg", "NormalComplete", "ScannedBytes", });
+          internal_static_CubeVisitResponse_ErrorInfo_descriptor =
+            internal_static_CubeVisitResponse_descriptor.getNestedTypes().get(1);
+          internal_static_CubeVisitResponse_ErrorInfo_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_CubeVisitResponse_ErrorInfo_descriptor,
+              new java.lang.String[] { "Type", "Message", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/kylin/blob/43c05667/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
index 00015fc..e01ff52 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto
@@ -54,11 +54,21 @@ message CubeVisitResponse {
         optional double freeSwapSpaceSize = 7;
         optional string hostname = 8;
         optional string etcMsg = 9;
-        optional int32 normalComplete =10;//when time outs, normalComplete will be false
+        optional int32 normalComplete =10;
         optional int64 scannedBytes = 11;
     }
+    enum ErrorType {
+        UNKNOWN_TYPE = 0;
+        TIMEOUT = 1;
+        RESOURCE_LIMIT_EXCEEDED = 2;
+    }
+    message ErrorInfo {
+        required ErrorType type = 1;
+        required string message = 2;
+    }
     required bytes compressedRows = 1;
     required Stats stats = 2;
+    optional ErrorInfo errorInfo = 3; // should be set when stats.normalComplete == false
 }
 
 service CubeVisitService {


[13/13] kylin git commit: KYLIN-2307 Create a branch for master with HBase 0.98 API

Posted by li...@apache.org.
KYLIN-2307 Create a branch for master with HBase 0.98 API


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

Branch: refs/heads/master-hbase0.98
Commit: 0dc56aa4a786fc07d0731b02b32fc740b3a4a2ac
Parents: 43c0566
Author: lidongsjtu <li...@apache.org>
Authored: Mon Jan 23 13:17:37 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Sun Feb 12 22:28:39 2017 +0800

----------------------------------------------------------------------
 dev-support/test_all_against_hdp_2_2_4_2_2.sh   |  25 ++++
 dev-support/test_all_against_hdp_2_4_0_0_169.sh |  25 ----
 .../sandbox/capacity-scheduler.xml              |  17 ++-
 examples/test_case_data/sandbox/core-site.xml   |  28 +---
 examples/test_case_data/sandbox/hbase-site.xml  | 119 +++++------------
 examples/test_case_data/sandbox/hdfs-site.xml   |  84 +++++-------
 examples/test_case_data/sandbox/hive-site.xml   |  89 +++++--------
 examples/test_case_data/sandbox/mapred-site.xml |  57 +++------
 examples/test_case_data/sandbox/yarn-site.xml   | 127 +++----------------
 .../kylin/provision/BuildCubeWithEngine.java    |  17 +--
 pom.xml                                         | 122 +-----------------
 .../kylin/rest/security/AclHBaseStorage.java    |   4 +-
 .../rest/security/MockAclHBaseStorage.java      |   8 +-
 .../apache/kylin/rest/security/MockHTable.java  |  95 +++++++++++---
 .../rest/security/RealAclHBaseStorage.java      |   9 +-
 .../apache/kylin/rest/service/AclService.java   |  25 ++--
 .../apache/kylin/rest/service/CubeService.java  |  35 +++--
 .../apache/kylin/rest/service/QueryService.java |  24 ++--
 .../apache/kylin/rest/service/UserService.java  |  17 +--
 .../kylin/storage/hbase/HBaseConnection.java    |  44 +++----
 .../kylin/storage/hbase/HBaseResourceStore.java |  31 +++--
 .../storage/hbase/cube/SimpleHBaseStore.java    |  20 +--
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java     |  13 +-
 .../storage/hbase/cube/v2/CubeHBaseScanRPC.java |   9 +-
 .../coprocessor/endpoint/CubeVisitService.java  |   4 +-
 .../storage/hbase/steps/CubeHTableUtil.java     |  16 +--
 .../storage/hbase/steps/DeprecatedGCStep.java   |  24 ++--
 .../storage/hbase/steps/HBaseCuboidWriter.java  |   7 +-
 .../kylin/storage/hbase/steps/MergeGCStep.java  |  23 ++--
 .../storage/hbase/util/CleanHtableCLI.java      |  12 +-
 .../storage/hbase/util/CubeMigrationCLI.java    |  37 +++---
 .../hbase/util/CubeMigrationCheckCLI.java       |  17 +--
 .../hbase/util/DeployCoprocessorCLI.java        |  27 ++--
 .../hbase/util/ExtendCubeToHybridCLI.java       |   8 +-
 .../hbase/util/GridTableHBaseBenchmark.java     |  34 ++---
 .../kylin/storage/hbase/util/HBaseClean.java    |  18 ++-
 .../hbase/util/HBaseRegionSizeCalculator.java   |  35 +++--
 .../kylin/storage/hbase/util/HBaseUsage.java    |   9 +-
 .../storage/hbase/util/HbaseStreamingInput.java |  30 ++---
 .../hbase/util/HtableAlterMetadataCLI.java      |   9 +-
 .../storage/hbase/util/OrphanHBaseCleanJob.java |  19 +--
 .../kylin/storage/hbase/util/PingHBaseCLI.java  |  15 +--
 .../kylin/storage/hbase/util/RowCounterCLI.java |  11 +-
 .../storage/hbase/util/StorageCleanupJob.java   |  20 ++-
 .../storage/hbase/util/UpdateHTableHostCLI.java |  17 +--
 tool/pom.xml                                    |  10 --
 .../org/apache/kylin/tool/CubeMigrationCLI.java |  19 +--
 .../kylin/tool/ExtendCubeToHybridCLI.java       |   8 +-
 48 files changed, 596 insertions(+), 877 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/dev-support/test_all_against_hdp_2_2_4_2_2.sh
----------------------------------------------------------------------
diff --git a/dev-support/test_all_against_hdp_2_2_4_2_2.sh b/dev-support/test_all_against_hdp_2_2_4_2_2.sh
new file mode 100755
index 0000000..f7780dd
--- /dev/null
+++ b/dev-support/test_all_against_hdp_2_2_4_2_2.sh
@@ -0,0 +1,25 @@
+#!/bin/bash
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+dir=$(dirname ${0})
+cd ${dir}
+cd ..
+
+mvn clean install -DskipTests 2>&1 | tee mci.log
+mvn verify -Dhdp.version=${HDP_VERSION:-"2.2.4.2-2"} -fae 2>&1 | tee mvnverify.log

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/dev-support/test_all_against_hdp_2_4_0_0_169.sh
----------------------------------------------------------------------
diff --git a/dev-support/test_all_against_hdp_2_4_0_0_169.sh b/dev-support/test_all_against_hdp_2_4_0_0_169.sh
deleted file mode 100755
index 2a3d24b..0000000
--- a/dev-support/test_all_against_hdp_2_4_0_0_169.sh
+++ /dev/null
@@ -1,25 +0,0 @@
-#!/bin/bash
-
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-dir=$(dirname ${0})
-cd ${dir}
-cd ..
-
-mvn clean install -DskipTests 2>&1 | tee mci.log
-mvn verify -Dhdp.version=${HDP_VERSION:-"2.4.0.0-169"} -fae 2>&1 | tee mvnverify.log

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/examples/test_case_data/sandbox/capacity-scheduler.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/capacity-scheduler.xml b/examples/test_case_data/sandbox/capacity-scheduler.xml
index e042aa5..7cb985c 100644
--- a/examples/test_case_data/sandbox/capacity-scheduler.xml
+++ b/examples/test_case_data/sandbox/capacity-scheduler.xml
@@ -47,6 +47,16 @@
     </property>
 
     <property>
+        <name>yarn.scheduler.capacity.root.accessible-node-labels.default.capacity</name>
+        <value>-1</value>
+    </property>
+
+    <property>
+        <name>yarn.scheduler.capacity.root.accessible-node-labels.default.maximum-capacity</name>
+        <value>-1</value>
+    </property>
+
+    <property>
         <name>yarn.scheduler.capacity.root.acl_administer_queue</name>
         <value>*</value>
     </property>
@@ -57,6 +67,11 @@
     </property>
 
     <property>
+        <name>yarn.scheduler.capacity.root.default-node-label-expression</name>
+        <value></value>
+    </property>
+
+    <property>
         <name>yarn.scheduler.capacity.root.default.acl_administer_jobs</name>
         <value>*</value>
     </property>
@@ -96,4 +111,4 @@
         <value>default</value>
     </property>
 
-</configuration>
\ No newline at end of file
+</configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/examples/test_case_data/sandbox/core-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/core-site.xml b/examples/test_case_data/sandbox/core-site.xml
index a4ad5c6..0c5f62b 100644
--- a/examples/test_case_data/sandbox/core-site.xml
+++ b/examples/test_case_data/sandbox/core-site.xml
@@ -19,6 +19,7 @@
     <property>
         <name>fs.defaultFS</name>
         <value>hdfs://sandbox.hortonworks.com:8020</value>
+        <final>true</final>
     </property>
 
     <property>
@@ -38,7 +39,7 @@
 
     <property>
         <name>hadoop.proxyuser.falcon.groups</name>
-        <value>*</value>
+        <value>users</value>
     </property>
 
     <property>
@@ -48,7 +49,7 @@
 
     <property>
         <name>hadoop.proxyuser.hbase.groups</name>
-        <value>*</value>
+        <value>users</value>
     </property>
 
     <property>
@@ -67,23 +68,13 @@
     </property>
 
     <property>
-        <name>hadoop.proxyuser.hdfs.groups</name>
-        <value>*</value>
-    </property>
-
-    <property>
-        <name>hadoop.proxyuser.hdfs.hosts</name>
-        <value>*</value>
-    </property>
-
-    <property>
         <name>hadoop.proxyuser.hive.groups</name>
-        <value>*</value>
+        <value>users</value>
     </property>
 
     <property>
         <name>hadoop.proxyuser.hive.hosts</name>
-        <value>sandbox.hortonworks.com</value>
+        <value>*</value>
     </property>
 
     <property>
@@ -132,15 +123,8 @@
     </property>
 
     <property>
-        <name>hadoop.security.key.provider.path</name>
-        <value></value>
-    </property>
-
-    <property>
         <name>io.compression.codecs</name>
-        <value>
-            org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.SnappyCodec
-        </value>
+        <value>org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.SnappyCodec</value>
     </property>
 
     <property>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/examples/test_case_data/sandbox/hbase-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hbase-site.xml b/examples/test_case_data/sandbox/hbase-site.xml
index 568de2e..46d5345 100644
--- a/examples/test_case_data/sandbox/hbase-site.xml
+++ b/examples/test_case_data/sandbox/hbase-site.xml
@@ -22,33 +22,8 @@
     </property>
 
     <property>
-        <name>hbase.bucketcache.ioengine</name>
-        <value></value>
-    </property>
-
-    <property>
-        <name>hbase.bucketcache.percentage.in.combinedcache</name>
-        <value></value>
-    </property>
-
-    <property>
-        <name>hbase.bucketcache.size</name>
-        <value></value>
-    </property>
-
-    <property>
-        <name>hbase.bulkload.staging.dir</name>
-        <value>/apps/hbase/staging</value>
-    </property>
-
-    <property>
         <name>hbase.client.keyvalue.maxsize</name>
-        <value>1048576</value>
-    </property>
-
-    <property>
-        <name>hbase.client.retries.number</name>
-        <value>35</value>
+        <value>10485760</value>
     </property>
 
     <property>
@@ -63,19 +38,12 @@
 
     <property>
         <name>hbase.coprocessor.master.classes</name>
-        <value>org.apache.ranger.authorization.hbase.RangerAuthorizationCoprocessor</value>
+        <value>com.xasecure.authorization.hbase.XaSecureAuthorizationCoprocessor</value>
     </property>
 
     <property>
         <name>hbase.coprocessor.region.classes</name>
-        <value>
-            org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint,org.apache.ranger.authorization.hbase.RangerAuthorizationCoprocessor
-        </value>
-    </property>
-
-    <property>
-        <name>hbase.coprocessor.regionserver.classes</name>
-        <value></value>
+        <value>com.xasecure.authorization.hbase.XaSecureAuthorizationCoprocessor</value>
     </property>
 
     <property>
@@ -119,11 +87,6 @@
     </property>
 
     <property>
-        <name>hbase.hstore.compaction.max</name>
-        <value>10</value>
-    </property>
-
-    <property>
         <name>hbase.hstore.compactionThreshold</name>
         <value>3</value>
     </property>
@@ -140,42 +103,32 @@
 
     <property>
         <name>hbase.master.info.port</name>
-        <value>16010</value>
+        <value>60010</value>
     </property>
 
     <property>
         <name>hbase.master.port</name>
-        <value>16000</value>
+        <value>60000</value>
     </property>
 
     <property>
-        <name>hbase.region.server.rpc.scheduler.factory.class</name>
-        <value></value>
+        <name>hbase.regionserver.global.memstore.lowerLimit</name>
+        <value>0.38</value>
     </property>
 
     <property>
-        <name>hbase.regionserver.global.memstore.size</name>
+        <name>hbase.regionserver.global.memstore.upperLimit</name>
         <value>0.4</value>
     </property>
 
     <property>
         <name>hbase.regionserver.handler.count</name>
-        <value>30</value>
+        <value>60</value>
     </property>
 
     <property>
         <name>hbase.regionserver.info.port</name>
-        <value>16030</value>
-    </property>
-
-    <property>
-        <name>hbase.regionserver.port</name>
-        <value>16020</value>
-    </property>
-
-    <property>
-        <name>hbase.regionserver.wal.codec</name>
-        <value>org.apache.hadoop.hbase.regionserver.wal.WALCellCodec</value>
+        <value>60030</value>
     </property>
 
     <property>
@@ -184,26 +137,11 @@
     </property>
 
     <property>
-        <name>hbase.rpc.controllerfactory.class</name>
-        <value></value>
-    </property>
-
-    <property>
-        <name>hbase.rpc.engine</name>
-        <value>org.apache.hadoop.hbase.ipc.SecureRpcEngine</value>
-    </property>
-
-    <property>
         <name>hbase.rpc.protection</name>
         <value>PRIVACY</value>
     </property>
 
     <property>
-        <name>hbase.rpc.timeout</name>
-        <value>90000</value>
-    </property>
-
-    <property>
         <name>hbase.security.authentication</name>
         <value>simple</value>
     </property>
@@ -220,7 +158,7 @@
 
     <property>
         <name>hbase.tmp.dir</name>
-        <value>/tmp/hbase-${user.name}</value>
+        <value>/hadoop/hbase</value>
     </property>
 
     <property>
@@ -240,27 +178,34 @@
 
     <property>
         <name>hfile.block.cache.size</name>
-        <value>0.4</value>
-    </property>
-
-    <property>
-        <name>phoenix.functions.allowUserDefinedFunctions</name>
-        <value></value>
-    </property>
-
-    <property>
-        <name>phoenix.query.timeoutMs</name>
-        <value>60000</value>
+        <value>0.40</value>
     </property>
 
     <property>
         <name>zookeeper.session.timeout</name>
-        <value>60000</value>
+        <value>30000</value>
     </property>
 
     <property>
         <name>zookeeper.znode.parent</name>
         <value>/hbase-unsecure</value>
     </property>
-
-</configuration>
\ No newline at end of file
+    <property>
+        <name>hbase.client.pause</name>
+        <value>100</value>
+        <description>General client pause value.  Used mostly as value to wait
+            before running a retry of a failed get, region lookup, etc.
+            See hbase.client.retries.number for description of how we backoff from
+            this initial pause amount and how this pause works w/ retries.</description>
+    </property>
+    <property>
+        <name>hbase.client.retries.number</name>
+        <value>5</value>
+        <description>Maximum retries.  Used as maximum for all retryable
+            operations such as the getting of a cell's value, starting a row update,
+            etc.  Retry interval is a rough function based on hbase.client.pause.  At
+            first we retry at this interval but then with backoff, we pretty quickly reach
+            retrying every ten seconds.  See HConstants#RETRY_BACKOFF for how the backup
+            ramps up.  Change this setting and hbase.client.pause to suit your workload.</description>
+    </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/examples/test_case_data/sandbox/hdfs-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hdfs-site.xml b/examples/test_case_data/sandbox/hdfs-site.xml
index 1d9040a..1175fff 100644
--- a/examples/test_case_data/sandbox/hdfs-site.xml
+++ b/examples/test_case_data/sandbox/hdfs-site.xml
@@ -18,7 +18,12 @@
 
     <property>
         <name>dfs.block.access.token.enable</name>
-        <value>true</value>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>dfs.block.size</name>
+        <value>34217472</value>
     </property>
 
     <property>
@@ -42,21 +47,11 @@
     </property>
 
     <property>
-        <name>dfs.client.retry.policy.enabled</name>
-        <value>false</value>
-    </property>
-
-    <property>
         <name>dfs.cluster.administrators</name>
         <value>hdfs</value>
     </property>
 
     <property>
-        <name>dfs.content-summary.limit</name>
-        <value>5000</value>
-    </property>
-
-    <property>
         <name>dfs.datanode.address</name>
         <value>0.0.0.0:50010</value>
     </property>
@@ -69,6 +64,7 @@
     <property>
         <name>dfs.datanode.data.dir</name>
         <value>/hadoop/hdfs/data</value>
+        <final>true</final>
     </property>
 
     <property>
@@ -84,6 +80,7 @@
     <property>
         <name>dfs.datanode.failed.volumes.tolerated</name>
         <value>0</value>
+        <final>true</final>
     </property>
 
     <property>
@@ -107,18 +104,13 @@
     </property>
 
     <property>
-        <name>dfs.domain.socket.path</name>
-        <value>/var/lib/hadoop-hdfs/dn_socket</value>
-    </property>
-
-    <property>
-        <name>dfs.encrypt.data.transfer.cipher.suites</name>
-        <value>AES/CTR/NoPadding</value>
+        <name>dfs.datanode.max.xcievers</name>
+        <value>1024</value>
     </property>
 
     <property>
-        <name>dfs.encryption.key.provider.uri</name>
-        <value></value>
+        <name>dfs.domain.socket.path</name>
+        <value>/var/lib/hadoop-hdfs/dn_socket</value>
     </property>
 
     <property>
@@ -158,12 +150,7 @@
 
     <property>
         <name>dfs.namenode.accesstime.precision</name>
-        <value>0</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.audit.log.async</name>
-        <value>true</value>
+        <value>3600000</value>
     </property>
 
     <property>
@@ -197,11 +184,6 @@
     </property>
 
     <property>
-        <name>dfs.namenode.fslock.fair</name>
-        <value>false</value>
-    </property>
-
-    <property>
         <name>dfs.namenode.handler.count</name>
         <value>100</value>
     </property>
@@ -209,6 +191,7 @@
     <property>
         <name>dfs.namenode.http-address</name>
         <value>sandbox.hortonworks.com:50070</value>
+        <final>true</final>
     </property>
 
     <property>
@@ -217,13 +200,9 @@
     </property>
 
     <property>
-        <name>dfs.namenode.inode.attributes.provider.class</name>
-        <value>org.apache.ranger.authorization.hadoop.RangerHdfsAuthorizer</value>
-    </property>
-
-    <property>
         <name>dfs.namenode.name.dir</name>
         <value>/hadoop/hdfs/namenode</value>
+        <final>true</final>
     </property>
 
     <property>
@@ -232,13 +211,8 @@
     </property>
 
     <property>
-        <name>dfs.namenode.rpc-address</name>
-        <value>sandbox.hortonworks.com:8020</value>
-    </property>
-
-    <property>
         <name>dfs.namenode.safemode.threshold-pct</name>
-        <value>0.999</value>
+        <value>1.0f</value>
     </property>
 
     <property>
@@ -262,6 +236,16 @@
     </property>
 
     <property>
+        <name>dfs.nfs.exports.allowed.hosts</name>
+        <value>* rw</value>
+    </property>
+
+    <property>
+        <name>dfs.nfs3.dump.dir</name>
+        <value>/tmp/.hdfs-nfs</value>
+    </property>
+
+    <property>
         <name>dfs.permissions.enabled</name>
         <value>true</value>
     </property>
@@ -273,7 +257,7 @@
 
     <property>
         <name>dfs.replication</name>
-        <value>3</value>
+        <value>1</value>
     </property>
 
     <property>
@@ -284,11 +268,13 @@
     <property>
         <name>dfs.support.append</name>
         <value>true</value>
+        <final>true</final>
     </property>
 
     <property>
         <name>dfs.webhdfs.enabled</name>
         <value>true</value>
+        <final>true</final>
     </property>
 
     <property>
@@ -296,14 +282,4 @@
         <value>022</value>
     </property>
 
-    <property>
-        <name>nfs.exports.allowed.hosts</name>
-        <value>* rw</value>
-    </property>
-
-    <property>
-        <name>nfs.file.dump.dir</name>
-        <value>/tmp/.hdfs-nfs</value>
-    </property>
-
-</configuration>
\ No newline at end of file
+</configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/examples/test_case_data/sandbox/hive-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hive-site.xml b/examples/test_case_data/sandbox/hive-site.xml
index a8c210e..1e78107 100644
--- a/examples/test_case_data/sandbox/hive-site.xml
+++ b/examples/test_case_data/sandbox/hive-site.xml
@@ -22,46 +22,11 @@
     </property>
 
     <property>
-        <name>atlas.cluster.name</name>
-        <value>Sandbox</value>
-    </property>
-
-    <property>
-        <name>atlas.hook.hive.maxThreads</name>
-        <value>1</value>
-    </property>
-
-    <property>
-        <name>atlas.hook.hive.minThreads</name>
-        <value>1</value>
-    </property>
-
-    <property>
-        <name>atlas.hook.hive.synchronous</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>atlas.rest.address</name>
-        <value>http://sandbox.hortonworks.com:21000</value>
-    </property>
-
-    <property>
-        <name>datanucleus.autoCreateSchema</name>
-        <value>false</value>
-    </property>
-
-    <property>
         <name>datanucleus.cache.level2.type</name>
         <value>none</value>
     </property>
 
     <property>
-        <name>datanucleus.fixedDatastore</name>
-        <value>true</value>
-    </property>
-
-    <property>
         <name>hive.auto.convert.join</name>
         <value>true</value>
     </property>
@@ -73,7 +38,7 @@
 
     <property>
         <name>hive.auto.convert.join.noconditionaltask.size</name>
-        <value>357913941</value>
+        <value>1000000000</value>
     </property>
 
     <property>
@@ -162,16 +127,6 @@
     </property>
 
     <property>
-        <name>hive.default.fileformat</name>
-        <value>TextFile</value>
-    </property>
-
-    <property>
-        <name>hive.default.fileformat.managed</name>
-        <value>TextFile</value>
-    </property>
-
-    <property>
         <name>hive.enforce.bucketing</name>
         <value>true</value>
     </property>
@@ -207,6 +162,11 @@
     </property>
 
     <property>
+        <name>hive.exec.failure.hooks</name>
+        <value>org.apache.hadoop.hive.ql.hooks.ATSHook</value>
+    </property>
+
+    <property>
         <name>hive.exec.max.created.files</name>
         <value>100000</value>
     </property>
@@ -237,11 +197,6 @@
     </property>
 
     <property>
-        <name>hive.exec.orc.encoding.strategy</name>
-        <value>SPEED</value>
-    </property>
-
-    <property>
         <name>hive.exec.parallel</name>
         <value>false</value>
     </property>
@@ -252,6 +207,16 @@
     </property>
 
     <property>
+        <name>hive.exec.post.hooks</name>
+        <value>org.apache.hadoop.hive.ql.hooks.ATSHook</value>
+    </property>
+
+    <property>
+        <name>hive.exec.pre.hooks</name>
+        <value>org.apache.hadoop.hive.ql.hooks.ATSHook</value>
+    </property>
+
+    <property>
         <name>hive.exec.reducers.bytes.per.reducer</name>
         <value>67108864</value>
     </property>
@@ -297,6 +262,11 @@
     </property>
 
     <property>
+        <name>hive.heapsize</name>
+        <value>250</value>
+    </property>
+
+    <property>
         <name>hive.limit.optimize.enable</name>
         <value>true</value>
     </property>
@@ -508,7 +478,7 @@
 
     <property>
         <name>hive.prewarm.numcontainers</name>
-        <value>3</value>
+        <value>10</value>
     </property>
 
     <property>
@@ -518,7 +488,7 @@
 
     <property>
         <name>hive.security.authorization.enabled</name>
-        <value>true</value>
+        <value>false</value>
     </property>
 
     <property>
@@ -538,7 +508,7 @@
 
     <property>
         <name>hive.security.metastore.authorization.manager</name>
-        <value>org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider</value>
+        <value>org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider,org.apache.hadoop.hive.ql.security.authorization.MetaStoreAuthzAPIAuthorizerEmbedOnly</value>
     </property>
 
     <property>
@@ -563,7 +533,12 @@
 
     <property>
         <name>hive.server2.enable.doAs</name>
-        <value>false</value>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>hive.server2.enable.impersonation</name>
+        <value>true</value>
     </property>
 
     <property>
@@ -573,7 +548,7 @@
 
     <property>
         <name>hive.server2.logging.operation.log.location</name>
-        <value>/tmp/hive/operation_logs</value>
+        <value>${system:java.io.tmpdir}/${system:user.name}/operation_logs</value>
     </property>
 
     <property>
@@ -678,7 +653,7 @@
 
     <property>
         <name>hive.tez.container.size</name>
-        <value>1024</value>
+        <value>250</value>
     </property>
 
     <property>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/examples/test_case_data/sandbox/mapred-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/mapred-site.xml b/examples/test_case_data/sandbox/mapred-site.xml
index be470f9..e90f594 100644
--- a/examples/test_case_data/sandbox/mapred-site.xml
+++ b/examples/test_case_data/sandbox/mapred-site.xml
@@ -18,7 +18,7 @@
 
     <property>
         <name>io.sort.mb</name>
-        <value>64</value>
+        <value>128</value>
     </property>
 
     <property>
@@ -27,13 +27,13 @@
     </property>
 
     <property>
-        <name>mapred.job.map.memory.mb</name>
-        <value>250</value>
+        <name>mapreduce.map.memory.mb</name>
+        <value>512</value>
     </property>
 
     <property>
-        <name>mapred.job.reduce.memory.mb</name>
-        <value>250</value>
+        <name>mapreduce.reduce.memory.mb</name>
+        <value>512</value>
     </property>
 
     <property>
@@ -48,9 +48,7 @@
 
     <property>
         <name>mapreduce.admin.user.env</name>
-        <value>
-            LD_LIBRARY_PATH=/usr/hdp/${hdp.version}/hadoop/lib/native:/usr/hdp/${hdp.version}/hadoop/lib/native/Linux-amd64-64
-        </value>
+        <value>LD_LIBRARY_PATH=/usr/hdp/${hdp.version}/hadoop/lib/native:/usr/hdp/${hdp.version}/hadoop/lib/native/Linux-amd64-64</value>
     </property>
 
     <property>
@@ -60,9 +58,7 @@
 
     <property>
         <name>mapreduce.application.classpath</name>
-        <value>
-            $PWD/mr-framework/hadoop/share/hadoop/mapreduce/*:$PWD/mr-framework/hadoop/share/hadoop/mapreduce/lib/*:$PWD/mr-framework/hadoop/share/hadoop/common/*:$PWD/mr-framework/hadoop/share/hadoop/common/lib/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/lib/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/lib/*:$PWD/mr-framework/hadoop/share/hadoop/tools/lib/*:/usr/hdp/${hdp.version}/hadoop/lib/hadoop-lzo-0.6.0.${hdp.version}.jar:/etc/hadoop/conf/secure
-        </value>
+        <value>/tmp/kylin/*,$HADOOP_CONF_DIR,/usr/hdp/${hdp.version}/hbase/lib/hbase-common.jar,/usr/hdp/current/hive-client/conf/,$PWD/mr-framework/hadoop/share/hadoop/mapreduce/*:$PWD/mr-framework/hadoop/share/hadoop/mapreduce/lib/*:$PWD/mr-framework/hadoop/share/hadoop/common/*:$PWD/mr-framework/hadoop/share/hadoop/common/lib/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/lib/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/lib/*:/usr/hdp/${hdp.version}/hadoop/lib/hadoop-lzo-0.6.0.${hdp.version}.jar:/usr/hdp/${hdp.version}/hadoop/lib/snappy-java-1.0.4.1.jar:/etc/hadoop/conf/secure</value>
     </property>
 
     <property>
@@ -81,18 +77,14 @@
     </property>
 
     <property>
-        <name>mapreduce.job.counters.max</name>
-        <value>130</value>
-    </property>
-
-    <property>
         <name>mapreduce.job.emit-timeline-data</name>
         <value>false</value>
     </property>
 
+    <!--the default value on hdp is 0.05, however for test environments we need to be conservative on resource -->
     <property>
         <name>mapreduce.job.reduce.slowstart.completedmaps</name>
-        <value>0.05</value>
+        <value>1</value>
     </property>
 
     <property>
@@ -116,28 +108,13 @@
     </property>
 
     <property>
-        <name>mapreduce.jobhistory.recovery.enable</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>mapreduce.jobhistory.recovery.store.class</name>
-        <value>org.apache.hadoop.mapreduce.v2.hs.HistoryServerLeveldbStateStoreService</value>
-    </property>
-
-    <property>
-        <name>mapreduce.jobhistory.recovery.store.leveldb.path</name>
-        <value>/hadoop/mapreduce/jhs</value>
-    </property>
-
-    <property>
         <name>mapreduce.jobhistory.webapp.address</name>
         <value>sandbox.hortonworks.com:19888</value>
     </property>
 
     <property>
         <name>mapreduce.map.java.opts</name>
-        <value>-Xmx1228m</value>
+        <value>-Xmx512m</value>
     </property>
 
     <property>
@@ -147,7 +124,7 @@
 
     <property>
         <name>mapreduce.map.memory.mb</name>
-        <value>1536</value>
+        <value>512</value>
     </property>
 
     <property>
@@ -182,7 +159,7 @@
 
     <property>
         <name>mapreduce.reduce.java.opts</name>
-        <value>-Xmx1638m</value>
+        <value>-Xmx200m</value>
     </property>
 
     <property>
@@ -192,7 +169,7 @@
 
     <property>
         <name>mapreduce.reduce.memory.mb</name>
-        <value>2048</value>
+        <value>512</value>
     </property>
 
     <property>
@@ -242,7 +219,7 @@
 
     <property>
         <name>mapreduce.task.io.sort.mb</name>
-        <value>859</value>
+        <value>128</value>
     </property>
 
     <property>
@@ -257,7 +234,7 @@
 
     <property>
         <name>yarn.app.mapreduce.am.command-opts</name>
-        <value>-Xmx819m -Dhdp.version=${hdp.version}</value>
+        <value>-Xmx512m</value>
     </property>
 
     <property>
@@ -267,7 +244,7 @@
 
     <property>
         <name>yarn.app.mapreduce.am.resource.mb</name>
-        <value>1024</value>
+        <value>512</value>
     </property>
 
     <property>
@@ -275,4 +252,4 @@
         <value>/user</value>
     </property>
 
-</configuration>
\ No newline at end of file
+</configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/examples/test_case_data/sandbox/yarn-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/yarn-site.xml b/examples/test_case_data/sandbox/yarn-site.xml
index ebdf44a..8256158 100644
--- a/examples/test_case_data/sandbox/yarn-site.xml
+++ b/examples/test_case_data/sandbox/yarn-site.xml
@@ -18,7 +18,7 @@
 
     <property>
         <name>hadoop.registry.rm.enabled</name>
-        <value>true</value>
+        <value>false</value>
     </property>
 
     <property>
@@ -28,29 +28,22 @@
 
     <property>
         <name>yarn.acl.enable</name>
-        <value>true</value>
+        <value>false</value>
     </property>
 
     <property>
         <name>yarn.admin.acl</name>
-        <value>*</value>
+        <value></value>
     </property>
 
     <property>
         <name>yarn.application.classpath</name>
-        <value>
-            $HADOOP_CONF_DIR,/usr/hdp/current/hadoop-client/*,/usr/hdp/current/hadoop-client/lib/*,/usr/hdp/current/hadoop-hdfs-client/*,/usr/hdp/current/hadoop-hdfs-client/lib/*,/usr/hdp/current/hadoop-yarn-client/*,/usr/hdp/current/hadoop-yarn-client/lib/*
-        </value>
-    </property>
-
-    <property>
-        <name>yarn.authorization-provider</name>
-        <value>org.apache.ranger.authorization.yarn.authorizer.RangerYarnAuthorizer</value>
+        <value>$HADOOP_CONF_DIR,/usr/hdp/current/hadoop-client/*,/usr/hdp/current/hadoop-client/lib/*,/usr/hdp/current/hadoop-hdfs-client/*,/usr/hdp/current/hadoop-hdfs-client/lib/*,/usr/hdp/current/hadoop-yarn-client/*,/usr/hdp/current/hadoop-yarn-client/lib/*</value>
     </property>
 
     <property>
         <name>yarn.client.nodemanager-connect.max-wait-ms</name>
-        <value>120000</value>
+        <value>60000</value>
     </property>
 
     <property>
@@ -79,11 +72,6 @@
     </property>
 
     <property>
-        <name>yarn.node-labels.enabled</name>
-        <value>false</value>
-    </property>
-
-    <property>
         <name>yarn.node-labels.fs-store.retry-policy-spec</name>
         <value>2000, 500</value>
     </property>
@@ -94,6 +82,11 @@
     </property>
 
     <property>
+        <name>yarn.node-labels.manager-class</name>
+        <value>org.apache.hadoop.yarn.server.resourcemanager.nodelabels.MemoryRMNodeLabelsManager</value>
+    </property>
+
+    <property>
         <name>yarn.nodemanager.address</name>
         <value>0.0.0.0:45454</value>
     </property>
@@ -105,7 +98,7 @@
 
     <property>
         <name>yarn.nodemanager.aux-services</name>
-        <value>mapreduce_shuffle,spark_shuffle</value>
+        <value>mapreduce_shuffle</value>
     </property>
 
     <property>
@@ -114,11 +107,6 @@
     </property>
 
     <property>
-        <name>yarn.nodemanager.aux-services.spark_shuffle.class</name>
-        <value>org.apache.spark.network.yarn.YarnShuffleService</value>
-    </property>
-
-    <property>
         <name>yarn.nodemanager.bind-host</name>
         <value>0.0.0.0</value>
     </property>
@@ -160,7 +148,7 @@
 
     <property>
         <name>yarn.nodemanager.health-checker.script.timeout-ms</name>
-        <value>120000</value>
+        <value>60000</value>
     </property>
 
     <property>
@@ -255,12 +243,12 @@
 
     <property>
         <name>yarn.nodemanager.resource.memory-mb</name>
-        <value>7168</value>
+        <value>9216</value>
     </property>
 
     <property>
         <name>yarn.nodemanager.resource.percentage-physical-cpu-limit</name>
-        <value>80</value>
+        <value>100</value>
     </property>
 
     <property>
@@ -349,11 +337,6 @@
     </property>
 
     <property>
-        <name>yarn.resourcemanager.scheduler.monitor.enable</name>
-        <value>false</value>
-    </property>
-
-    <property>
         <name>yarn.resourcemanager.state-store.max-completed-applications</name>
         <value>${yarn.resourcemanager.max-completed-applications}</value>
     </property>
@@ -385,7 +368,7 @@
 
     <property>
         <name>yarn.resourcemanager.webapp.https.address</name>
-        <value>sandbox.hortonworks.com:8090</value>
+        <value>localhost:8090</value>
     </property>
 
     <property>
@@ -425,7 +408,7 @@
 
     <property>
         <name>yarn.resourcemanager.zk-address</name>
-        <value>sandbox.hortonworks.com:2181</value>
+        <value>localhost:2181</value>
     </property>
 
     <property>
@@ -450,22 +433,12 @@
 
     <property>
         <name>yarn.scheduler.maximum-allocation-mb</name>
-        <value>7168</value>
-    </property>
-
-    <property>
-        <name>yarn.scheduler.maximum-allocation-vcores</name>
-        <value>3</value>
+        <value>9216</value>
     </property>
 
     <property>
         <name>yarn.scheduler.minimum-allocation-mb</name>
-        <value>1024</value>
-    </property>
-
-    <property>
-        <name>yarn.scheduler.minimum-allocation-vcores</name>
-        <value>1</value>
+        <value>1536</value>
     </property>
 
     <property>
@@ -494,41 +467,6 @@
     </property>
 
     <property>
-        <name>yarn.timeline-service.entity-group-fs-store.active-dir</name>
-        <value>/ats/active/</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.entity-group-fs-store.cleaner-interval-seconds</name>
-        <value>3600</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.entity-group-fs-store.done-dir</name>
-        <value>/ats/done/</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.entity-group-fs-store.group-id-plugin-classes</name>
-        <value>org.apache.tez.dag.history.logging.ats.TimelineCachePluginImpl</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.entity-group-fs-store.retain-seconds</name>
-        <value>604800</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.entity-group-fs-store.scan-interval-seconds</name>
-        <value>60</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.entity-group-fs-store.summary-store</name>
-        <value>org.apache.hadoop.yarn.server.timeline.RollingLevelDBTimelineStore</value>
-    </property>
-
-    <property>
         <name>yarn.timeline-service.generic-application-history.store-class</name>
         <value>org.apache.hadoop.yarn.server.applicationhistoryservice.NullApplicationHistoryStore</value>
     </property>
@@ -544,11 +482,6 @@
     </property>
 
     <property>
-        <name>yarn.timeline-service.leveldb-state-store.path</name>
-        <value>/hadoop/yarn/timeline</value>
-    </property>
-
-    <property>
         <name>yarn.timeline-service.leveldb-timeline-store.path</name>
         <value>/hadoop/yarn/timeline</value>
     </property>
@@ -574,23 +507,8 @@
     </property>
 
     <property>
-        <name>yarn.timeline-service.plugin.enabled</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.recovery.enabled</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.state-store-class</name>
-        <value>org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore</value>
-    </property>
-
-    <property>
         <name>yarn.timeline-service.store-class</name>
-        <value>org.apache.hadoop.yarn.server.timeline.EntityGroupFSTimelineStore</value>
+        <value>org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore</value>
     </property>
 
     <property>
@@ -604,11 +522,6 @@
     </property>
 
     <property>
-        <name>yarn.timeline-service.version</name>
-        <value>1.5</value>
-    </property>
-
-    <property>
         <name>yarn.timeline-service.webapp.address</name>
         <value>sandbox.hortonworks.com:8188</value>
     </property>
@@ -618,4 +531,4 @@
         <value>sandbox.hortonworks.com:8190</value>
     </property>
 
-</configuration>
\ No newline at end of file
+</configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
index 726d72f..d43bc1e 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
@@ -32,9 +32,11 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HTable;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.common.util.HBaseMetadataTestCase;
@@ -58,7 +60,6 @@ import org.apache.kylin.job.impl.threadpool.DefaultScheduler;
 import org.apache.kylin.source.ISource;
 import org.apache.kylin.source.SourceFactory;
 import org.apache.kylin.source.SourcePartition;
-import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.storage.hbase.util.HBaseRegionSizeCalculator;
 import org.apache.kylin.storage.hbase.util.ZookeeperJobLock;
 import org.apache.kylin.tool.StorageCleanupJob;
@@ -95,10 +96,10 @@ public class BuildCubeWithEngine {
             logger.error("error", e);
             exitCode = 1;
         }
-
+        
         long millis = System.currentTimeMillis() - start;
         System.out.println("Time elapsed: " + (millis / 1000) + " sec - in " + BuildCubeWithEngine.class.getName());
-
+        
         System.exit(exitCode);
     }
 
@@ -358,10 +359,10 @@ public class BuildCubeWithEngine {
 
     @SuppressWarnings("unused")
     private void checkHFilesInHBase(CubeSegment segment) throws IOException {
-        try (Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl())) {
-            String tableName = segment.getStorageLocationIdentifier();
-
-            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
+        Configuration conf = HBaseConfiguration.create(HadoopUtil.getCurrentConfiguration());
+        String tableName = segment.getStorageLocationIdentifier();
+        try (HTable table = new HTable(conf, tableName)) {
+            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
             Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
             long totalSize = 0;
             for (Long size : sizeMap.values()) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b82eee2..f967575 100644
--- a/pom.xml
+++ b/pom.xml
@@ -46,20 +46,20 @@
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
 
         <!-- Hadoop versions -->
-        <hadoop2.version>2.7.1</hadoop2.version>
-        <yarn.version>2.7.1</yarn.version>
+        <hadoop2.version>2.6.0</hadoop2.version>
+        <yarn.version>2.6.0</yarn.version>
 
         <!-- Hive versions -->
-        <hive.version>1.2.1</hive.version>
-        <hive-hcatalog.version>1.2.1</hive-hcatalog.version>
+        <hive.version>0.14.0</hive.version>
+        <hive-hcatalog.version>0.14.0</hive-hcatalog.version>
 
         <!-- HBase versions -->
-        <hbase-hadoop2.version>1.1.1</hbase-hadoop2.version>
+        <hbase-hadoop2.version>0.98.8-hadoop2</hbase-hadoop2.version>
         <kafka.version>0.10.1.0</kafka.version>
 
         <!-- Hadoop deps, keep compatible with hadoop2.version -->
         <zookeeper.version>3.4.6</zookeeper.version>
-        <curator.version>2.7.1</curator.version>
+        <curator.version>2.6.0</curator.version>
         <jackson.version>2.2.4</jackson.version>
         <jsr305.version>3.0.1</jsr305.version>
         <guava.version>14.0</guava.version>
@@ -355,11 +355,6 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-yarn-common</artifactId>
-                <version>${hadoop2.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.hadoop</groupId>
                 <artifactId>hadoop-mapreduce-client-core</artifactId>
                 <version>${hadoop2.version}</version>
             </dependency>
@@ -817,11 +812,6 @@
             <id>conjars</id>
             <url>http://conjars.org/repo/</url>
         </repository>
-
-        <repository>
-            <id>cloudera</id>
-            <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
-        </repository>
     </repositories>
 
     <build>
@@ -1174,106 +1164,6 @@
             </build>
         </profile>
         <profile>
-            <id>cdh5.7</id>
-            <properties>
-                <hadoop2.version>2.6.0-cdh5.7.0</hadoop2.version>
-                <yarn.version>2.6.0-cdh5.7.0</yarn.version>
-                <hive.version>1.1.0-cdh5.7.0</hive.version>
-                <hive-hcatalog.version>1.1.0-cdh5.7.0</hive-hcatalog.version>
-                <hbase-hadoop2.version>1.2.0-cdh5.7.0</hbase-hadoop2.version>
-                <zookeeper.version>3.4.5-cdh5.7.0</zookeeper.version>
-            </properties>
-            <build>
-                <plugins>
-                    <plugin>
-                        <groupId>org.apache.maven.plugins</groupId>
-                        <artifactId>maven-compiler-plugin</artifactId>
-                        <configuration>
-                            <fork>true</fork>
-                            <meminitial>1024m</meminitial>
-                            <maxmem>2048m</maxmem>
-                        </configuration>
-                    </plugin>
-
-                    <plugin>
-                        <groupId>org.apache.maven.plugins</groupId>
-                        <artifactId>maven-dependency-plugin</artifactId>
-                        <executions>
-                            <execution>
-                                <id>copy-jamm</id>
-                                <goals>
-                                    <goal>copy</goal>
-                                </goals>
-                                <phase>generate-test-resources</phase>
-                                <configuration>
-                                    <artifactItems>
-                                        <artifactItem>
-                                            <groupId>com.github.jbellis</groupId>
-                                            <artifactId>jamm</artifactId>
-                                            <outputDirectory>${project.build.testOutputDirectory}</outputDirectory>
-                                            <destFileName>jamm.jar</destFileName>
-                                        </artifactItem>
-                                    </artifactItems>
-                                </configuration>
-                            </execution>
-                        </executions>
-                    </plugin>
-
-                    <plugin>
-                        <groupId>org.jacoco</groupId>
-                        <artifactId>jacoco-maven-plugin</artifactId>
-                        <configuration>
-                            <append>true</append>
-                            <destFile>
-                                ${sonar.jacoco.reportPath}
-                            </destFile>
-                        </configuration>
-                        <executions>
-                            <execution>
-                                <id>pre-test</id>
-                                <goals>
-                                    <goal>prepare-agent</goal>
-                                </goals>
-                                <configuration>
-                                    <propertyName>surefireArgLine</propertyName>
-                                </configuration>
-                            </execution>
-                            <execution>
-                                <id>post-test</id>
-                                <phase>test</phase>
-                                <goals>
-                                    <goal>report</goal>
-                                </goals>
-                            </execution>
-                        </executions>
-                    </plugin>
-
-                    <plugin>
-                        <groupId>org.apache.maven.plugins</groupId>
-                        <artifactId>maven-surefire-plugin</artifactId>
-                        <version>2.19.1</version>
-                        <configuration>
-                            <reportsDirectory>${project.basedir}/../target/surefire-reports</reportsDirectory>
-                            <excludes>
-                                <exclude>**/IT*.java</exclude>
-                            </excludes>
-                            <systemProperties>
-                                <property>
-                                    <name>buildCubeUsingProvidedData</name>
-                                    <value>false</value>
-                                </property>
-                                <property>
-                                    <name>log4j.configuration</name>
-                                    <value>file:${project.basedir}/../build/conf/kylin-tools-log4j.properties</value>
-                                </property>
-                            </systemProperties>
-                            <argLine>-javaagent:${project.build.testOutputDirectory}/jamm.jar ${argLine} ${surefireArgLine}</argLine>
-                        </configuration>
-                    </plugin>
-                </plugins>
-            </build>
-        </profile>
-        <profile>
             <!-- This profile adds/overrides few features of the 'apache-release'
                  profile in the parent pom. -->
             <id>apache-release</id>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
index 8095bf8..ea68855 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
@@ -20,7 +20,7 @@ package org.apache.kylin.rest.security;
 
 import java.io.IOException;
 
-import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.HTableInterface;
 
 /**
  */
@@ -36,6 +36,6 @@ public interface AclHBaseStorage {
 
     String prepareHBaseTable(Class<?> clazz) throws IOException;
 
-    Table getTable(String tableName) throws IOException;
+    HTableInterface getTable(String tableName) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
index cc76b87..d9326f5 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
@@ -21,7 +21,7 @@ package org.apache.kylin.rest.security;
 import java.io.IOException;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.rest.service.AclService;
 import org.apache.kylin.rest.service.QueryService;
@@ -34,8 +34,8 @@ public class MockAclHBaseStorage implements AclHBaseStorage {
     private static final String aclTableName = "MOCK-ACL-TABLE";
     private static final String userTableName = "MOCK-USER-TABLE";
 
-    private Table mockedAclTable;
-    private Table mockedUserTable;
+    private HTableInterface mockedAclTable;
+    private HTableInterface mockedUserTable;
     private RealAclHBaseStorage realAcl;
 
     public MockAclHBaseStorage() {
@@ -65,7 +65,7 @@ public class MockAclHBaseStorage implements AclHBaseStorage {
     }
 
     @Override
-    public Table getTable(String tableName) throws IOException {
+    public HTableInterface getTable(String tableName) throws IOException {
         if (realAcl != null) {
             return realAcl.getTable(tableName);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java b/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
index 972eea9..d0aa0ed 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
@@ -91,7 +91,7 @@ import com.google.protobuf.ServiceException;
  *     <li>remove some methods for loading data, checking values ...</li>
  * </ul>
  */
-public class MockHTable implements Table {
+public class MockHTable implements HTableInterface {
     private final String tableName;
     private final List<String> columnFamilies = new ArrayList<>();
 
@@ -114,6 +114,14 @@ public class MockHTable implements Table {
         this.columnFamilies.add(columnFamily);
     }
 
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public byte[] getTableName() {
+        return tableName.getBytes();
+    }
+
     @Override
     public TableName getName() {
         return null;
@@ -192,8 +200,8 @@ public class MockHTable implements Table {
     }
 
     @Override
-    public boolean[] existsAll(List<Get> list) throws IOException {
-        return new boolean[0];
+    public Boolean[] exists(List<Get> gets) throws IOException {
+        return new Boolean[0];
     }
 
     /**
@@ -298,6 +306,15 @@ public class MockHTable implements Table {
      * {@inheritDoc}
      */
     @Override
+    public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
+        // FIXME: implement
+        return null;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
     public ResultScanner getScanner(Scan scan) throws IOException {
         final List<Result> ret = new ArrayList<Result>();
         byte[] st = scan.getStartRow();
@@ -429,7 +446,7 @@ public class MockHTable implements Table {
              */
         }
         if (filter.hasFilterRow() && !filteredOnRowKey) {
-            filter.filterRow();
+            filter.filterRow(nkvs);
         }
         if (filter.filterRow() || filteredOnRowKey) {
             nkvs.clear();
@@ -518,11 +535,6 @@ public class MockHTable implements Table {
         return false;
     }
 
-    @Override
-    public boolean checkAndPut(byte[] bytes, byte[] bytes1, byte[] bytes2, CompareFilter.CompareOp compareOp, byte[] bytes3, Put put) throws IOException {
-        return false;
-    }
-
     /**
      * {@inheritDoc}
      */
@@ -543,7 +555,7 @@ public class MockHTable implements Table {
                 continue;
             }
             for (KeyValue kv : delete.getFamilyMap().get(family)) {
-                if (kv.isDelete()) {
+                if (kv.isDeleteFamily()) {
                     data.get(row).get(kv.getFamily()).clear();
                 } else {
                     data.get(row).get(kv.getFamily()).remove(kv.getQualifier());
@@ -580,11 +592,6 @@ public class MockHTable implements Table {
         return false;
     }
 
-    @Override
-    public boolean checkAndDelete(byte[] bytes, byte[] bytes1, byte[] bytes2, CompareFilter.CompareOp compareOp, byte[] bytes3, Delete delete) throws IOException {
-        return false;
-    }
-
     /**
      * {@inheritDoc}
      */
@@ -598,7 +605,7 @@ public class MockHTable implements Table {
      */
     @Override
     public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) throws IOException {
-        return incrementColumnValue(row, family, qualifier, amount, null);
+        return incrementColumnValue(row, family, qualifier, amount, true);
     }
 
     @Override
@@ -610,6 +617,37 @@ public class MockHTable implements Table {
      * {@inheritDoc}
      */
     @Override
+    public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
+        if (check(row, family, qualifier, null)) {
+            Put put = new Put(row);
+            put.add(family, qualifier, Bytes.toBytes(amount));
+            put(put);
+            return amount;
+        }
+        long newValue = Bytes.toLong(data.get(row).get(family).get(qualifier).lastEntry().getValue()) + amount;
+        data.get(row).get(family).get(qualifier).put(System.currentTimeMillis(), Bytes.toBytes(newValue));
+        return newValue;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean isAutoFlush() {
+        return true;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void flushCommits() throws IOException {
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
     public void close() throws IOException {
     }
 
@@ -635,6 +673,29 @@ public class MockHTable implements Table {
      * {@inheritDoc}
      */
     @Override
+    public void setAutoFlush(boolean autoFlush) {
+        throw new NotImplementedException();
+
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
+        throw new NotImplementedException();
+
+    }
+
+    @Override
+    public void setAutoFlushTo(boolean autoFlush) {
+        throw new NotImplementedException();
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
     public long getWriteBufferSize() {
         throw new NotImplementedException();
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
index d1a1384..1d520c4 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
@@ -21,8 +21,7 @@ package org.apache.kylin.rest.security;
 import java.io.IOException;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.rest.service.AclService;
 import org.apache.kylin.rest.service.QueryService;
@@ -59,11 +58,11 @@ public class RealAclHBaseStorage implements AclHBaseStorage {
     }
 
     @Override
-    public Table getTable(String tableName) throws IOException {
+    public HTableInterface getTable(String tableName) throws IOException {
         if (StringUtils.equals(tableName, aclTableName)) {
-            return HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
+            return HBaseConnection.get(hbaseUrl).getTable(aclTableName);
         } else if (StringUtils.equals(tableName, userTableName)) {
-            return HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
+            return HBaseConnection.get(hbaseUrl).getTable(userTableName);
         } else {
             throw new IllegalStateException("getTable failed" + tableName);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java b/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
index 3e3efec..d693a67 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
@@ -33,7 +33,7 @@ import javax.annotation.PostConstruct;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -124,7 +124,7 @@ public class AclService implements MutableAclService {
     @Override
     public List<ObjectIdentity> findChildren(ObjectIdentity parentIdentity) {
         List<ObjectIdentity> oids = new ArrayList<ObjectIdentity>();
-        Table htable = null;
+        HTableInterface htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
@@ -173,7 +173,7 @@ public class AclService implements MutableAclService {
     @Override
     public Map<ObjectIdentity, Acl> readAclsById(List<ObjectIdentity> oids, List<Sid> sids) throws NotFoundException {
         Map<ObjectIdentity, Acl> aclMaps = new HashMap<ObjectIdentity, Acl>();
-        Table htable = null;
+        HTableInterface htable = null;
         Result result = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
@@ -226,16 +226,17 @@ public class AclService implements MutableAclService {
         Authentication auth = SecurityContextHolder.getContext().getAuthentication();
         PrincipalSid sid = new PrincipalSid(auth);
 
-        Table htable = null;
+        HTableInterface htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
             Put put = new Put(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
-            put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
-            put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
-            put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
+            put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
+            put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
+            put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
 
             htable.put(put);
+            htable.flushCommits();
 
             logger.debug("ACL of " + objectIdentity + " created successfully.");
         } catch (IOException e) {
@@ -249,7 +250,7 @@ public class AclService implements MutableAclService {
 
     @Override
     public void deleteAcl(ObjectIdentity objectIdentity, boolean deleteChildren) throws ChildrenExistException {
-        Table htable = null;
+        HTableInterface htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
@@ -265,6 +266,7 @@ public class AclService implements MutableAclService {
             }
 
             htable.delete(delete);
+            htable.flushCommits();
 
             logger.debug("ACL of " + objectIdentity + " deleted successfully.");
         } catch (IOException e) {
@@ -282,7 +284,7 @@ public class AclService implements MutableAclService {
             throw e;
         }
 
-        Table htable = null;
+        HTableInterface htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
@@ -293,16 +295,17 @@ public class AclService implements MutableAclService {
             Put put = new Put(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
 
             if (null != acl.getParentAcl()) {
-                put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
+                put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
             }
 
             for (AccessControlEntry ace : acl.getEntries()) {
                 AceInfo aceInfo = new AceInfo(ace);
-                put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
+                put.add(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
             }
 
             if (!put.isEmpty()) {
                 htable.put(put);
+                htable.flushCommits();
 
                 logger.debug("ACL of " + acl.getObjectIdentity() + " updated successfully.");
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
index c8c87cb..d28c87c 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -27,7 +27,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.WeakHashMap;
 
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.HTable;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeInstance;
@@ -393,24 +395,33 @@ public class CubeService extends BasicService {
         if (htableInfoCache.containsKey(tableName)) {
             return htableInfoCache.get(tableName);
         }
-        Connection conn = HBaseConnection.get(this.getConfig().getStorageUrl());
+
+        Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
+        HTable table = null;
         HBaseResponse hr = null;
         long tableSize = 0;
         int regionCount = 0;
 
-        HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
-        Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
+        try {
+            table = new HTable(hconf, tableName);
 
-        for (long s : sizeMap.values()) {
-            tableSize += s;
-        }
+            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
+            Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
 
-        regionCount = sizeMap.size();
+            for (long s : sizeMap.values()) {
+                tableSize += s;
+            }
+
+            regionCount = sizeMap.size();
+
+            // Set response.
+            hr = new HBaseResponse();
+            hr.setTableSize(tableSize);
+            hr.setRegionCount(regionCount);
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
 
-        // Set response.
-        hr = new HBaseResponse();
-        hr.setTableSize(tableSize);
-        hr.setRegionCount(regionCount);
         htableInfoCache.put(tableName, hr);
 
         return hr;

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 4c02aa4..970cd56 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -49,11 +49,11 @@ import javax.sql.DataSource;
 import org.apache.calcite.avatica.ColumnMetaData.Rep;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.QueryContext;
 import org.apache.kylin.common.debug.BackdoorToggles;
@@ -164,13 +164,14 @@ public class QueryService extends BasicService {
         Query[] queryArray = new Query[queries.size()];
 
         byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
-        Table htable = null;
+        HTableInterface htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
+            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
             Put put = new Put(Bytes.toBytes(creator));
-            put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
 
             htable.put(put);
+            htable.flushCommits();
         } finally {
             IOUtils.closeQuietly(htable);
         }
@@ -196,13 +197,14 @@ public class QueryService extends BasicService {
 
         Query[] queryArray = new Query[queries.size()];
         byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
-        Table htable = null;
+        HTableInterface htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
+            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
             Put put = new Put(Bytes.toBytes(creator));
-            put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
 
             htable.put(put);
+            htable.flushCommits();
         } finally {
             IOUtils.closeQuietly(htable);
         }
@@ -214,12 +216,12 @@ public class QueryService extends BasicService {
         }
 
         List<Query> queries = new ArrayList<Query>();
-        Table htable = null;
+        HTableInterface htable = null;
         try {
-            org.apache.hadoop.hbase.client.Connection conn = HBaseConnection.get(hbaseUrl);
+            HConnection conn = HBaseConnection.get(hbaseUrl);
             HBaseConnection.createHTableIfNeeded(conn, userTableName, USER_QUERY_FAMILY);
 
-            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
+            htable = conn.getTable(userTableName);
             Get get = new Get(Bytes.toBytes(creator));
             get.addFamily(Bytes.toBytes(USER_QUERY_FAMILY));
             Result result = htable.get(get);

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java b/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
index ab54882..07c7c6f 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
@@ -30,11 +30,11 @@ import javax.annotation.PostConstruct;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.rest.security.AclHBaseStorage;
@@ -72,7 +72,7 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public UserDetails loadUserByUsername(String username) throws UsernameNotFoundException {
-        Table htable = null;
+        HTableInterface htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
@@ -144,16 +144,16 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public void updateUser(UserDetails user) {
-        Table htable = null;
+        HTableInterface htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
             Pair<byte[], byte[]> pair = userToHBaseRow(user);
             Put put = new Put(pair.getKey());
-
-            put.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
+            put.add(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
 
             htable.put(put);
+            htable.flushCommits();
         } catch (IOException e) {
             throw new RuntimeException(e.getMessage(), e);
         } finally {
@@ -163,13 +163,14 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public void deleteUser(String username) {
-        Table htable = null;
+        HTableInterface htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
             Delete delete = new Delete(Bytes.toBytes(username));
 
             htable.delete(delete);
+            htable.flushCommits();
         } catch (IOException e) {
             throw new RuntimeException(e.getMessage(), e);
         } finally {
@@ -184,7 +185,7 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public boolean userExists(String username) {
-        Table htable = null;
+        HTableInterface htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
@@ -215,7 +216,7 @@ public class UserService implements UserDetailsManager {
         s.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN));
 
         List<UserDetails> all = new ArrayList<UserDetails>();
-        Table htable = null;
+        HTableInterface htable = null;
         ResultScanner scanner = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);

http://git-wip-us.apache.org/repos/asf/kylin/blob/0dc56aa4/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 53c95cb..335bfe7 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
@@ -40,9 +40,9 @@ 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;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.kylin.common.KylinConfig;
@@ -64,7 +64,7 @@ public class HBaseConnection {
     private static final Logger logger = LoggerFactory.getLogger(HBaseConnection.class);
 
     private static final Map<String, Configuration> configCache = new ConcurrentHashMap<String, Configuration>();
-    private static final Map<String, Connection> connPool = new ConcurrentHashMap<String, Connection>();
+    private static final Map<String, HConnection> connPool = new ConcurrentHashMap<String, HConnection>();
     private static final ThreadLocal<Configuration> configThreadLocal = new ThreadLocal<>();
 
     private static ExecutorService coprocessorPool = null;
@@ -75,7 +75,7 @@ public class HBaseConnection {
             public void run() {
                 closeCoprocessorPool();
 
-                for (Connection conn : connPool.values()) {
+                for (HConnection conn : connPool.values()) {
                     try {
                         conn.close();
                     } catch (IOException e) {
@@ -144,7 +144,7 @@ public class HBaseConnection {
         // using a hbase:xxx URL is deprecated, instead hbase config is always loaded from hbase-site.xml in classpath
         if (!(StringUtils.isEmpty(url) || "hbase".equals(url)))
             throw new IllegalArgumentException("to use hbase storage, pls set 'kylin.storage.url=hbase' in kylin.properties");
-
+        
         Configuration conf = HBaseConfiguration.create(HadoopUtil.getCurrentConfiguration());
         addHBaseClusterNNHAConfiguration(conf);
 
@@ -213,9 +213,9 @@ public class HBaseConnection {
 
     // ============================================================================
 
-    // returned Connection can be shared by multiple threads and does not require close()
+    // returned HConnection can be shared by multiple threads and does not require close()
     @SuppressWarnings("resource")
-    public static Connection get(String url) {
+    public static HConnection get(String url) {
         // find configuration
         Configuration conf = configCache.get(url);
         if (conf == null) {
@@ -223,13 +223,13 @@ public class HBaseConnection {
             configCache.put(url, conf);
         }
 
-        Connection connection = connPool.get(url);
+        HConnection connection = connPool.get(url);
         try {
             while (true) {
                 // I don't use DCL since recreate a connection is not a big issue.
                 if (connection == null || connection.isClosed()) {
                     logger.info("connection is null or closed, creating a new one");
-                    connection = ConnectionFactory.createConnection(conf);
+                    connection = HConnectionManager.createConnection(conf);
                     connPool.put(url, connection);
                 }
 
@@ -248,8 +248,8 @@ public class HBaseConnection {
         return connection;
     }
 
-    public static boolean tableExists(Connection conn, String tableName) throws IOException {
-        Admin hbase = conn.getAdmin();
+    public static boolean tableExists(HConnection conn, String tableName) throws IOException {
+        HBaseAdmin hbase = new HBaseAdmin(conn);
         try {
             return hbase.tableExists(TableName.valueOf(tableName));
         } finally {
@@ -269,18 +269,18 @@ public class HBaseConnection {
         deleteTable(HBaseConnection.get(hbaseUrl), tableName);
     }
 
-    public static void createHTableIfNeeded(Connection conn, String table, String... families) throws IOException {
-        Admin hbase = conn.getAdmin();
-        TableName tableName = TableName.valueOf(table);
+    public static void createHTableIfNeeded(HConnection conn, String table, String... families) throws IOException {
+        HBaseAdmin hbase = new HBaseAdmin(conn);
+
         try {
             if (tableExists(conn, table)) {
                 logger.debug("HTable '" + table + "' already exists");
-                Set<String> existingFamilies = getFamilyNames(hbase.getTableDescriptor(tableName));
+                Set<String> existingFamilies = getFamilyNames(hbase.getTableDescriptor(TableName.valueOf(table)));
                 boolean wait = false;
                 for (String family : families) {
                     if (existingFamilies.contains(family) == false) {
                         logger.debug("Adding family '" + family + "' to HTable '" + table + "'");
-                        hbase.addColumn(tableName, newFamilyDescriptor(family));
+                        hbase.addColumn(table, newFamilyDescriptor(family));
                         // addColumn() is async, is there a way to wait it finish?
                         wait = true;
                     }
@@ -333,8 +333,8 @@ public class HBaseConnection {
         return fd;
     }
 
-    public static void deleteTable(Connection conn, String tableName) throws IOException {
-        Admin hbase = conn.getAdmin();
+    public static void deleteTable(HConnection conn, String tableName) throws IOException {
+        HBaseAdmin hbase = new HBaseAdmin(conn);
 
         try {
             if (!tableExists(conn, tableName)) {
@@ -344,10 +344,10 @@ public class HBaseConnection {
 
             logger.debug("delete HTable '" + tableName + "'");
 
-            if (hbase.isTableEnabled(TableName.valueOf(tableName))) {
-                hbase.disableTable(TableName.valueOf(tableName));
+            if (hbase.isTableEnabled(tableName)) {
+                hbase.disableTable(tableName);
             }
-            hbase.deleteTable(TableName.valueOf(tableName));
+            hbase.deleteTable(tableName);
 
             logger.debug("HTable '" + tableName + "' deleted");
         } finally {