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/01/17 03:32:06 UTC

[01/21] kylin git commit: KYLIN-2383 let HLLC handle NULL input correctly [Forced Update!]

Repository: kylin
Updated Branches:
  refs/heads/master-cdh5.7 094b94510 -> cec70b82e (forced update)


KYLIN-2383 let HLLC handle NULL input correctly


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

Branch: refs/heads/master-cdh5.7
Commit: c6108a78eeca8e4b906aeba4f94e28addd974676
Parents: b0a406f
Author: Li Yang <li...@apache.org>
Authored: Thu Jan 12 15:59:51 2017 +0800
Committer: Li Yang <li...@apache.org>
Committed: Thu Jan 12 16:01:17 2017 +0800

----------------------------------------------------------------------
 .../apache/kylin/measure/MeasureIngester.java   |  1 +
 .../org/apache/kylin/measure/MeasureType.java   |  1 +
 .../kylin/measure/hllc/HLLCMeasureType.java     | 20 ++++--
 .../kylin/measure/hllc/HLLCMeasureTypeTest.java | 65 ++++++++++++++++++++
 .../apache/kylin/rest/service/QueryService.java |  3 +
 5 files changed, 86 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/c6108a78/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
index 26b7298..ed2cb02 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
@@ -26,6 +26,7 @@ import java.util.Collection;
 import java.util.Map;
 
 abstract public class MeasureIngester<V> implements java.io.Serializable {
+    private static final long serialVersionUID = 1L;
 
     public static MeasureIngester<?> create(MeasureDesc measure) {
         return measure.getFunction().getMeasureType().newIngester();

http://git-wip-us.apache.org/repos/asf/kylin/blob/c6108a78/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
index 3338c8c..f609dd5 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
@@ -39,6 +39,7 @@ import java.util.Map;
  * @param <T> the Java type of aggregation data object, e.g. HLLCounter
  */
 abstract public class MeasureType<T> implements java.io.Serializable {
+    private static final long serialVersionUID = 1L;
 
     /* ============================================================================
      * Define

http://git-wip-us.apache.org/repos/asf/kylin/blob/c6108a78/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
index de36b08..51c5a66 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
@@ -34,6 +34,7 @@ import org.apache.kylin.metadata.model.TblColRef;
 import com.google.common.collect.ImmutableMap;
 
 public class HLLCMeasureType extends MeasureType<HLLCounter> {
+    private static final long serialVersionUID = 1L;
 
     public static final String FUNC_COUNT_DISTINCT = FunctionDesc.FUNC_COUNT_DISTINCT;
     public static final String DATATYPE_HLLC = "hllc";
@@ -93,15 +94,26 @@ public class HLLCMeasureType extends MeasureType<HLLCounter> {
     @Override
     public MeasureIngester<HLLCounter> newIngester() {
         return new MeasureIngester<HLLCounter>() {
+            private static final long serialVersionUID = 1L;
+            
             HLLCounter current = new HLLCounter(dataType.getPrecision());
 
             @Override
             public HLLCounter valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
                 HLLCounter hllc = current;
                 hllc.clear();
-                for (String v : values) {
-                    if (v != null)
-                        hllc.add(v);
+                if (values.length == 1) {
+                    if (values[0] != null)
+                        hllc.add(values[0]);
+                } else {
+                    boolean allNull = true;
+                    StringBuilder buf = new StringBuilder();
+                    for (String v : values) {
+                        allNull = (allNull && v == null);
+                        buf.append(v);
+                    }
+                    if (!allNull)
+                        hllc.add(buf.toString());
                 }
                 return hllc;
             }
@@ -133,5 +145,5 @@ public class HLLCMeasureType extends MeasureType<HLLCounter> {
     public static boolean isCountDistinct(FunctionDesc func) {
         return FUNC_COUNT_DISTINCT.equalsIgnoreCase(func.getExpression());
     }
-
+    
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/c6108a78/core-metadata/src/test/java/org/apache/kylin/measure/hllc/HLLCMeasureTypeTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/hllc/HLLCMeasureTypeTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/hllc/HLLCMeasureTypeTest.java
new file mode 100644
index 0000000..acd2ea3
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/hllc/HLLCMeasureTypeTest.java
@@ -0,0 +1,65 @@
+/*
+ * 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.measure.hllc;
+
+import static org.junit.Assert.*;
+
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.measure.MeasureTypeFactory;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class HLLCMeasureTypeTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testIngest() {
+        MeasureType<HLLCounter> mtype = (MeasureType<HLLCounter>) MeasureTypeFactory.create(HLLCMeasureType.FUNC_COUNT_DISTINCT, DataType.getType("hllc(10)"));
+        MeasureIngester<HLLCounter> ingester = mtype.newIngester();
+        HLLCounter hllc;
+        
+        hllc = ingester.valueOf(new String[] { null }, null, null);
+        assertEquals(0, hllc.getCountEstimate());
+        
+        hllc = ingester.valueOf(new String[] { null, null }, null, null);
+        assertEquals(0, hllc.getCountEstimate());
+        
+        hllc = ingester.valueOf(new String[] { "" }, null, null);
+        assertEquals(1, hllc.getCountEstimate());
+        
+        hllc = ingester.valueOf(new String[] { "", null }, null, null);
+        assertEquals(1, hllc.getCountEstimate());
+        
+        hllc = ingester.valueOf(new String[] { "abc" }, null, null);
+        assertEquals(1, hllc.getCountEstimate());
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/c6108a78/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 bc644cc..98eb7cb 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
@@ -325,6 +325,9 @@ public class QueryService extends BasicService {
         if (!(Constant.SERVER_MODE_QUERY.equals(serverMode.toLowerCase()) || Constant.SERVER_MODE_ALL.equals(serverMode.toLowerCase()))) {
             throw new InternalErrorException("Query is not allowed in " + serverMode + " mode.");
         }
+        if (StringUtils.isBlank(sqlRequest.getProject())) {
+            throw new InternalErrorException("Project cannot be empty. Please select a project.");
+        }
 
         final String queryId = UUID.randomUUID().toString();
         if (sqlRequest.getBackdoorToggles() != null)


[10/21] kylin git commit: KYLIN-2357 Make ERROR_RECORD_LOG_THRESHOLD configurable

Posted by li...@apache.org.
KYLIN-2357 Make ERROR_RECORD_LOG_THRESHOLD configurable


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

Branch: refs/heads/master-cdh5.7
Commit: 8d374f84368194d78208ea3b7683a697e8ec4314
Parents: eb18ac5
Author: kangkaisen <ka...@live.com>
Authored: Sat Jan 7 19:16:30 2017 +0800
Committer: kangkaisen <ka...@163.com>
Committed: Sat Jan 14 19:14:42 2017 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/kylin/common/KylinConfigBase.java   | 4 ++++
 .../java/org/apache/kylin/engine/mr/common/BatchConstants.java   | 1 -
 .../org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java   | 2 +-
 .../kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java     | 2 +-
 4 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/8d374f84/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 494c641..d6774ff 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
@@ -423,6 +423,10 @@ abstract public class KylinConfigBase implements Serializable {
         return Integer.parseInt(getOptional("kylin.job.scheduler.default", "0"));
     }
 
+    public Integer getErrorRecordThreshold() {
+        return Integer.parseInt(getOptional("kylin.job.error-record-threshold", "0"));
+    }
+
     // ============================================================================
     // SOURCE.HIVE
     // ============================================================================

http://git-wip-us.apache.org/repos/asf/kylin/blob/8d374f84/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
index 078d80f..0281539 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BatchConstants.java
@@ -80,7 +80,6 @@ public interface BatchConstants {
      */
     String MAPREDUCE_COUNTER_GROUP_NAME = "Cube Builder";
     int NORMAL_RECORD_LOG_THRESHOLD = 100000;
-    int ERROR_RECORD_LOG_THRESHOLD = 100;
 
     /**
      * dictionaries builder class

http://git-wip-us.apache.org/repos/asf/kylin/blob/8d374f84/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
index d08e29a..93e413b 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
@@ -87,7 +87,7 @@ abstract public class BaseCuboidMapperBase<KEYIN, VALUEIN> extends KylinMapper<K
 
         // TODO expose errorRecordCounter as hadoop counter
         errorRecordCounter++;
-        if (errorRecordCounter > BatchConstants.ERROR_RECORD_LOG_THRESHOLD) {
+        if (errorRecordCounter > cubeSegment.getConfig().getErrorRecordThreshold()) {
             if (ex instanceof IOException)
                 throw (IOException) ex;
             else if (ex instanceof RuntimeException)

http://git-wip-us.apache.org/repos/asf/kylin/blob/8d374f84/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java
index 74aebb0..34624e3 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java
@@ -105,7 +105,7 @@ abstract public class FactDistinctColumnsMapperBase<KEYIN, VALUEIN> extends Kyli
         ex.printStackTrace(System.err);
 
         errorRecordCounter++;
-        if (errorRecordCounter > BatchConstants.ERROR_RECORD_LOG_THRESHOLD) {
+        if (errorRecordCounter > cubeSeg.getConfig().getErrorRecordThreshold()) {
             if (ex instanceof IOException)
                 throw (IOException) ex;
             else if (ex instanceof RuntimeException)


[02/21] kylin git commit: fix checkstyle

Posted by li...@apache.org.
fix checkstyle


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

Branch: refs/heads/master-cdh5.7
Commit: a11a31da0323c35686d982de6f02ea507573f753
Parents: c6108a7
Author: Li Yang <li...@apache.org>
Authored: Thu Jan 12 16:21:49 2017 +0800
Committer: Li Yang <li...@apache.org>
Committed: Thu Jan 12 16:21:49 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/measure/hllc/HLLCMeasureTypeTest.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/a11a31da/core-metadata/src/test/java/org/apache/kylin/measure/hllc/HLLCMeasureTypeTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/hllc/HLLCMeasureTypeTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/hllc/HLLCMeasureTypeTest.java
index acd2ea3..e884094 100644
--- a/core-metadata/src/test/java/org/apache/kylin/measure/hllc/HLLCMeasureTypeTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/hllc/HLLCMeasureTypeTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.kylin.measure.hllc;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.measure.MeasureIngester;


[20/21] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

Posted by li...@apache.org.
KYLIN-1528 Create a branch for v1.5 with HBase 1.x API


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

Branch: refs/heads/master-cdh5.7
Commit: a34f8151b7e6511ca703cb539d3b14816278eb52
Parents: 576d2dd
Author: shaofengshi <sh...@apache.org>
Authored: Wed Mar 23 17:07:05 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Jan 17 11:28:32 2017 +0800

----------------------------------------------------------------------
 examples/test_case_data/sandbox/hbase-site.xml  | 19 +---
 .../kylin/provision/BuildCubeWithEngine.java    | 19 ++--
 pom.xml                                         | 12 +--
 .../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 ++--
 .../org/apache/kylin/tool/CubeMigrationCLI.java | 19 ++--
 .../kylin/tool/ExtendCubeToHybridCLI.java       |  8 +-
 39 files changed, 366 insertions(+), 437 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 46d5345..734908e 100644
--- a/examples/test_case_data/sandbox/hbase-site.xml
+++ b/examples/test_case_data/sandbox/hbase-site.xml
@@ -190,22 +190,5 @@
         <name>zookeeper.znode.parent</name>
         <value>/hbase-unsecure</value>
     </property>
-    <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/a34f8151/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 62978db..e46cbe1 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,11 +32,9 @@ 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.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.common.util.HBaseMetadataTestCase;
@@ -58,6 +56,7 @@ 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;
@@ -92,10 +91,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);
     }
 
@@ -279,7 +278,7 @@ public class BuildCubeWithEngine {
 
         String cubeName = "ci_inner_join_cube";
         clearSegment(cubeName);
-        
+
         SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
         f.setTimeZone(TimeZone.getTimeZone("GMT"));
         long date1 = 0;
@@ -334,10 +333,10 @@ public class BuildCubeWithEngine {
 
     @SuppressWarnings("unused")
     private void checkHFilesInHBase(CubeSegment segment) throws IOException {
-        Configuration conf = HBaseConfiguration.create(HadoopUtil.getCurrentConfiguration());
-        String tableName = segment.getStorageLocationIdentifier();
-        try (HTable table = new HTable(conf, tableName)) {
-            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
+        try (Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl())) {
+            String tableName = segment.getStorageLocationIdentifier();
+
+            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
             Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
             long totalSize = 0;
             for (Long size : sizeMap.values()) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e8ccb6c..1cea2d5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -46,20 +46,20 @@
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
 
         <!-- Hadoop versions -->
-        <hadoop2.version>2.6.0</hadoop2.version>
-        <yarn.version>2.6.0</yarn.version>
+        <hadoop2.version>2.7.1</hadoop2.version>
+        <yarn.version>2.7.1</yarn.version>
 
         <!-- Hive versions -->
-        <hive.version>0.14.0</hive.version>
-        <hive-hcatalog.version>0.14.0</hive-hcatalog.version>
+        <hive.version>1.2.1</hive.version>
+        <hive-hcatalog.version>1.2.1</hive-hcatalog.version>
 
         <!-- HBase versions -->
-        <hbase-hadoop2.version>0.98.8-hadoop2</hbase-hadoop2.version>
+        <hbase-hadoop2.version>1.1.1</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.6.0</curator.version>
+        <curator.version>2.7.1</curator.version>
         <jackson.version>2.2.4</jackson.version>
         <jsr305.version>3.0.1</jsr305.version>
         <guava.version>14.0</guava.version>

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 ea68855..8095bf8 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.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 
 /**
  */
@@ -36,6 +36,6 @@ public interface AclHBaseStorage {
 
     String prepareHBaseTable(Class<?> clazz) throws IOException;
 
-    HTableInterface getTable(String tableName) throws IOException;
+    Table getTable(String tableName) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 d9326f5..cc76b87 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.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 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 HTableInterface mockedAclTable;
-    private HTableInterface mockedUserTable;
+    private Table mockedAclTable;
+    private Table mockedUserTable;
     private RealAclHBaseStorage realAcl;
 
     public MockAclHBaseStorage() {
@@ -65,7 +65,7 @@ public class MockAclHBaseStorage implements AclHBaseStorage {
     }
 
     @Override
-    public HTableInterface getTable(String tableName) throws IOException {
+    public Table getTable(String tableName) throws IOException {
         if (realAcl != null) {
             return realAcl.getTable(tableName);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 d0aa0ed..972eea9 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.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 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 HTableInterface {
+public class MockHTable implements Table {
     private final String tableName;
     private final List<String> columnFamilies = new ArrayList<>();
 
@@ -114,14 +114,6 @@ public class MockHTable implements HTableInterface {
         this.columnFamilies.add(columnFamily);
     }
 
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public byte[] getTableName() {
-        return tableName.getBytes();
-    }
-
     @Override
     public TableName getName() {
         return null;
@@ -200,8 +192,8 @@ public class MockHTable implements HTableInterface {
     }
 
     @Override
-    public Boolean[] exists(List<Get> gets) throws IOException {
-        return new Boolean[0];
+    public boolean[] existsAll(List<Get> list) throws IOException {
+        return new boolean[0];
     }
 
     /**
@@ -306,15 +298,6 @@ public class MockHTable implements HTableInterface {
      * {@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();
@@ -446,7 +429,7 @@ public class MockHTable implements HTableInterface {
              */
         }
         if (filter.hasFilterRow() && !filteredOnRowKey) {
-            filter.filterRow(nkvs);
+            filter.filterRow();
         }
         if (filter.filterRow() || filteredOnRowKey) {
             nkvs.clear();
@@ -535,6 +518,11 @@ public class MockHTable implements HTableInterface {
         return false;
     }
 
+    @Override
+    public boolean checkAndPut(byte[] bytes, byte[] bytes1, byte[] bytes2, CompareFilter.CompareOp compareOp, byte[] bytes3, Put put) throws IOException {
+        return false;
+    }
+
     /**
      * {@inheritDoc}
      */
@@ -555,7 +543,7 @@ public class MockHTable implements HTableInterface {
                 continue;
             }
             for (KeyValue kv : delete.getFamilyMap().get(family)) {
-                if (kv.isDeleteFamily()) {
+                if (kv.isDelete()) {
                     data.get(row).get(kv.getFamily()).clear();
                 } else {
                     data.get(row).get(kv.getFamily()).remove(kv.getQualifier());
@@ -592,6 +580,11 @@ public class MockHTable implements HTableInterface {
         return false;
     }
 
+    @Override
+    public boolean checkAndDelete(byte[] bytes, byte[] bytes1, byte[] bytes2, CompareFilter.CompareOp compareOp, byte[] bytes3, Delete delete) throws IOException {
+        return false;
+    }
+
     /**
      * {@inheritDoc}
      */
@@ -605,7 +598,7 @@ public class MockHTable implements HTableInterface {
      */
     @Override
     public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) throws IOException {
-        return incrementColumnValue(row, family, qualifier, amount, true);
+        return incrementColumnValue(row, family, qualifier, amount, null);
     }
 
     @Override
@@ -617,37 +610,6 @@ public class MockHTable implements HTableInterface {
      * {@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 {
     }
 
@@ -673,29 +635,6 @@ public class MockHTable implements HTableInterface {
      * {@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/a34f8151/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 1d520c4..d1a1384 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,7 +21,8 @@ package org.apache.kylin.rest.security;
 import java.io.IOException;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.rest.service.AclService;
 import org.apache.kylin.rest.service.QueryService;
@@ -58,11 +59,11 @@ public class RealAclHBaseStorage implements AclHBaseStorage {
     }
 
     @Override
-    public HTableInterface getTable(String tableName) throws IOException {
+    public Table getTable(String tableName) throws IOException {
         if (StringUtils.equals(tableName, aclTableName)) {
-            return HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            return HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
         } else if (StringUtils.equals(tableName, userTableName)) {
-            return HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            return HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
         } else {
             throw new IllegalStateException("getTable failed" + tableName);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 d693a67..3e3efec 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.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 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>();
-        HTableInterface htable = null;
+        Table 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>();
-        HTableInterface htable = null;
+        Table htable = null;
         Result result = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
@@ -226,17 +226,16 @@ public class AclService implements MutableAclService {
         Authentication auth = SecurityContextHolder.getContext().getAuthentication();
         PrincipalSid sid = new PrincipalSid(auth);
 
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
             Put put = new Put(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
-            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));
+            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));
 
             htable.put(put);
-            htable.flushCommits();
 
             logger.debug("ACL of " + objectIdentity + " created successfully.");
         } catch (IOException e) {
@@ -250,7 +249,7 @@ public class AclService implements MutableAclService {
 
     @Override
     public void deleteAcl(ObjectIdentity objectIdentity, boolean deleteChildren) throws ChildrenExistException {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
@@ -266,7 +265,6 @@ public class AclService implements MutableAclService {
             }
 
             htable.delete(delete);
-            htable.flushCommits();
 
             logger.debug("ACL of " + objectIdentity + " deleted successfully.");
         } catch (IOException e) {
@@ -284,7 +282,7 @@ public class AclService implements MutableAclService {
             throw e;
         }
 
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
@@ -295,17 +293,16 @@ public class AclService implements MutableAclService {
             Put put = new Put(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
 
             if (null != acl.getParentAcl()) {
-                put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
+                put.addColumn(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.add(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
+                put.addColumn(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/a34f8151/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 d28c87c..c8c87cb 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,9 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.WeakHashMap;
 
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeInstance;
@@ -395,33 +393,24 @@ public class CubeService extends BasicService {
         if (htableInfoCache.containsKey(tableName)) {
             return htableInfoCache.get(tableName);
         }
-
-        Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
-        HTable table = null;
+        Connection conn = HBaseConnection.get(this.getConfig().getStorageUrl());
         HBaseResponse hr = null;
         long tableSize = 0;
         int regionCount = 0;
 
-        try {
-            table = new HTable(hconf, tableName);
-
-            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
-            Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
+        HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
+        Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
 
-            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);
+        for (long s : sizeMap.values()) {
+            tableSize += s;
         }
 
+        regionCount = sizeMap.size();
+
+        // 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/a34f8151/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 98eb7cb..7ce38ea 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,14 +164,13 @@ public class QueryService extends BasicService {
         Query[] queryArray = new Query[queries.size()];
 
         byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Put put = new Put(Bytes.toBytes(creator));
-            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+            put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
 
             htable.put(put);
-            htable.flushCommits();
         } finally {
             IOUtils.closeQuietly(htable);
         }
@@ -197,14 +196,13 @@ public class QueryService extends BasicService {
 
         Query[] queryArray = new Query[queries.size()];
         byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Put put = new Put(Bytes.toBytes(creator));
-            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+            put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
 
             htable.put(put);
-            htable.flushCommits();
         } finally {
             IOUtils.closeQuietly(htable);
         }
@@ -216,12 +214,12 @@ public class QueryService extends BasicService {
         }
 
         List<Query> queries = new ArrayList<Query>();
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            HConnection conn = HBaseConnection.get(hbaseUrl);
+            org.apache.hadoop.hbase.client.Connection conn = HBaseConnection.get(hbaseUrl);
             HBaseConnection.createHTableIfNeeded(conn, userTableName, USER_QUERY_FAMILY);
 
-            htable = conn.getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(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/a34f8151/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 07c7c6f..ab54882 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 {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
@@ -144,16 +144,16 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public void updateUser(UserDetails user) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
             Pair<byte[], byte[]> pair = userToHBaseRow(user);
             Put put = new Put(pair.getKey());
-            put.add(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
+
+            put.addColumn(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,14 +163,13 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public void deleteUser(String username) {
-        HTableInterface htable = null;
+        Table 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 {
@@ -185,7 +184,7 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public boolean userExists(String username) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
@@ -216,7 +215,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>();
-        HTableInterface htable = null;
+        Table htable = null;
         ResultScanner scanner = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 cbf81b6..b769391 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.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
+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.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, HConnection> connPool = new ConcurrentHashMap<String, HConnection>();
+    private static final Map<String, Connection> connPool = new ConcurrentHashMap<String, Connection>();
     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 (HConnection conn : connPool.values()) {
+                for (Connection 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 HConnection can be shared by multiple threads and does not require close()
+    // returned Connection can be shared by multiple threads and does not require close()
     @SuppressWarnings("resource")
-    public static HConnection get(String url) {
+    public static Connection get(String url) {
         // find configuration
         Configuration conf = configCache.get(url);
         if (conf == null) {
@@ -223,13 +223,13 @@ public class HBaseConnection {
             configCache.put(url, conf);
         }
 
-        HConnection connection = connPool.get(url);
+        Connection 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 = HConnectionManager.createConnection(conf);
+                    connection = ConnectionFactory.createConnection(conf);
                     connPool.put(url, connection);
                 }
 
@@ -248,8 +248,8 @@ public class HBaseConnection {
         return connection;
     }
 
-    public static boolean tableExists(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    public static boolean tableExists(Connection conn, String tableName) throws IOException {
+        Admin hbase = conn.getAdmin();
         try {
             return hbase.tableExists(TableName.valueOf(tableName));
         } finally {
@@ -269,18 +269,18 @@ public class HBaseConnection {
         deleteTable(HBaseConnection.get(hbaseUrl), tableName);
     }
 
-    public static void createHTableIfNeeded(HConnection conn, String table, String... families) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
-
+    public static void createHTableIfNeeded(Connection conn, String table, String... families) throws IOException {
+        Admin hbase = conn.getAdmin();
+        TableName tableName = TableName.valueOf(table);
         try {
             if (tableExists(conn, table)) {
                 logger.debug("HTable '" + table + "' already exists");
-                Set<String> existingFamilies = getFamilyNames(hbase.getTableDescriptor(TableName.valueOf(table)));
+                Set<String> existingFamilies = getFamilyNames(hbase.getTableDescriptor(tableName));
                 boolean wait = false;
                 for (String family : families) {
                     if (existingFamilies.contains(family) == false) {
                         logger.debug("Adding family '" + family + "' to HTable '" + table + "'");
-                        hbase.addColumn(table, newFamilyDescriptor(family));
+                        hbase.addColumn(tableName, 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(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    public static void deleteTable(Connection conn, String tableName) throws IOException {
+        Admin hbase = conn.getAdmin();
 
         try {
             if (!tableExists(conn, tableName)) {
@@ -344,10 +344,10 @@ public class HBaseConnection {
 
             logger.debug("delete HTable '" + tableName + "'");
 
-            if (hbase.isTableEnabled(tableName)) {
-                hbase.disableTable(tableName);
+            if (hbase.isTableEnabled(TableName.valueOf(tableName))) {
+                hbase.disableTable(TableName.valueOf(tableName));
             }
-            hbase.deleteTable(tableName);
+            hbase.deleteTable(TableName.valueOf(tableName));
 
             logger.debug("HTable '" + tableName + "' deleted");
         } finally {

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 2a12984..2a4a2f0 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,14 +30,15 @@ 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 +70,7 @@ public class HBaseResourceStore extends ResourceStore {
     final String tableNameBase;
     final String hbaseUrl;
 
-    HConnection getConnection() throws IOException {
+    Connection getConnection() throws IOException {
         return HBaseConnection.get(hbaseUrl);
     }
 
@@ -120,7 +121,7 @@ public class HBaseResourceStore extends ResourceStore {
         byte[] endRow = Bytes.toBytes(lookForPrefix);
         endRow[endRow.length - 1]++;
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         Scan scan = new Scan(startRow, endRow);
         if ((filter != null && filter instanceof KeyOnlyFilter) == false) {
             scan.addColumn(B_FAMILY, B_COLUMN_TS);
@@ -237,13 +238,12 @@ public class HBaseResourceStore extends ResourceStore {
         IOUtils.copy(content, bout);
         bout.close();
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(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 {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             byte[] row = Bytes.toBytes(resPath);
             byte[] bOldTS = oldTS == 0 ? null : Bytes.toBytes(oldTS);
@@ -264,8 +264,6 @@ 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);
@@ -274,7 +272,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     @Override
     protected void deleteResourceImpl(String resPath) throws IOException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             boolean hdfsResourceExist = false;
             Result result = internalGetFromHTable(table, resPath, true, false);
@@ -287,7 +285,6 @@ 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);
@@ -308,7 +305,7 @@ public class HBaseResourceStore extends ResourceStore {
     }
 
     private Result getFromHTable(String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             return internalGetFromHTable(table, path, fetchContent, fetchTimestamp);
         } finally {
@@ -316,7 +313,7 @@ public class HBaseResourceStore extends ResourceStore {
         }
     }
 
-    private Result internalGetFromHTable(HTableInterface table, String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
+    private Result internalGetFromHTable(Table table, String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
         byte[] rowkey = Bytes.toBytes(path);
 
         Get get = new Get(rowkey);
@@ -335,7 +332,7 @@ public class HBaseResourceStore extends ResourceStore {
         return exists ? result : null;
     }
 
-    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, HTableInterface table) throws IOException {
+    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, Table table) throws IOException {
         Path redirectPath = bigCellHDFSPath(resPath);
         FileSystem fileSystem = HadoopUtil.getWorkingFileSystem();
 
@@ -360,7 +357,7 @@ public class HBaseResourceStore extends ResourceStore {
         return redirectPath;
     }
 
-    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, HTableInterface table) throws IOException {
+    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, Table table) throws IOException {
         int kvSizeLimit = Integer.parseInt(getConnection().getConfiguration().get("hbase.client.keyvalue.maxsize", "10485760"));
         if (content.length > kvSizeLimit) {
             writeLargeCellToHdfs(resPath, content, table);
@@ -368,8 +365,8 @@ public class HBaseResourceStore extends ResourceStore {
         }
 
         Put put = new Put(row);
-        put.add(B_FAMILY, B_COLUMN, content);
-        put.add(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
+        put.addColumn(B_FAMILY, B_COLUMN, content);
+        put.addColumn(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
 
         return put;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 b141190..f63d9c2 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,12 +26,13 @@ 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.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.Connection;
 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;
@@ -86,14 +87,13 @@ public class SimpleHBaseStore implements IGTStore {
     }
 
     private class Writer implements IGTWriter {
-        final HTableInterface table;
+        final BufferedMutator table;
         final ByteBuffer rowkey = ByteBuffer.allocate(50);
         final ByteBuffer value = ByteBuffer.allocate(50);
 
         Writer() throws IOException {
-            HConnection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
-            table = conn.getTable(htableName);
-            table.setAutoFlush(false, true);
+            Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+            table = conn.getBufferedMutator(htableName);
         }
 
         @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.put(put);
+            table.mutate(put);
         }
 
         @Override
         public void close() throws IOException {
-            table.flushCommits();
+            table.flush();
             table.close();
         }
     }
 
     class Reader implements IGTScanner {
-        final HTableInterface table;
+        final Table table;
         final ResultScanner scanner;
 
         int count = 0;
 
         Reader() throws IOException {
-            HConnection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+            Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
             table = conn.getTable(htableName);
 
             Scan scan = new Scan();

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 df1817e..cad5a3f 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
@@ -26,8 +26,9 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.zip.DataFormatException;
 
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+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.coprocessor.Batch;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
@@ -51,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.CubeVisitResponse;
-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;
 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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -117,7 +118,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
         final ImmutableBitSet selectedColBlocks = scanRequest.getSelectedColBlocks().set(0);
 
         // globally shared connection, does not require close
-        final HConnection conn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
+        final Connection conn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
 
         final List<IntList> hbaseColumnsToGTIntList = Lists.newArrayList();
         List<List<Integer>> hbaseColumnsToGT = getHBaseColumnsGTMapping(selectedColBlocks);
@@ -172,7 +173,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
                     final boolean[] abnormalFinish = new boolean[1];
 
                     try {
-                        HTableInterface table = conn.getTable(cubeSeg.getStorageLocationIdentifier(), HBaseConnection.getCoprocessorPool());
+                        Table table = conn.getTable(TableName.valueOf(cubeSeg.getStorageLocationIdentifier()), HBaseConnection.getCoprocessorPool());
 
                         final CubeVisitRequest request = builder.build();
                         final byte[] startKey = epRange.getFirst();

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 3cefc5f..a52af90 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,11 +24,12 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 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;
@@ -154,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
-        HConnection hbaseConn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
-        final HTableInterface hbaseTable = hbaseConn.getTable(cubeSeg.getStorageLocationIdentifier());
+        Connection hbaseConn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
+        final Table hbaseTable = hbaseConn.getTable(TableName.valueOf(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/a34f8151/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 21a0efb..810747f 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
@@ -142,7 +142,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
         if (shardLength == 0) {
             return;
         }
-        byte[] regionStartKey = ArrayUtils.isEmpty(region.getStartKey()) ? new byte[shardLength] : region.getStartKey();
+        byte[] regionStartKey = ArrayUtils.isEmpty(region.getRegionInfo().getStartKey()) ? new byte[shardLength] : region.getRegionInfo().getStartKey();
         Bytes.putBytes(rawScan.startKey, 0, regionStartKey, 0, shardLength);
         Bytes.putBytes(rawScan.endKey, 0, regionStartKey, 0, shardLength);
     }
@@ -179,7 +179,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
         try (SetThreadName ignored = new SetThreadName("Query %s", queryId)) {
             this.serviceStartTime = System.currentTimeMillis();
 
-            region = env.getRegion();
+            region = (HRegion)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/a34f8151/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 2814ad6..feb4842 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,7 +26,8 @@ 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.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.regionserver.BloomType;
@@ -79,7 +80,8 @@ public class CubeHTableUtil {
         tableDesc.setValue(IRealizationConstants.HTableSegmentTag, cubeSegment.toString());
 
         Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+        Admin admin = conn.getAdmin();
 
         try {
             if (User.isHBaseSecurityEnabled(conf)) {
@@ -92,7 +94,7 @@ public class CubeHTableUtil {
                 tableDesc.addFamily(cf);
             }
 
-            if (admin.tableExists(tableName)) {
+            if (admin.tableExists(TableName.valueOf(tableName))) {
                 // admin.disableTable(tableName);
                 // admin.deleteTable(tableName);
                 throw new RuntimeException("HBase table " + tableName + " exists!");
@@ -101,7 +103,7 @@ public class CubeHTableUtil {
             DeployCoprocessorCLI.deployCoprocessor(tableDesc);
 
             admin.createTable(tableDesc, splitKeys);
-            Preconditions.checkArgument(admin.isTableAvailable(tableName), "table " + tableName + " created, but is not available due to some reasons");
+            Preconditions.checkArgument(admin.isTableAvailable(TableName.valueOf(tableName)), "table " + tableName + " created, but is not available due to some reasons");
             logger.info("create hbase table " + tableName + " done.");
         } finally {
             IOUtils.closeQuietly(admin);
@@ -110,8 +112,7 @@ public class CubeHTableUtil {
     }
 
     public static void deleteHTable(TableName tableName) throws IOException {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Admin admin = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getAdmin();
         try {
             if (admin.tableExists(tableName)) {
                 logger.info("disabling hbase table " + tableName);
@@ -126,8 +127,7 @@ 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 {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Admin admin = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getAdmin();
         try {
             if (admin.tableExists(tableName)) {
                 logger.info("disabling hbase table " + tableName);

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 fbe64d9..347807d 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.client.HBaseAdmin;
+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.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.HiveCmdBuilder;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.job.exception.ExecuteException;
@@ -100,19 +100,21 @@ public class DeprecatedGCStep extends AbstractExecutable {
         List<String> oldTables = getOldHTables();
         if (oldTables != null && oldTables.size() > 0) {
             String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-            HBaseAdmin admin = null;
+            Admin admin = null;
             try {
-                admin = new HBaseAdmin(conf);
+
+                Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+                admin = conn.getAdmin();
+
                 for (String table : oldTables) {
-                    if (admin.tableExists(table)) {
-                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
+                    if (admin.tableExists(TableName.valueOf(table))) {
+                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(TableName.valueOf(table));
                         String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
                         if (metadataUrlPrefix.equalsIgnoreCase(host)) {
-                            if (admin.isTableEnabled(table)) {
-                                admin.disableTable(table);
+                            if (admin.isTableEnabled(TableName.valueOf(table))) {
+                                admin.disableTable(TableName.valueOf(table));
                             }
-                            admin.deleteTable(table);
+                            admin.deleteTable(TableName.valueOf(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/a34f8151/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 d5b36df..6587d4e 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 HTableInterface hTable;
+    private final Table 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, HTableInterface hTable) {
+    public HBaseCuboidWriter(CubeSegment segment, Table hTable) {
         this.keyValueCreators = Lists.newArrayList();
         this.cubeSegment = segment;
         this.cubeDesc = cubeSegment.getCubeDesc();
@@ -117,7 +117,6 @@ 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/a34f8151/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 5b2441c..2f7e164 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.client.HBaseAdmin;
+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.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,19 +69,20 @@ public class MergeGCStep extends AbstractExecutable {
         List<String> oldTables = getOldHTables();
         if (oldTables != null && oldTables.size() > 0) {
             String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-            HBaseAdmin admin = null;
+            Admin admin = null;
             try {
-                admin = new HBaseAdmin(conf);
+                Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+                admin = conn.getAdmin();
+
                 for (String table : oldTables) {
-                    if (admin.tableExists(table)) {
-                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
+                    if (admin.tableExists(TableName.valueOf(table))) {
+                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(TableName.valueOf((table)));
                         String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
                         if (metadataUrlPrefix.equalsIgnoreCase(host)) {
-                            if (admin.isTableEnabled(table)) {
-                                admin.disableTable(table);
+                            if (admin.isTableEnabled(TableName.valueOf(table))) {
+                                admin.disableTable(TableName.valueOf(table));
                             }
-                            admin.deleteTable(table);
+                            admin.deleteTable(TableName.valueOf(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/a34f8151/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 a150607..56f867a 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,9 +21,11 @@ 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.client.HBaseAdmin;
+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.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
@@ -38,8 +40,8 @@ public class CleanHtableCLI extends AbstractApplication {
     protected static final Logger logger = LoggerFactory.getLogger(CleanHtableCLI.class);
 
     private void clean() throws IOException {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
 
         for (HTableDescriptor descriptor : hbaseAdmin.listTables()) {
             String name = descriptor.getNameAsString().toLowerCase();
@@ -50,7 +52,7 @@ public class CleanHtableCLI extends AbstractApplication {
                 System.out.println();
 
                 descriptor.setValue(IRealizationConstants.HTableOwner, "DL-eBay-Kylin@ebay.com");
-                hbaseAdmin.modifyTable(descriptor.getNameAsString(), descriptor);
+                hbaseAdmin.modifyTable(TableName.valueOf(descriptor.getNameAsString()), descriptor);
             }
         }
         hbaseAdmin.close();

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 31864f6..40e9fb8 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 HBaseAdmin hbaseAdmin;
+    private static Admin 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();
 
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(srcConfig.getStorageUrl());
+        hbaseAdmin = conn.getAdmin();
 
         hdfsFS = HadoopUtil.getWorkingFileSystem();
 
@@ -233,6 +233,7 @@ 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))
@@ -326,8 +327,8 @@ public class CubeMigrationCLI {
 
         switch (opt.type) {
         case CHANGE_HTABLE_HOST: {
-            String tableName = (String) opt.params[0];
-            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+            TableName tableName = TableName.valueOf((String) opt.params[0]);
+            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(tableName);
             hbaseAdmin.disableTable(tableName);
             desc.setValue(IRealizationConstants.HTableTag, dstConfig.getMetadataUrlPrefix());
             hbaseAdmin.modifyTable(tableName, desc);
@@ -449,11 +450,11 @@ public class CubeMigrationCLI {
             Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
             ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
             String projUUID = project.getUuid();
-            HTableInterface srcAclHtable = null;
-            HTableInterface destAclHtable = null;
+            Table srcAclHtable = null;
+            Table destAclHtable = null;
             try {
-                srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
-                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+                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));
 
                 // cube acl
                 Result result = srcAclHtable.get(new Get(Bytes.toBytes(cubeId)));
@@ -473,7 +474,6 @@ 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: {
-            String tableName = (String) opt.params[0];
-            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+            TableName tableName = TableName.valueOf((String) opt.params[0]);
+            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(tableName);
             hbaseAdmin.disableTable(tableName);
             desc.setValue(IRealizationConstants.HTableTag, srcConfig.getMetadataUrlPrefix());
             hbaseAdmin.modifyTable(tableName, desc);
@@ -539,13 +539,12 @@ public class CubeMigrationCLI {
         case COPY_ACL: {
             String cubeId = (String) opt.params[0];
             String modelId = (String) opt.params[1];
-            HTableInterface destAclHtable = null;
+            Table destAclHtable = null;
             try {
-                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(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);
             }
@@ -562,7 +561,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/a34f8151/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 8bd4abf..20d0f7d 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.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 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 HBaseAdmin hbaseAdmin;
+    private Admin hbaseAdmin;
 
     private List<String> issueExistHTables;
     private List<String> inconsistentHTables;
@@ -130,9 +130,8 @@ public class CubeMigrationCheckCLI {
         this.dstCfg = kylinConfig;
         this.ifFix = isFix;
 
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        hbaseAdmin = new HBaseAdmin(conf);
-
+        Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+        hbaseAdmin = conn.getAdmin();
         issueExistHTables = Lists.newArrayList();
         inconsistentHTables = Lists.newArrayList();
     }
@@ -189,10 +188,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(sepNameList[0]);
+                hbaseAdmin.disableTable(TableName.valueOf(sepNameList[0]));
                 desc.setValue(IRealizationConstants.HTableTag, dstCfg.getMetadataUrlPrefix());
-                hbaseAdmin.modifyTable(sepNameList[0], desc);
-                hbaseAdmin.enableTable(sepNameList[0]);
+                hbaseAdmin.modifyTable(TableName.valueOf(sepNameList[0]), desc);
+                hbaseAdmin.enableTable(TableName.valueOf(sepNameList[0]));
             }
         } else {
             logger.info("------ Inconsistent HTables Needed To Be Fixed ------");


[03/21] kylin git commit: KYLIN-2378 Set job thread name with job uuid

Posted by li...@apache.org.
KYLIN-2378 Set job thread name with job uuid


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

Branch: refs/heads/master-cdh5.7
Commit: 0f36b9b37097b95c16dd62cbe32781e5af8911dc
Parents: a11a31d
Author: kangkaisen <ka...@live.com>
Authored: Mon Jan 9 17:21:38 2017 +0800
Committer: kangkaisen <ka...@163.com>
Committed: Thu Jan 12 17:42:54 2017 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/job/impl/threadpool/DefaultScheduler.java    | 3 ++-
 .../apache/kylin/job/impl/threadpool/DistributedScheduler.java    | 3 ++-
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/0f36b9b3/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
index ddb3a21..403abc4 100644
--- a/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
+++ b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DefaultScheduler.java
@@ -29,6 +29,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.kylin.common.util.SetThreadName;
 import org.apache.kylin.job.Scheduler;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.engine.JobEngineConfig;
@@ -137,7 +138,7 @@ public class DefaultScheduler implements Scheduler<AbstractExecutable>, Connecti
 
         @Override
         public void run() {
-            try {
+            try (SetThreadName ignored = new SetThreadName("Job %s", executable.getId())) {
                 executable.execute(context);
                 // trigger the next step asap
                 fetcherPool.schedule(fetcher, 0, TimeUnit.SECONDS);

http://git-wip-us.apache.org/repos/asf/kylin/blob/0f36b9b3/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DistributedScheduler.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DistributedScheduler.java b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DistributedScheduler.java
index be07d76..3436529 100644
--- a/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DistributedScheduler.java
+++ b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DistributedScheduler.java
@@ -35,6 +35,7 @@ import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.SetThreadName;
 import org.apache.kylin.job.Scheduler;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.engine.JobEngineConfig;
@@ -173,7 +174,7 @@ public class DistributedScheduler implements Scheduler<AbstractExecutable>, Conn
 
         @Override
         public void run() {
-            try {
+            try (SetThreadName ignored = new SetThreadName("Job %s", executable.getId())) {
                 String segmentId = executable.getParam(SEGMENT_ID);
                 if (jobLock.lockWithName(segmentId, serverName)) {
                     logger.info(executable.toString() + " scheduled in server: " + serverName);


[06/21] kylin git commit: KYLIN-2387 A new BitmapCounter with better performance

Posted by li...@apache.org.
KYLIN-2387 A new BitmapCounter with better performance


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

Branch: refs/heads/master-cdh5.7
Commit: d19533c45414315b3ec29428edfd7c1df6d33604
Parents: 4b97721
Author: gaodayue <ga...@meituan.com>
Authored: Wed Jan 11 15:12:31 2017 +0800
Committer: gaodayue <ga...@meituan.com>
Committed: Fri Jan 13 22:32:05 2017 +0800

----------------------------------------------------------------------
 .../common/util/ByteBufferOutputStream.java     |  43 +++++
 .../gridtable/AggregationCacheMemSizeTest.java  |  10 +-
 .../metadata/measure/MeasureCodecTest.java      |  10 +-
 .../kylin/measure/bitmap/BitmapAggregator.java  |  46 +++--
 .../kylin/measure/bitmap/BitmapCounter.java     | 181 ++++---------------
 .../bitmap/BitmapDistinctCountAggFunc.java      |  35 ++--
 .../BitmapIntersectDistinctCountAggFunc.java    |  37 ++--
 .../kylin/measure/bitmap/BitmapMeasureType.java |  52 +++---
 .../kylin/measure/bitmap/BitmapSerializer.java  |  31 +---
 .../measure/bitmap/ImmutableBitmapCounter.java  | 112 ++++++++++++
 .../measure/bitmap/MutableBitmapCounter.java    |  60 ++++++
 .../measure/AggregatorMemEstimateTest.java      |   4 +-
 .../measure/bitmap/BitmapAggregatorTest.java    |  38 ++--
 .../kylin/measure/bitmap/BitmapCounterTest.java |  88 +++++----
 .../measure/bitmap/BitmapSerializerTest.java    |  60 +++---
 15 files changed, 457 insertions(+), 350 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-common/src/main/java/org/apache/kylin/common/util/ByteBufferOutputStream.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/ByteBufferOutputStream.java b/core-common/src/main/java/org/apache/kylin/common/util/ByteBufferOutputStream.java
new file mode 100644
index 0000000..2e3ff07
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/util/ByteBufferOutputStream.java
@@ -0,0 +1,43 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+/**
+ * An OutputStream whose target is a {@link ByteBuffer}.
+ */
+public class ByteBufferOutputStream extends OutputStream {
+    protected final ByteBuffer buffer;
+
+    public ByteBufferOutputStream(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+        buffer.put((byte) b);
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+        buffer.put(b, off, len);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
index b164e54..63c7672 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
@@ -33,7 +33,7 @@ import org.apache.kylin.measure.basic.BigDecimalSumAggregator;
 import org.apache.kylin.measure.basic.DoubleSumAggregator;
 import org.apache.kylin.measure.basic.LongSumAggregator;
 import org.apache.kylin.measure.bitmap.BitmapAggregator;
-import org.apache.kylin.measure.bitmap.BitmapCounter;
+import org.apache.kylin.measure.bitmap.MutableBitmapCounter;
 import org.apache.kylin.measure.hllc.HLLCAggregator;
 import org.apache.kylin.measure.hllc.HLLCounter;
 import org.github.jamm.MemoryMeter;
@@ -43,13 +43,13 @@ import com.google.common.base.Stopwatch;
 
 public class AggregationCacheMemSizeTest {
     private static final MemoryMeter meter = new MemoryMeter();
-    private static final BitmapCounter[] bitmaps = new BitmapCounter[5];
+    private static final MutableBitmapCounter[] bitmaps = new MutableBitmapCounter[5];
     private static final Random random = new Random();
 
     // consider bitmaps with variant cardinality
     static {
         for (int i = 0; i < bitmaps.length; i++) {
-            bitmaps[i] = new BitmapCounter();
+            bitmaps[i] = new MutableBitmapCounter();
         }
 
         final int totalBits = 1_000_000;
@@ -116,8 +116,8 @@ public class AggregationCacheMemSizeTest {
     }
 
     private BitmapAggregator createBitmapAggr(boolean lowCardinality) {
-        BitmapCounter counter = new BitmapCounter();
-        counter.merge(lowCardinality ? bitmaps[0] : bitmaps[3]);
+        MutableBitmapCounter counter = new MutableBitmapCounter();
+        counter.orWith(lowCardinality ? bitmaps[0] : bitmaps[3]);
 
         BitmapAggregator result = new BitmapAggregator();
         result.aggregate(counter);

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
index ad4d90b..97c9751 100644
--- a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
@@ -6,9 +6,9 @@
  * 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.
@@ -25,7 +25,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.measure.BufferedMeasureCodec;
-import org.apache.kylin.measure.bitmap.BitmapCounter;
+import org.apache.kylin.measure.bitmap.MutableBitmapCounter;
 import org.apache.kylin.measure.hllc.HLLCounter;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
@@ -34,7 +34,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
- * 
+ *
  */
 public class MeasureCodecTest extends LocalFileMetadataTestCase {
     @BeforeClass
@@ -58,7 +58,7 @@ public class MeasureCodecTest extends LocalFileMetadataTestCase {
         HLLCounter hllc = new HLLCounter(16);
         hllc.add("1234567");
         hllc.add("abcdefg");
-        BitmapCounter bitmap = new BitmapCounter();
+        MutableBitmapCounter bitmap = new MutableBitmapCounter();
         bitmap.add(123);
         bitmap.add(45678);
         bitmap.add(Integer.MAX_VALUE - 10);

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java
index cd0b4bb..2c91bfa 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java
@@ -20,37 +20,49 @@ package org.apache.kylin.measure.bitmap;
 
 import org.apache.kylin.measure.MeasureAggregator;
 
-/**
- * Created by sunyerui on 15/12/2.
- */
 public class BitmapAggregator extends MeasureAggregator<BitmapCounter> {
 
-    private BitmapCounter sum = null;
+    private ImmutableBitmapCounter sum;
+    private boolean isMutable;
 
     @Override
     public void reset() {
         sum = null;
+        isMutable = false;
     }
 
     @Override
     public void aggregate(BitmapCounter value) {
+        ImmutableBitmapCounter v = (ImmutableBitmapCounter) value;
+
+        // Here we optimize for case when group only has 1 value. In such situation, no
+        // aggregation is needed, so we just keep a reference to the first value, saving
+        // the cost of deserialization and merging.
         if (sum == null) {
-            sum = new BitmapCounter(value);
-        } else {
-            sum.merge(value);
+            sum = v;
+            return;
+        }
+
+        MutableBitmapCounter mutable;
+        if (!isMutable) {   // when aggregate the second value
+            mutable = sum.toMutable();
+            sum = mutable;
+            isMutable = true;
+        } else {    // for the third, forth, ...
+            mutable = (MutableBitmapCounter) sum;
         }
+        mutable.orWith(v);
     }
 
     @Override
     public BitmapCounter aggregate(BitmapCounter value1, BitmapCounter value2) {
-        if (value1 == null) {
-            return new BitmapCounter(value2);
-        } else if (value2 == null) {
-            return new BitmapCounter(value1);
+        MutableBitmapCounter merged = new MutableBitmapCounter();
+        if (value1 != null) {
+            merged.orWith((ImmutableBitmapCounter) value1);
+        }
+        if (value2 != null) {
+            merged.orWith((ImmutableBitmapCounter) value2);
         }
-
-        BitmapCounter merged = new BitmapCounter(value1);
-        merged.merge(value2);
         return merged;
     }
 
@@ -61,10 +73,6 @@ public class BitmapAggregator extends MeasureAggregator<BitmapCounter> {
 
     @Override
     public int getMemBytesEstimate() {
-        if (sum == null) {
-            return Integer.MIN_VALUE;
-        } else {
-            return sum.getMemBytes();
-        }
+        return sum == null ? 0 : sum.getMemBytes();
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
index ac932ce..f07059c 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
@@ -18,158 +18,43 @@
 
 package org.apache.kylin.measure.bitmap;
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Iterator;
 
-import org.apache.kylin.common.util.ByteBufferBackedInputStream;
-import org.roaringbitmap.buffer.MutableRoaringBitmap;
-
 /**
- * Created by sunyerui on 15/12/1.
+ * An implementation-agnostic bitmap type.
  */
-public class BitmapCounter implements Comparable<BitmapCounter>, java.io.Serializable {
-
-    private MutableRoaringBitmap bitmap = new MutableRoaringBitmap();
-
-    public BitmapCounter() {
-    }
-
-    public BitmapCounter(BitmapCounter another) {
-        merge(another);
-    }
-
-    public void clear() {
-        bitmap.clear();
-    }
-
-    public BitmapCounter clone() {
-        BitmapCounter newCounter = new BitmapCounter();
-        newCounter.bitmap = bitmap.clone();
-        return newCounter;
-    }
-
-    public void add(int value) {
-        bitmap.add(value);
-    }
-
-    public void add(String value) {
-        if (value == null || value.isEmpty()) {
-            return;
-        }
-        add(Integer.parseInt(value));
-    }
-
-    public void merge(BitmapCounter another) {
-        this.bitmap.or(another.bitmap);
-    }
-
-    public void intersect(BitmapCounter another) {
-        this.bitmap.and(another.bitmap);
-    }
-
-    public long getCount() {
-        return this.bitmap.getCardinality();
-    }
-
-    public int getMemBytes() {
-        return this.bitmap.getSizeInBytes();
-    }
-
-    public Iterator<Integer> iterator() {
-        return bitmap.iterator();
-    }
-
-    public void writeRegisters(ByteBuffer out) throws IOException {
-        ByteArrayOutputStream bos = new ByteArrayOutputStream();
-        DataOutputStream dos = new DataOutputStream(bos);
-        bitmap.runOptimize();
-        bitmap.serialize(dos);
-        dos.close();
-        ByteBuffer bb = ByteBuffer.wrap(bos.toByteArray());
-        out.put(bb);
-    }
-
-    public void readRegisters(ByteBuffer in) throws IOException {
-        try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(in))) {
-            bitmap.deserialize(is);
-        }
-    }
-
-    @Override
-    public String toString() {
-        long count = getCount();
-        if (count <= 10) {
-            return "(" + count + ")" + bitmap.toString();
-        } else {
-            StringBuilder sb = new StringBuilder();
-            sb.append("(").append(count).append("){");
-            int values = 0;
-            for (Integer v : bitmap) {
-                if (values++ < 10) {
-                    sb.append(v).append(",");
-                } else {
-                    sb.append("...");
-                    break;
-                }
-            }
-            sb.append("}");
-            return sb.toString();
-        }
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + bitmap.hashCode();
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        BitmapCounter other = (BitmapCounter) obj;
-        return bitmap.equals(other.bitmap);
-    }
-
-    @Override
-    public int compareTo(BitmapCounter o) {
-        if (o == null)
-            return 1;
-
-        long e1 = this.getCount();
-        long e2 = o.getCount();
-
-        if (e1 == e2)
-            return 0;
-        else if (e1 > e2)
-            return 1;
-        else
-            return -1;
-    }
-
-    public int peekLength(ByteBuffer in) {
-        int mark = in.position();
-        int len;
-
-        MutableRoaringBitmap bitmap = new MutableRoaringBitmap();
-        try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(in))) {
-            bitmap.deserialize(is);
-        } catch (IOException e) {
-            throw new IllegalStateException(e);
-        }
-
-        len = in.position() - mark;
-        in.position(mark);
-        return len;
-    }
+public interface BitmapCounter extends Iterable<Integer> {
+    /**
+     * @return cardinality of the bitmap
+     */
+    long getCount();
+
+    /**
+     * @return estimated memory footprint of this counter
+     */
+    int getMemBytes();
+
+    /**
+     * @return a iterator of the ints stored in this counter.
+     */
+    Iterator<Integer> iterator();
+
+    /**
+     * Serialize this counter. The current counter is not modified.
+     */
+    void serialize(ByteBuffer out) throws IOException;
+
+    /**
+     * Deserialize a counter from its serialized form.
+     * <p> After deserialize, any changes to `in` should not affect the returned counter.
+     */
+    BitmapCounter deserialize(ByteBuffer in) throws IOException;
+
+    /**
+     * @return size of the counter stored in the current position of `in`.
+     * The position field must not be modified.
+     */
+    int peekLength(ByteBuffer in);
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java
index d039b6d..66215e1 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java
@@ -18,35 +18,30 @@
 
 package org.apache.kylin.measure.bitmap;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
- * Created by sunyerui on 15/12/22.
+ * Bitmap-based distinct count UDAF, called by calcite runtime.
  */
 public class BitmapDistinctCountAggFunc {
 
-    private static final Logger logger = LoggerFactory.getLogger(BitmapDistinctCountAggFunc.class);
-
-    public static BitmapCounter init() {
-        return null;
+    public static BitmapAggregator init() {
+        return new BitmapAggregator();
     }
 
-    public static BitmapCounter add(BitmapCounter counter, Object v) {
-        BitmapCounter c = (BitmapCounter) v;
-        if (counter == null) {
-            return new BitmapCounter(c);
-        } else {
-            counter.merge(c);
-            return counter;
-        }
+    public static BitmapAggregator add(BitmapAggregator agg, Object value) {
+        agg.aggregate((BitmapCounter) value);
+        return agg;
     }
 
-    public static BitmapCounter merge(BitmapCounter counter0, Object counter1) {
-        return add(counter0, counter1);
+    public static BitmapAggregator merge(BitmapAggregator agg, Object value) {
+        BitmapAggregator agg2 = (BitmapAggregator) value;
+        if (agg2.getState() == null) {
+            return agg;
+        }
+        return add(agg, agg2.getState());
     }
 
-    public static long result(BitmapCounter counter) {
-        return counter == null ? 0L : counter.getCount();
+    public static long result(BitmapAggregator agg) {
+        BitmapCounter finalState = agg.getState();
+        return finalState == null ? 0 : finalState.getCount();
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapIntersectDistinctCountAggFunc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapIntersectDistinctCountAggFunc.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapIntersectDistinctCountAggFunc.java
index cf42d1b..dcdf945 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapIntersectDistinctCountAggFunc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapIntersectDistinctCountAggFunc.java
@@ -17,9 +17,6 @@
 */
 package org.apache.kylin.measure.bitmap;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -31,10 +28,9 @@ import java.util.Map;
  *          requires an bitmap count distinct measure of uuid, and an dimension of event
  */
 public class BitmapIntersectDistinctCountAggFunc {
-    private static final Logger logger = LoggerFactory.getLogger(BitmapIntersectDistinctCountAggFunc.class);
 
     public static class RetentionPartialResult {
-        Map<Object, BitmapCounter> map;
+        Map<Object, MutableBitmapCounter> map;
         List keyList;
 
         public RetentionPartialResult() {
@@ -45,29 +41,34 @@ public class BitmapIntersectDistinctCountAggFunc {
             if (this.keyList == null) {
                 this.keyList = keyList;
             }
-            BitmapCounter counter = map.get(key);
-            if (counter == null) {
-                counter = new BitmapCounter();
-                map.put(key, counter);
+            if (this.keyList != null && this.keyList.contains(key)) {
+                MutableBitmapCounter counter = map.get(key);
+                if (counter == null) {
+                    counter = new MutableBitmapCounter();
+                    map.put(key, counter);
+                }
+                counter.orWith((ImmutableBitmapCounter) value);
             }
-            counter.merge((BitmapCounter)value);
         }
 
         public long result() {
             if (keyList == null || keyList.isEmpty()) {
                 return 0;
             }
-            BitmapCounter counter = null;
+            // if any specified key not in map, the intersection must be 0
             for (Object key : keyList) {
-                BitmapCounter c = map.get(key);
-                if (c == null) {
-                    // We have a key in filter list but not in map, meaning there's no intersect data
+                if (!map.containsKey(key)) {
                     return 0;
+                }
+            }
+            MutableBitmapCounter counter = null;
+            for (Object key : keyList) {
+                MutableBitmapCounter c = map.get(key);
+                if (counter == null) {
+                    counter = new MutableBitmapCounter();
+                    counter.orWith(c);
                 } else {
-                    if (counter == null) {
-                        counter = c.clone();
-                    }
-                    counter.intersect(c);
+                    counter.andWith(c);
                 }
             }
             return counter.getCount();

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
index 6ad82a1..b6f1975 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
@@ -18,8 +18,8 @@
 
 package org.apache.kylin.measure.bitmap;
 
+import java.util.Arrays;
 import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -38,6 +38,8 @@ import org.apache.kylin.metadata.realization.SQLDigest.SQLCall;
 
 import com.google.common.collect.ImmutableMap;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 /**
  * Created by sunyerui on 15/12/10.
  */
@@ -77,11 +79,14 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
 
     @Override
     public void validate(FunctionDesc functionDesc) throws IllegalArgumentException {
-        if (FUNC_COUNT_DISTINCT.equals(functionDesc.getExpression()) == false)
-            throw new IllegalArgumentException("BitmapMeasureType func is not " + FUNC_COUNT_DISTINCT + " but " + functionDesc.getExpression());
-
-        if (DATATYPE_BITMAP.equals(functionDesc.getReturnDataType().getName()) == false)
-            throw new IllegalArgumentException("BitmapMeasureType datatype is not " + DATATYPE_BITMAP + " but " + functionDesc.getReturnDataType().getName());
+        checkArgument(FUNC_COUNT_DISTINCT.equals(functionDesc.getExpression()),
+                "BitmapMeasureType only support function %s, got %s", FUNC_COUNT_DISTINCT, functionDesc.getExpression());
+        checkArgument(functionDesc.getParameterCount() == 1,
+                "BitmapMeasureType only support 1 parameter, got %d", functionDesc.getParameterCount());
+
+        String returnType = functionDesc.getReturnDataType().getName();
+        checkArgument(DATATYPE_BITMAP.equals(returnType),
+                "BitmapMeasureType's return type must be %s, got %s", DATATYPE_BITMAP, returnType);
     }
 
     @Override
@@ -92,24 +97,29 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
     @Override
     public MeasureIngester<BitmapCounter> newIngester() {
         return new MeasureIngester<BitmapCounter>() {
-            BitmapCounter current = new BitmapCounter();
+            MutableBitmapCounter current = new MutableBitmapCounter();
 
             @Override
             public BitmapCounter valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
-                BitmapCounter bitmap = current;
+                checkArgument(values.length == 1, "expect 1 value, got %s", Arrays.toString(values));
+
+                MutableBitmapCounter bitmap = current;
                 bitmap.clear();
+
+                if (values[0] == null) {
+                    return bitmap;
+                }
+
+                int id;
                 if (needDictionaryColumn(measureDesc.getFunction())) {
                     TblColRef literalCol = measureDesc.getFunction().getParameter().getColRefs().get(0);
                     Dictionary<String> dictionary = dictionaryMap.get(literalCol);
-                    if (values != null && values.length > 0 && values[0] != null) {
-                        int id = dictionary.getIdFromValue(values[0]);
-                        bitmap.add(id);
-                    }
+                    id = dictionary.getIdFromValue(values[0]);
                 } else {
-                    for (String value : values) {
-                        bitmap.add(value);
-                    }
+                    id = Integer.parseInt(values[0]);
                 }
+
+                bitmap.add(id);
                 return bitmap;
             }
 
@@ -122,11 +132,9 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
                 Dictionary<String> sourceDict = oldDicts.get(colRef);
                 Dictionary<String> mergedDict = newDicts.get(colRef);
 
-                BitmapCounter retValue = new BitmapCounter();
+                MutableBitmapCounter retValue = new MutableBitmapCounter();
                 byte[] literal = new byte[sourceDict.getSizeOfValue()];
-                Iterator<Integer> iterator = value.iterator();
-                while (iterator.hasNext()) {
-                    int id = iterator.next();
+                for (int id : value) {
                     int newId;
                     int size = sourceDict.getValueBytesFromId(id, literal, 0);
                     if (size < 0) {
@@ -141,7 +149,7 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
 
             @Override
             public void reset() {
-                current = new BitmapCounter();
+                current = new MutableBitmapCounter();
             }
         };
     }
@@ -174,8 +182,8 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
         return true;
     }
 
-    static final Map<String, Class<?>> UDAF_MAP = ImmutableMap.<String, Class<?>> of(//
-            FUNC_COUNT_DISTINCT, BitmapDistinctCountAggFunc.class, //
+    static final Map<String, Class<?>> UDAF_MAP = ImmutableMap.of(
+            FUNC_COUNT_DISTINCT, BitmapDistinctCountAggFunc.class,
             FUNC_INTERSECT_COUNT_DISTINCT, BitmapIntersectDistinctCountAggFunc.class);
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
index 089d18c..8b15d1c 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
@@ -18,54 +18,41 @@
 
 package org.apache.kylin.measure.bitmap;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
 import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 
-/**
- * Created by sunyerui on 15/12/1.
- */
-public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> {
+import java.io.IOException;
+import java.nio.ByteBuffer;
 
-    private ThreadLocal<BitmapCounter> current = new ThreadLocal<>();
+public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> {
+    private static final BitmapCounter DELEGATE = new ImmutableBitmapCounter();
 
+    // called by reflection
     public BitmapSerializer(DataType type) {
     }
 
     @Override
     public void serialize(BitmapCounter value, ByteBuffer out) {
         try {
-            value.writeRegisters(out);
+            value.serialize(out);
         } catch (IOException e) {
             throw new RuntimeException(e);
         }
     }
 
-    private BitmapCounter current() {
-        BitmapCounter counter = current.get();
-        if (counter == null) {
-            counter = new BitmapCounter();
-            current.set(counter);
-        }
-        return counter;
-    }
-
     @Override
     public BitmapCounter deserialize(ByteBuffer in) {
-        BitmapCounter counter = current();
+
         try {
-            counter.readRegisters(in);
+            return DELEGATE.deserialize(in);
         } catch (IOException e) {
             throw new RuntimeException(e);
         }
-        return counter;
     }
 
     @Override
     public int peekLength(ByteBuffer in) {
-        return current().peekLength(in);
+        return DELEGATE.peekLength(in);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/ImmutableBitmapCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/ImmutableBitmapCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/ImmutableBitmapCounter.java
new file mode 100644
index 0000000..5c39a4a
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/ImmutableBitmapCounter.java
@@ -0,0 +1,112 @@
+/*
+ * 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.measure.bitmap;
+
+import org.apache.kylin.common.util.ByteBufferOutputStream;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+
+/**
+ * A thin wrapper around {@link ImmutableRoaringBitmap}.
+ */
+public class ImmutableBitmapCounter implements BitmapCounter {
+
+    protected ImmutableRoaringBitmap bitmap;
+
+    public ImmutableBitmapCounter() {
+        this(ImmutableRoaringBitmap.bitmapOf());
+    }
+
+    public ImmutableBitmapCounter(ImmutableRoaringBitmap bitmap) {
+        this.bitmap = bitmap;
+    }
+
+    @Override
+    public long getCount() {
+        return bitmap.getCardinality();
+    }
+
+    @Override
+    public int getMemBytes() {
+        return bitmap.getSizeInBytes();
+    }
+
+    @Override
+    public Iterator<Integer> iterator() {
+        return bitmap.iterator();
+    }
+
+    @Override
+    public void serialize(ByteBuffer out) throws IOException {
+        if (out.remaining() < bitmap.serializedSizeInBytes()) {
+            throw new BufferOverflowException();
+        }
+        bitmap.serialize(new DataOutputStream(new ByteBufferOutputStream(out)));
+    }
+
+    @Override
+    public BitmapCounter deserialize(ByteBuffer in) throws IOException {
+        int size = peekLength(in);
+        // make a copy of the content to be safe
+        byte[] dst = new byte[size];
+        in.get(dst);
+
+        // just map the buffer, faster than deserialize
+        ImmutableRoaringBitmap bitmap = new ImmutableRoaringBitmap(ByteBuffer.wrap(dst));
+        return new ImmutableBitmapCounter(bitmap);
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        // only look at the metadata of the bitmap, no deserialization happens
+        ImmutableRoaringBitmap bitmap = new ImmutableRoaringBitmap(in);
+        return bitmap.serializedSizeInBytes();
+    }
+
+    /**
+     * Copies the content of this counter to a counter that can be modified.
+     * @return a mutable counter
+     */
+    public MutableBitmapCounter toMutable() {
+        MutableBitmapCounter mutable = new MutableBitmapCounter();
+        mutable.orWith(this);
+        return mutable;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return (obj instanceof ImmutableBitmapCounter) &&
+                bitmap.equals(((ImmutableBitmapCounter) obj).bitmap);
+    }
+
+    @Override
+    public int hashCode() {
+        return bitmap.hashCode();
+    }
+
+    @Override
+    public String toString() {
+        return "BitmapCounter[" + getCount() + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/MutableBitmapCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/MutableBitmapCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/MutableBitmapCounter.java
new file mode 100644
index 0000000..af01790
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/MutableBitmapCounter.java
@@ -0,0 +1,60 @@
+/*
+ * 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.measure.bitmap;
+
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * A thin wrapper around {@link MutableRoaringBitmap}.
+ */
+public class MutableBitmapCounter extends ImmutableBitmapCounter {
+
+    public MutableBitmapCounter() {
+        super(new MutableRoaringBitmap());
+    }
+
+    private MutableRoaringBitmap getBitmap() {
+        return (MutableRoaringBitmap) bitmap;
+    }
+
+    public void clear() {
+        getBitmap().clear();
+    }
+
+    public void add(int value) {
+        getBitmap().add(value);
+    }
+
+    public void orWith(ImmutableBitmapCounter another) {
+        getBitmap().or(another.bitmap);
+    }
+
+    public void andWith(ImmutableBitmapCounter another) {
+        getBitmap().and(another.bitmap);
+    }
+
+    @Override
+    public void serialize(ByteBuffer out) throws IOException {
+        getBitmap().runOptimize();
+        super.serialize(out);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java
index 3d48ac2..39921c2 100644
--- a/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java
@@ -33,7 +33,7 @@ import org.apache.kylin.measure.basic.LongMaxAggregator;
 import org.apache.kylin.measure.basic.LongMinAggregator;
 import org.apache.kylin.measure.basic.LongSumAggregator;
 import org.apache.kylin.measure.bitmap.BitmapAggregator;
-import org.apache.kylin.measure.bitmap.BitmapCounter;
+import org.apache.kylin.measure.bitmap.MutableBitmapCounter;
 import org.apache.kylin.measure.extendedcolumn.ExtendedColumnMeasureType;
 import org.apache.kylin.measure.hllc.HLLCAggregator;
 import org.apache.kylin.measure.hllc.HLLCounter;
@@ -104,7 +104,7 @@ public class AggregatorMemEstimateTest extends LocalFileMetadataTestCase {
         hllcAggregator.aggregate(new HLLCounter(14));
 
         BitmapAggregator bitmapAggregator = new BitmapAggregator();
-        BitmapCounter bitmapCounter = new BitmapCounter();
+        MutableBitmapCounter bitmapCounter = new MutableBitmapCounter();
         for (int i = 4000; i <= 100000; i += 2) {
             bitmapCounter.add(i);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapAggregatorTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapAggregatorTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapAggregatorTest.java
index e216d0b..03eb53a 100644
--- a/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapAggregatorTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapAggregatorTest.java
@@ -18,39 +18,29 @@
 
 package org.apache.kylin.measure.bitmap;
 
+import org.junit.Test;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
-import org.junit.Test;
-
-/**
- * Created by sunyerui on 15/12/31.
- */
 public class BitmapAggregatorTest {
 
     @Test
     public void testAggregator() {
-        BitmapCounter counter = new BitmapCounter();
-        counter.add(1);
-        counter.add(3333);
-        counter.add("123");
-        counter.add(123);
-        assertEquals(3, counter.getCount());
-
-        BitmapCounter counter2 = new BitmapCounter();
-        counter2.add("23456");
-        counter2.add(12273456);
-        counter2.add("4258");
-        counter2.add(123);
-        assertEquals(4, counter2.getCount());
-
         BitmapAggregator aggregator = new BitmapAggregator();
-        assertNull(aggregator.getState());
-        assertEquals(Integer.MIN_VALUE, aggregator.getMemBytesEstimate());
+        assertNull(null, aggregator.getState());
+
+        aggregator.aggregate(new ImmutableBitmapCounter(
+                ImmutableRoaringBitmap.bitmapOf(10, 20, 30, 40)
+        ));
+        assertEquals(4, aggregator.getState().getCount());
+
+        aggregator.aggregate(new ImmutableBitmapCounter(
+                ImmutableRoaringBitmap.bitmapOf(25, 30, 35, 40, 45)
+        ));
+        assertEquals(7, aggregator.getState().getCount());
 
-        aggregator.aggregate(counter);
-        aggregator.aggregate(counter2);
-        assertEquals(6, aggregator.getState().getCount());
         aggregator.reset();
         assertNull(aggregator.getState());
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapCounterTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapCounterTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapCounterTest.java
index c9c1b51..70e4ecc 100644
--- a/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapCounterTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapCounterTest.java
@@ -18,58 +18,66 @@
 
 package org.apache.kylin.measure.bitmap;
 
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
 import org.junit.Test;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
-/**
- * Created by sunyerui on 15/12/31.
- */
 public class BitmapCounterTest {
 
     @Test
-    public void testAddAndMergeValues() {
-        BitmapCounter counter = new BitmapCounter();
-        counter.add(1);
-        counter.add(3333);
-        counter.add("123");
-        counter.add(123);
-        assertEquals(3, counter.getCount());
+    public void testBitmapCounter() {
+        ImmutableRoaringBitmap bitmap = ImmutableRoaringBitmap.bitmapOf(10, 20, 30, 1000);
+        ImmutableBitmapCounter counter = new ImmutableBitmapCounter(bitmap);
+        assertEquals(4, counter.getCount());
+        assertTrue(counter.getMemBytes() > 0);
 
-        BitmapCounter counter2 = new BitmapCounter();
-        counter2.add("23456");
-        counter2.add(12273456);
-        counter2.add("4258");
-        counter2.add(123);
-        counter2.add(-2147483648);
-        counter2.add(-2);
-        assertEquals(6, counter2.getCount());
+        MutableBitmapCounter counter2 = new MutableBitmapCounter();
+        assertEquals(0, counter2.getCount());
+        counter2.add(10);
+        counter2.add(30);
+        counter2.add(40);
+        counter2.add(2000);
+        assertEquals(4, counter2.getCount());
 
-        counter.merge(counter2);
-        assertEquals(8, counter.getCount());
-        System.out.print("counter size: " + counter.getMemBytes() + ", counter2 size: " + counter2.getMemBytes());
+        counter2.orWith(counter);
+        assertEquals(4, counter.getCount());
+        assertEquals(6, counter2.getCount());  // in-place change
+
+        int i = 0;
+        int[] values = new int[(int) counter2.getCount()];
+        for (int value : counter2) {
+            values[i++] = value;
+        }
+        assertArrayEquals(new int[]{10, 20, 30, 40, 1000, 2000}, values);
+
+        counter2.clear();
+        assertEquals(0, counter2.getCount());
     }
 
     @Test
-    public void testSerDeCounter() throws IOException {
-        BitmapCounter counter = new BitmapCounter();
-        for (int i = 1; i < 1000; i++) {
-            counter.add(i);
-        }
-        ByteBuffer buffer = ByteBuffer.allocate(10 * 1024 * 1024);
-        counter.writeRegisters(buffer);
-        int len = buffer.position();
+    public void testToMutableBitmapCounter() {
+        ImmutableRoaringBitmap bitmap = ImmutableRoaringBitmap.bitmapOf(10, 20, 30, 1000);
+        ImmutableBitmapCounter immutable = new ImmutableBitmapCounter(bitmap);
+        MutableBitmapCounter mutable = new MutableBitmapCounter();
+        mutable.orWith(immutable);
+
+        assertEquals(4, immutable.getCount());
+        assertEquals(4, mutable.getCount());
+        assertTrue(immutable.equals(mutable));
+        assertTrue(mutable.equals(immutable));
 
-        buffer.position(0);
-        assertEquals(len, counter.peekLength(buffer));
-        assertEquals(0, buffer.position());
+        MutableBitmapCounter newCounter = immutable.toMutable();
+        newCounter.add(40);
+        assertEquals(4, immutable.getCount());
+        assertEquals(5, newCounter.getCount());
 
-        BitmapCounter counter2 = new BitmapCounter();
-        counter2.readRegisters(buffer);
-        assertEquals(999, counter2.getCount());
+        newCounter = mutable.toMutable();
+        newCounter.add(40);
+        assertEquals(4, mutable.getCount());
+        assertEquals(5, newCounter.getCount());
     }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/d19533c4/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapSerializerTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapSerializerTest.java
index 41efb2c..71fcae6 100644
--- a/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapSerializerTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapSerializerTest.java
@@ -18,19 +18,20 @@
 
 package org.apache.kylin.measure.bitmap;
 
-import static org.junit.Assert.assertEquals;
-
-import java.nio.ByteBuffer;
-
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.metadata.datatype.DataType;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
-/**
- * Created by sunyerui on 15/12/31.
- */
 public class BitmapSerializerTest extends LocalFileMetadataTestCase {
     @BeforeClass
     public static void setUp() throws Exception {
@@ -43,27 +44,36 @@ public class BitmapSerializerTest extends LocalFileMetadataTestCase {
     }
 
     @Test
-    public void testSerDeCounter() {
-        BitmapCounter counter = new BitmapCounter();
-        counter.add(1);
-        counter.add(3333);
-        counter.add("123");
-        counter.add(123);
-        assertEquals(3, counter.getCount());
-
-        ByteBuffer buffer = ByteBuffer.allocate(10 * 1024 * 1024);
+    public void testBitmapSerDe() {
         BitmapSerializer serializer = new BitmapSerializer(DataType.ANY);
+
+        ImmutableRoaringBitmap bitmap = ImmutableRoaringBitmap.bitmapOf(1, 1234, 5678, 100000);
+        ImmutableBitmapCounter counter = new ImmutableBitmapCounter(bitmap);
+
+        ByteBuffer buffer = ByteBuffer.allocate(1024 * 1024);
         serializer.serialize(counter, buffer);
-        int len = buffer.position();
+        int size = buffer.position();
+        buffer.flip();
+
+        assertEquals(size, serializer.peekLength(buffer));
+        assertEquals(0, buffer.position()); // peek doesn't change buffer
+
+        BitmapCounter counter2 = serializer.deserialize(buffer);
+        assertEquals(size, buffer.position()); // deserialize advance positions to next record
+        assertEquals(4, counter2.getCount());
 
-        buffer.position(0);
-        BitmapSerializer deSerializer = new BitmapSerializer(DataType.ANY);
-        BitmapCounter counter2 = deSerializer.deserialize(buffer);
-        assertEquals(3, counter2.getCount());
+        buffer.flip();
+        for (int i = 0; i < size; i++) {
+            buffer.put((byte) 0); // clear buffer content
+        }
+        assertEquals(4, counter2.getCount());
 
-        buffer.position(0);
-        assertEquals(len, deSerializer.peekLength(buffer));
-        assertEquals(8 * 1024 * 1024, deSerializer.maxLength());
-        System.out.println("counter size " + deSerializer.getStorageBytesEstimate());
+        buffer = ByteBuffer.allocate(size - 1);
+        try {
+            serializer.serialize(counter, buffer);
+            Assert.fail();
+        } catch (Exception e) {
+            assertTrue(e instanceof BufferOverflowException);
+        }
     }
 }
\ No newline at end of file


[12/21] kylin git commit: KYLIN-2393 Add "hive.auto.convert.join" and "hive.stats.autogather" to kylin_hive_conf.xml

Posted by li...@apache.org.
KYLIN-2393 Add "hive.auto.convert.join" and "hive.stats.autogather" to kylin_hive_conf.xml


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

Branch: refs/heads/master-cdh5.7
Commit: 61d43028241b8043609bf9e36bd7c35d558c4da2
Parents: e79a55d
Author: shaofengshi <sh...@apache.org>
Authored: Sun Jan 15 22:14:16 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sun Jan 15 22:14:16 2017 +0800

----------------------------------------------------------------------
 build/conf/kylin_hive_conf.xml | 15 ++++++++++++++-
 1 file changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/61d43028/build/conf/kylin_hive_conf.xml
----------------------------------------------------------------------
diff --git a/build/conf/kylin_hive_conf.xml b/build/conf/kylin_hive_conf.xml
index c201240..2fc6dab 100644
--- a/build/conf/kylin_hive_conf.xml
+++ b/build/conf/kylin_hive_conf.xml
@@ -24,7 +24,13 @@
     <property>
         <name>hive.exec.compress.output</name>
         <value>true</value>
-        <description>enable compress</description>
+        <description>Enable compress</description>
+    </property>
+
+    <property>
+        <name>hive.auto.convert.join</name>
+        <value>true</value>
+        <description>Enables the optimization about converting common join into mapjoin</description>
     </property>
 
     <property>
@@ -75,4 +81,11 @@
             the configured value. No limits if set to -1.
         </description>
     </property>
+
+    <property>
+        <name>hive.stats.autogather</name>
+        <value>true</value>
+        <description>Collect statistics for newly created intermediate table</description>
+    </property>
+
 </configuration>


[04/21] kylin git commit: Minor fix, enable config to overwrite internal StorageEngine types

Posted by li...@apache.org.
Minor fix, enable config to overwrite internal StorageEngine types

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/362acd9a
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/362acd9a
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/362acd9a

Branch: refs/heads/master-cdh5.7
Commit: 362acd9ac215da2403af1d682d66a8dd097ffabe
Parents: 0f36b9b
Author: Yifan Zhang <ev...@gmail.com>
Authored: Fri Jan 13 10:51:13 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Fri Jan 13 11:13:48 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/common/KylinConfigBase.java   | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/362acd9a/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 04051b4..494c641 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
@@ -412,9 +412,10 @@ abstract public class KylinConfigBase implements Serializable {
     }
 
     public Map<Integer, String> getSchedulers() {
-        Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.job.scheduler.provider."));
+        Map<Integer, String> r = Maps.newLinkedHashMap();
         r.put(0, "org.apache.kylin.job.impl.threadpool.DefaultScheduler");
         r.put(2, "org.apache.kylin.job.impl.threadpool.DistributedScheduler");
+        r.putAll(convertKeyToInteger(getPropertiesByPrefix("kylin.job.scheduler.provider.")));
         return r;
     }
 
@@ -427,10 +428,11 @@ abstract public class KylinConfigBase implements Serializable {
     // ============================================================================
 
     public Map<Integer, String> getSourceEngines() {
-        Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.source.provider."));
+        Map<Integer, String> r = Maps.newLinkedHashMap();
         // ref constants in ISourceAware
         r.put(0, "org.apache.kylin.source.hive.HiveSource");
         r.put(1, "org.apache.kylin.source.kafka.KafkaSource");
+        r.putAll(convertKeyToInteger(getPropertiesByPrefix("kylin.source.provider.")));
         return r;
     }
 
@@ -503,11 +505,12 @@ abstract public class KylinConfigBase implements Serializable {
     // ============================================================================
 
     public Map<Integer, String> getStorageEngines() {
-        Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.storage.provider."));
+        Map<Integer, String> r = Maps.newLinkedHashMap();
         // ref constants in IStorageAware
         r.put(0, "org.apache.kylin.storage.hbase.HBaseStorage");
         r.put(1, "org.apache.kylin.storage.hybrid.HybridStorage");
         r.put(2, "org.apache.kylin.storage.hbase.HBaseStorage");
+        r.putAll(convertKeyToInteger(getPropertiesByPrefix("kylin.storage.provider.")));
         return r;
     }
 
@@ -658,11 +661,12 @@ abstract public class KylinConfigBase implements Serializable {
     // ============================================================================
 
     public Map<Integer, String> getJobEngines() {
-        Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.engine.provider."));
+        Map<Integer, String> r = Maps.newLinkedHashMap();
         // ref constants in IEngineAware
         r.put(0, "org.apache.kylin.engine.mr.MRBatchCubingEngine");
         r.put(2, "org.apache.kylin.engine.mr.MRBatchCubingEngine2");
         r.put(4, "org.apache.kylin.engine.spark.SparkBatchCubingEngine2");
+        r.putAll(convertKeyToInteger(getPropertiesByPrefix("kylin.engine.provider.")));
         return r;
     }
 


[05/21] kylin git commit: KYLIN-2386 Revert KYLIN-2349 and KYLIN-2353

Posted by li...@apache.org.
KYLIN-2386 Revert KYLIN-2349 and KYLIN-2353


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

Branch: refs/heads/master-cdh5.7
Commit: 4b977215186281908a8c29741128242146a2b934
Parents: 362acd9
Author: gaodayue <ga...@meituan.com>
Authored: Fri Jan 13 16:01:00 2017 +0800
Committer: gaodayue <ga...@meituan.com>
Committed: Fri Jan 13 19:20:46 2017 +0800

----------------------------------------------------------------------
 .../kylin/measure/bitmap/BitmapCounter.java     | 144 ++++++-------------
 .../bitmap/BitmapDistinctCountAggFunc.java      |   2 +-
 .../kylin/measure/bitmap/BitmapSerializer.java  |  15 +-
 3 files changed, 60 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/4b977215/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
index 19d7f5d..ac932ce 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
@@ -34,9 +34,6 @@ import org.roaringbitmap.buffer.MutableRoaringBitmap;
 public class BitmapCounter implements Comparable<BitmapCounter>, java.io.Serializable {
 
     private MutableRoaringBitmap bitmap = new MutableRoaringBitmap();
-    private final int VERSION = 2;
-    private Integer count;
-    private ByteBuffer buffer;
 
     public BitmapCounter() {
     }
@@ -45,43 +42,18 @@ public class BitmapCounter implements Comparable<BitmapCounter>, java.io.Seriali
         merge(another);
     }
 
-    private MutableRoaringBitmap getBitmap() {
-        if (!bitmap.isEmpty()) {
-            return bitmap;
-        }
-
-        if (buffer != null) {
-            @SuppressWarnings("unused")
-            int version = buffer.getInt();
-            @SuppressWarnings("unused")
-            int size = buffer.getInt();
-            count = buffer.getInt();
-
-            try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(buffer))) {
-                bitmap.deserialize(is);
-            } catch (IOException e) {
-                throw new RuntimeException("deserialize bitmap failed!");
-            }
-
-            buffer = null;
-        }
-
-        return bitmap;
-    }
-
     public void clear() {
-        getBitmap().clear();
+        bitmap.clear();
     }
 
     public BitmapCounter clone() {
         BitmapCounter newCounter = new BitmapCounter();
-        newCounter.bitmap = getBitmap().clone();
+        newCounter.bitmap = bitmap.clone();
         return newCounter;
     }
 
     public void add(int value) {
-        getBitmap().add(value);
-        count = null;
+        bitmap.add(value);
     }
 
     public void add(String value) {
@@ -92,107 +64,68 @@ public class BitmapCounter implements Comparable<BitmapCounter>, java.io.Seriali
     }
 
     public void merge(BitmapCounter another) {
-        getBitmap().or(another.getBitmap());
-        count = null;
+        this.bitmap.or(another.bitmap);
     }
 
     public void intersect(BitmapCounter another) {
-        getBitmap().and(another.getBitmap());
-        count = null;
+        this.bitmap.and(another.bitmap);
     }
 
-    public int getCount() {
-        if (count != null) {
-            return count;
-        }
-
-        return getBitmap().getCardinality();
+    public long getCount() {
+        return this.bitmap.getCardinality();
     }
 
     public int getMemBytes() {
-        return getBitmap().getSizeInBytes();
+        return this.bitmap.getSizeInBytes();
     }
 
     public Iterator<Integer> iterator() {
-        return getBitmap().iterator();
+        return bitmap.iterator();
     }
 
     public void writeRegisters(ByteBuffer out) throws IOException {
         ByteArrayOutputStream bos = new ByteArrayOutputStream();
         DataOutputStream dos = new DataOutputStream(bos);
-        MutableRoaringBitmap bitmap = getBitmap();
         bitmap.runOptimize();
         bitmap.serialize(dos);
         dos.close();
         ByteBuffer bb = ByteBuffer.wrap(bos.toByteArray());
-
-        out.putInt(VERSION);
-        out.putInt(bos.size() + 4 + 4 + 4);
-        out.putInt(getCount());
         out.put(bb);
     }
 
     public void readRegisters(ByteBuffer in) throws IOException {
-        int mark = in.position();
-        int version = in.getInt();
-
-        // keep forward compatibility
-        if (version == VERSION) {
-            int size = in.getInt();
-            count = in.getInt();
-            in.position(mark);
-            buffer = cloneBuffer(in, size);
-        } else {
-            in.position(mark);
-            try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(in))) {
-                getBitmap().deserialize(is);
-            }
+        try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(in))) {
+            bitmap.deserialize(is);
         }
     }
 
-    private ByteBuffer cloneBuffer(ByteBuffer src, int size) throws IOException {
-        int mark = src.position();
-        int limit = src.limit();
-
-        src.limit(mark + size);
-        ByteBuffer clone = ByteBuffer.allocate(size);
-        clone.put(src.slice());
-        clone.flip();
-
-        src.position(mark + size);
-        src.limit(limit);
-
-        return clone;
-    }
-
-    public int peekLength(ByteBuffer in) {
-        int mark = in.position();
-        int len;
-        int version = in.getInt();
-
-        // keep forward compatibility
-        if (version == VERSION) {
-            len = in.getInt();
+    @Override
+    public String toString() {
+        long count = getCount();
+        if (count <= 10) {
+            return "(" + count + ")" + bitmap.toString();
         } else {
-            in.position(mark);
-            try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(in))) {
-                MutableRoaringBitmap bitmap = new MutableRoaringBitmap();
-                bitmap.deserialize(is);
-                len = in.position() - mark;
-            } catch (IOException e) {
-                throw new IllegalStateException(e);
+            StringBuilder sb = new StringBuilder();
+            sb.append("(").append(count).append("){");
+            int values = 0;
+            for (Integer v : bitmap) {
+                if (values++ < 10) {
+                    sb.append(v).append(",");
+                } else {
+                    sb.append("...");
+                    break;
+                }
             }
+            sb.append("}");
+            return sb.toString();
         }
-
-        in.position(mark);
-        return len;
     }
 
     @Override
     public int hashCode() {
         final int prime = 31;
         int result = 1;
-        result = prime * result + getBitmap().hashCode();
+        result = prime * result + bitmap.hashCode();
         return result;
     }
 
@@ -205,7 +138,7 @@ public class BitmapCounter implements Comparable<BitmapCounter>, java.io.Seriali
         if (getClass() != obj.getClass())
             return false;
         BitmapCounter other = (BitmapCounter) obj;
-        return getBitmap().equals(other.getBitmap());
+        return bitmap.equals(other.bitmap);
     }
 
     @Override
@@ -223,5 +156,20 @@ public class BitmapCounter implements Comparable<BitmapCounter>, java.io.Seriali
         else
             return -1;
     }
-    
+
+    public int peekLength(ByteBuffer in) {
+        int mark = in.position();
+        int len;
+
+        MutableRoaringBitmap bitmap = new MutableRoaringBitmap();
+        try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(in))) {
+            bitmap.deserialize(is);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+
+        len = in.position() - mark;
+        in.position(mark);
+        return len;
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/4b977215/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java
index 3a1a800..d039b6d 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java
@@ -35,7 +35,7 @@ public class BitmapDistinctCountAggFunc {
     public static BitmapCounter add(BitmapCounter counter, Object v) {
         BitmapCounter c = (BitmapCounter) v;
         if (counter == null) {
-            return c;
+            return new BitmapCounter(c);
         } else {
             counter.merge(c);
             return counter;

http://git-wip-us.apache.org/repos/asf/kylin/blob/4b977215/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
index 4890295..089d18c 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
@@ -29,6 +29,8 @@ import org.apache.kylin.metadata.datatype.DataTypeSerializer;
  */
 public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> {
 
+    private ThreadLocal<BitmapCounter> current = new ThreadLocal<>();
+
     public BitmapSerializer(DataType type) {
     }
 
@@ -41,9 +43,18 @@ public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> {
         }
     }
 
+    private BitmapCounter current() {
+        BitmapCounter counter = current.get();
+        if (counter == null) {
+            counter = new BitmapCounter();
+            current.set(counter);
+        }
+        return counter;
+    }
+
     @Override
     public BitmapCounter deserialize(ByteBuffer in) {
-        BitmapCounter counter = new BitmapCounter();
+        BitmapCounter counter = current();
         try {
             counter.readRegisters(in);
         } catch (IOException e) {
@@ -54,7 +65,7 @@ public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> {
 
     @Override
     public int peekLength(ByteBuffer in) {
-        return new BitmapCounter().peekLength(in);
+        return current().peekLength(in);
     }
 
     @Override


[11/21] kylin git commit: KYLIN-2304 minor, fix variable name error

Posted by li...@apache.org.
KYLIN-2304 minor, fix variable name error


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

Branch: refs/heads/master-cdh5.7
Commit: e79a55d94c6c45c315f991b8a8e6f606a6b2c25e
Parents: 8d374f8
Author: kangkaisen <ka...@163.com>
Authored: Sat Jan 14 19:31:55 2017 +0800
Committer: kangkaisen <ka...@163.com>
Committed: Sat Jan 14 19:31:55 2017 +0800

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


http://git-wip-us.apache.org/repos/asf/kylin/blob/e79a55d9/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
index 80403cc..b797167 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
@@ -1191,7 +1191,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
 
         // init new AppendTrieDictionary
         AppendTrieDictionary newDict = new AppendTrieDictionary();
-        newDict.initParams(baseDir.replaceFirst(srcConfig.getHdfsWorkingDirectory(), dstConfig.getHdfsWorkingDirectory()), baseId, maxId, maxValueLength, nValues, bytesConverter);
+        newDict.initParams(baseDir.replaceFirst(srcConfig.getHdfsWorkingDirectory(), dstConfig.getHdfsWorkingDirectory()), baseId, maxId, maxValueLength, nValues, bytesConvert);
         newDict.initDictSliceMap((CachedTreeMap) dictSliceMap);
 
         return newDict;


[15/21] kylin git commit: KYLIN-2396 percentile pre-aggregation support

Posted by li...@apache.org.
KYLIN-2396 percentile pre-aggregation support


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

Branch: refs/heads/master-cdh5.7
Commit: 95d4412194fb6fc08c99600d162826d9350a7a80
Parents: 4625b05
Author: lidongsjtu <li...@apache.org>
Authored: Mon Jan 16 11:12:36 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Mon Jan 16 17:25:28 2017 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/util/MathUtil.java  |  32 ++++++
 core-metadata/pom.xml                           |   4 +
 .../kylin/measure/MeasureTypeFactory.java       |   4 +-
 .../measure/percentile/PercentileAggFunc.java   |  44 ++++++++
 .../percentile/PercentileAggregator.java        |  64 ++++++++++++
 .../measure/percentile/PercentileContUdf.java   |  37 +++++++
 .../measure/percentile/PercentileCounter.java   |  97 ++++++++++++++++++
 .../percentile/PercentileMeasureType.java       | 102 +++++++++++++++++++
 .../percentile/PercentileSerializer.java        |  71 +++++++++++++
 .../percentile/PercentileAggregatorTest.java    |  55 ++++++++++
 .../percentile/PercentileCounterTest.java       |  79 ++++++++++++++
 .../percentile/PercentileSerializerTest.java    |  68 +++++++++++++
 .../kylin/measure/percentile/TDigestTest.java   |  57 +++++++++++
 .../localmeta/cube_desc/ci_inner_join_cube.json |  12 ++-
 .../localmeta/cube_desc/ci_left_join_cube.json  |  12 ++-
 .../apache/kylin/query/ITKylinQueryTest.java    |  18 ++--
 .../resources/query/sql_percentile/query01.sql  |   2 +
 pom.xml                                         |   6 ++
 storage-hbase/pom.xml                           |   1 +
 19 files changed, 755 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-common/src/main/java/org/apache/kylin/common/util/MathUtil.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/MathUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/MathUtil.java
new file mode 100644
index 0000000..ae674c6
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/util/MathUtil.java
@@ -0,0 +1,32 @@
+/*
+ * 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.util;
+
+import java.util.List;
+
+public class MathUtil {
+    public static double findMedianInSortedList(List<Double> m) {
+        int middle = m.size() / 2;
+        if (m.size() % 2 == 1) {
+            return m.get(middle);
+        } else {
+            return (m.get(middle - 1) + m.get(middle)) / 2.0;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-metadata/pom.xml
----------------------------------------------------------------------
diff --git a/core-metadata/pom.xml b/core-metadata/pom.xml
index b3fe885..87c4438 100644
--- a/core-metadata/pom.xml
+++ b/core-metadata/pom.xml
@@ -47,6 +47,10 @@
             <groupId>org.roaringbitmap</groupId>
             <artifactId>RoaringBitmap</artifactId>
         </dependency>
+        <dependency>
+            <groupId>com.tdunning</groupId>
+            <artifactId>t-digest</artifactId>
+        </dependency>
 
         <!-- Env & Test -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java
index 8e58858..5d0e007 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java
@@ -28,6 +28,7 @@ import org.apache.kylin.measure.bitmap.BitmapMeasureType;
 import org.apache.kylin.measure.dim.DimCountDistinctMeasureType;
 import org.apache.kylin.measure.extendedcolumn.ExtendedColumnMeasureType;
 import org.apache.kylin.measure.hllc.HLLCMeasureType;
+import org.apache.kylin.measure.percentile.PercentileMeasureType;
 import org.apache.kylin.measure.raw.RawMeasureType;
 import org.apache.kylin.measure.topn.TopNMeasureType;
 import org.apache.kylin.metadata.datatype.DataType;
@@ -109,6 +110,7 @@ abstract public class MeasureTypeFactory<T> {
         factoryInsts.add(new TopNMeasureType.Factory());
         factoryInsts.add(new RawMeasureType.Factory());
         factoryInsts.add(new ExtendedColumnMeasureType.Factory());
+        factoryInsts.add(new PercentileMeasureType.Factory());
 
         logger.info("Checking custom measure types from kylin config");
 
@@ -143,7 +145,7 @@ abstract public class MeasureTypeFactory<T> {
             List<MeasureTypeFactory<?>> list = factories.get(funcName);
             if (list == null)
                 list = Lists.newArrayListWithCapacity(2);
-                factories.put(funcName, list);
+            factories.put(funcName, list);
             list.add(factory);
         }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileAggFunc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileAggFunc.java b/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileAggFunc.java
new file mode 100644
index 0000000..ad02019
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileAggFunc.java
@@ -0,0 +1,44 @@
+/*
+ * 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.measure.percentile;
+
+public class PercentileAggFunc {
+    public static PercentileCounter init() {
+        return null;
+    }
+
+    public static PercentileCounter add(PercentileCounter counter, Object v, Object r) {
+        PercentileCounter c = (PercentileCounter) v;
+        Number n = (Number) r;
+        if (counter == null) {
+            counter = new PercentileCounter(c.compression, n.doubleValue());
+        }
+        counter.merge(c);
+        return counter;
+    }
+
+    public static PercentileCounter merge(PercentileCounter counter0, PercentileCounter counter1) {
+        counter0.merge(counter1);
+        return counter0;
+    }
+
+    public static double result(PercentileCounter counter) {
+        return counter == null ? 0L : counter.getResultEstimate();
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileAggregator.java
new file mode 100644
index 0000000..d6b93eb
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileAggregator.java
@@ -0,0 +1,64 @@
+/*
+ * 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.measure.percentile;
+
+import org.apache.kylin.measure.MeasureAggregator;
+
+public class PercentileAggregator extends MeasureAggregator<PercentileCounter> {
+    final double compression;
+    PercentileCounter sum = null;
+
+    public PercentileAggregator(double compression) {
+        this.compression = compression;
+    }
+
+    @Override
+    public void reset() {
+        sum = null;
+    }
+
+    @Override
+    public void aggregate(PercentileCounter value) {
+        if (sum == null)
+            sum = new PercentileCounter(value);
+        else
+            sum.merge(value);
+    }
+
+    @Override
+    public PercentileCounter aggregate(PercentileCounter value1, PercentileCounter value2) {
+        PercentileCounter merged = new PercentileCounter(value1);
+        merged.merge(value2);
+        return merged;
+    }
+
+    @Override
+    public PercentileCounter getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        // 10K as upbound
+        // Test on random double data, 20 tDigest, each has 5000000 doubles. Finally merged into one tDigest.
+        // Before compress: 10309 bytes
+        // After compress: 8906 bytes
+        return 10 * 1024;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileContUdf.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileContUdf.java b/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileContUdf.java
new file mode 100644
index 0000000..4ef6b75
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileContUdf.java
@@ -0,0 +1,37 @@
+/*
+ * 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.measure.percentile;
+
+public class PercentileContUdf {
+    public static double init() {
+        return 0;
+    }
+
+    public static double add(double accumulator, double v, double r) {
+        return 0;
+    }
+
+    public static double merge(double accumulator0, double accumulator1) {
+        return 0;
+    }
+
+    public static double result(long accumulator) {
+        return 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileCounter.java
new file mode 100644
index 0000000..bf505cf
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileCounter.java
@@ -0,0 +1,97 @@
+/*
+ * 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.measure.percentile;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+
+import com.tdunning.math.stats.AVLTreeDigest;
+import com.tdunning.math.stats.TDigest;
+
+public class PercentileCounter implements Serializable {
+    private static final double INVALID_QUANTILE_RATIO = -1;
+
+    double compression;
+    double quantileRatio;
+
+    TDigest registers;
+
+    public PercentileCounter(double compression) {
+        this(compression, INVALID_QUANTILE_RATIO);
+    }
+
+    public PercentileCounter(PercentileCounter another) {
+        this(another.compression, another.quantileRatio);
+        merge(another);
+    }
+
+    public PercentileCounter(double compression, double quantileRatio) {
+        this.compression = compression;
+        this.quantileRatio = quantileRatio;
+        reInitRegisters();
+    }
+
+    private void reInitRegisters() {
+        this.registers = TDigest.createAvlTreeDigest(this.compression);
+    }
+
+    public void add(double v) {
+        registers.add(v);
+    }
+
+    public void merge(PercentileCounter counter) {
+        assert this.compression == counter.compression;
+        registers.add(counter.registers);
+    }
+
+    public double getResultEstimate() {
+        return registers.quantile(quantileRatio);
+    }
+
+    public void writeRegisters(ByteBuffer out) {
+        registers.compress();
+        registers.asSmallBytes(out);
+    }
+
+    public void readRegisters(ByteBuffer in) {
+        registers = AVLTreeDigest.fromBytes(in);
+        compression = registers.compression();
+    }
+
+    public int getBytesEstimate() {
+        return maxLength();
+    }
+
+    public int maxLength() {
+        // 10KB for max length
+        return 10 * 1024;
+    }
+
+    public int peekLength(ByteBuffer in) {
+        int mark = in.position();
+        AVLTreeDigest.fromBytes(in);
+        int total = in.position() - mark;
+        in.position(mark);
+        return total;
+    }
+
+    public void clear() {
+        reInitRegisters();
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileMeasureType.java
new file mode 100644
index 0000000..45ebe89
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileMeasureType.java
@@ -0,0 +1,102 @@
+/*
+ * 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.measure.percentile;
+
+import java.util.Map;
+
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.measure.MeasureTypeFactory;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.collect.ImmutableMap;
+
+public class PercentileMeasureType extends MeasureType<PercentileCounter> {
+    // compression ratio saved in DataType.precision
+    private final DataType dataType;
+    public static final String FUNC_PERCENTILE = "PERCENTILE";
+    public static final String DATATYPE_PERCENTILE = "percentile";
+
+    public PercentileMeasureType(String funcName, DataType dataType) {
+        this.dataType = dataType;
+    }
+
+    public static class Factory extends MeasureTypeFactory<PercentileCounter> {
+
+        @Override
+        public MeasureType<PercentileCounter> createMeasureType(String funcName, DataType dataType) {
+            return new PercentileMeasureType(funcName, dataType);
+        }
+
+        @Override
+        public String getAggrFunctionName() {
+            return FUNC_PERCENTILE;
+        }
+
+        @Override
+        public String getAggrDataTypeName() {
+            return DATATYPE_PERCENTILE;
+        }
+
+        @Override
+        public Class<? extends DataTypeSerializer<PercentileCounter>> getAggrDataTypeSerializer() {
+            return PercentileSerializer.class;
+        }
+    }
+
+    @Override
+    public MeasureIngester<PercentileCounter> newIngester() {
+        return new MeasureIngester<PercentileCounter>() {
+            PercentileCounter current = new PercentileCounter(dataType.getPrecision());
+
+            @Override
+            public PercentileCounter valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
+                PercentileCounter counter = current;
+                counter.clear();
+                for (String v : values) {
+                    if (v != null)
+                        counter.add(Double.parseDouble(v));
+                }
+                return counter;
+            }
+        };
+    }
+
+    @Override
+    public MeasureAggregator<PercentileCounter> newAggregator() {
+        return new PercentileAggregator(dataType.getPrecision());
+    }
+
+    @Override
+    public boolean needRewrite() {
+        return true;
+    }
+
+    static final Map<String, Class<?>> UDAF_MAP = ImmutableMap.<String, Class<?>> of(PercentileMeasureType.FUNC_PERCENTILE, PercentileAggFunc.class);
+
+    @Override
+    public Map<String, Class<?>> getRewriteCalciteAggrFunctions() {
+        return UDAF_MAP;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileSerializer.java b/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileSerializer.java
new file mode 100644
index 0000000..a0a2a77
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/percentile/PercentileSerializer.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.measure.percentile;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+
+public class PercentileSerializer extends DataTypeSerializer<PercentileCounter> {
+    // be thread-safe and avoid repeated obj creation
+    private ThreadLocal<PercentileCounter> current = new ThreadLocal<>();
+
+    private double compression;
+
+    public PercentileSerializer(DataType type) {
+        this.compression = type.getPrecision();
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return current().peekLength(in);
+    }
+
+    @Override
+    public int maxLength() {
+        return current().maxLength();
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return current().getBytesEstimate();
+    }
+
+    private PercentileCounter current() {
+        PercentileCounter counter = current.get();
+        if (counter == null) {
+            counter = new PercentileCounter(compression);
+            current.set(counter);
+        }
+        return counter;
+    }
+
+    @Override
+    public void serialize(PercentileCounter value, ByteBuffer out) {
+        value.writeRegisters(out);
+    }
+
+    @Override
+    public PercentileCounter deserialize(ByteBuffer in) {
+        PercentileCounter counter = current();
+        counter.readRegisters(in);
+        return counter;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-metadata/src/test/java/org/apache/kylin/measure/percentile/PercentileAggregatorTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/percentile/PercentileAggregatorTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/percentile/PercentileAggregatorTest.java
new file mode 100644
index 0000000..9d59d46
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/percentile/PercentileAggregatorTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.measure.percentile;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.kylin.common.util.MathUtil;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class PercentileAggregatorTest {
+    @Test
+    public void testAggregate() {
+        double compression = 100;
+        int datasize = 10000;
+        PercentileAggregator aggregator = new PercentileAggregator(compression);
+        Random random = new Random();
+        List<Double> dataset = Lists.newArrayListWithCapacity(datasize);
+        for (int i = 0; i < datasize; i++) {
+            double d = random.nextDouble();
+            dataset.add(d);
+
+            PercentileCounter c = new PercentileCounter(compression, 0.5);
+            c.add(d);
+            aggregator.aggregate(c);
+        }
+        Collections.sort(dataset);
+
+        double actualResult = aggregator.getState().getResultEstimate();
+        double expectResult = MathUtil.findMedianInSortedList(dataset);
+        assertEquals(expectResult, actualResult, 0.001);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-metadata/src/test/java/org/apache/kylin/measure/percentile/PercentileCounterTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/percentile/PercentileCounterTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/percentile/PercentileCounterTest.java
new file mode 100644
index 0000000..abaa409
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/percentile/PercentileCounterTest.java
@@ -0,0 +1,79 @@
+/*
+ * 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.measure.percentile;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.kylin.common.util.MathUtil;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+import com.tdunning.math.stats.TDigest;
+
+public class PercentileCounterTest {
+    @Test
+    public void testBasic() {
+        int times = 1;
+        int compression = 100;
+        for (int t = 0; t < times; t++) {
+            PercentileCounter counter = new PercentileCounter(compression, 0.5);
+            Random random = new Random();
+            int dataSize = 10000;
+            List<Double> dataset = Lists.newArrayListWithCapacity(dataSize);
+            for (int i = 0; i < dataSize; i++) {
+                double d = random.nextDouble();
+                counter.add(d);
+                dataset.add(d);
+            }
+            Collections.sort(dataset);
+
+            double actualResult = counter.getResultEstimate();
+            double expectedResult = MathUtil.findMedianInSortedList(dataset);
+            assertEquals(expectedResult, actualResult, 0.001);
+        }
+    }
+
+    @Test
+    public void testTDigest() {
+        double compression = 100;
+        double quantile = 0.5;
+
+        PercentileCounter counter = new PercentileCounter(compression, quantile);
+        TDigest tDigest = TDigest.createAvlTreeDigest(compression);
+
+        Random random = new Random();
+        int dataSize = 10000;
+        List<Double> dataset = Lists.newArrayListWithCapacity(dataSize);
+        for (int i = 0; i < dataSize; i++) {
+            double d = random.nextDouble();
+            counter.add(d);
+            tDigest.add(d);
+        }
+        double actualResult = counter.getResultEstimate();
+
+        Collections.sort(dataset);
+        double expectedResult = tDigest.quantile(quantile);
+
+        assertEquals(expectedResult, actualResult, 0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-metadata/src/test/java/org/apache/kylin/measure/percentile/PercentileSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/percentile/PercentileSerializerTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/percentile/PercentileSerializerTest.java
new file mode 100644
index 0000000..07b4495
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/percentile/PercentileSerializerTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.measure.percentile;
+
+import static org.junit.Assert.assertEquals;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Created by dongli on 5/21/16.
+ */
+public class PercentileSerializerTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setup() throws Exception {
+        createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanAfterClass();
+    }
+
+    @Test
+    public void testBasic() {
+        PercentileSerializer serializer = new PercentileSerializer(DataType.getType("percentile(100)"));
+        PercentileCounter counter = new PercentileCounter(100, 0.5);
+        Random random = new Random();
+        for (int i = 0; i < 1000; i++) {
+            counter.add(random.nextDouble());
+        }
+        double markResult = counter.getResultEstimate();
+
+        ByteBuffer buffer = ByteBuffer.allocateDirect(serializer.getStorageBytesEstimate());
+        serializer.serialize(counter, buffer);
+
+        buffer.flip();
+        counter = serializer.deserialize(buffer);
+        PercentileCounter counter1 = new PercentileCounter(100, 0.5);
+        counter1.merge(counter);
+
+        assertEquals(markResult, counter1.getResultEstimate(), 0.01);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/core-metadata/src/test/java/org/apache/kylin/measure/percentile/TDigestTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/percentile/TDigestTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/percentile/TDigestTest.java
new file mode 100644
index 0000000..1adb604
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/percentile/TDigestTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.measure.percentile;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.kylin.common.util.MathUtil;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+import com.tdunning.math.stats.TDigest;
+
+@Ignore
+public class TDigestTest {
+    @Test
+    public void testBasic() {
+        int times = 1;
+        int compression = 100;
+        for (int t = 0; t < times; t++) {
+            TDigest tDigest = TDigest.createAvlTreeDigest(compression);
+            Random random = new Random();
+            int dataSize = 10000;
+            List<Double> dataset = Lists.newArrayListWithCapacity(dataSize);
+            for (int i = 0; i < dataSize; i++) {
+                double d = random.nextDouble();
+                tDigest.add(d);
+                dataset.add(d);
+            }
+            Collections.sort(dataset);
+
+            double actualResult = tDigest.quantile(0.5);
+            double expectedResult = MathUtil.findMedianInSortedList(dataset);
+            assertEquals(expectedResult, actualResult, 0.01);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/examples/test_case_data/localmeta/cube_desc/ci_inner_join_cube.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/ci_inner_join_cube.json b/examples/test_case_data/localmeta/cube_desc/ci_inner_join_cube.json
index 1ebd7f3..0fda3b3 100644
--- a/examples/test_case_data/localmeta/cube_desc/ci_inner_join_cube.json
+++ b/examples/test_case_data/localmeta/cube_desc/ci_inner_join_cube.json
@@ -267,6 +267,16 @@
       },
       "returntype" : "raw"
     }
+  }, {
+    "name" : "GVM_PERCENTILE",
+    "function" : {
+      "expression" : "PERCENTILE",
+      "parameter" : {
+        "type" : "column",
+        "value" : "TEST_KYLIN_FACT.PRICE"
+      },
+      "returntype" : "percentile(100)"
+    }
   } ],
   "dictionaries": [ {
     "column": "TEST_KYLIN_FACT.TEST_COUNT_DISTINCT_BITMAP",
@@ -358,7 +368,7 @@
       "name" : "f3",
       "columns" : [ {
         "qualifier" : "m",
-        "measure_refs" : [ "TEST_EXTENDED_COLUMN", "TRANS_ID_RAW", "PRICE_RAW", "CAL_DT_RAW", "BUYER_CONTACT", "SELLER_CONTACT" ]
+        "measure_refs" : [ "TEST_EXTENDED_COLUMN", "TRANS_ID_RAW", "PRICE_RAW", "CAL_DT_RAW", "BUYER_CONTACT", "SELLER_CONTACT", "GVM_PERCENTILE" ]
       } ]
     } ]
   },

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/examples/test_case_data/localmeta/cube_desc/ci_left_join_cube.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/ci_left_join_cube.json b/examples/test_case_data/localmeta/cube_desc/ci_left_join_cube.json
index 4048b6e..51139ae 100644
--- a/examples/test_case_data/localmeta/cube_desc/ci_left_join_cube.json
+++ b/examples/test_case_data/localmeta/cube_desc/ci_left_join_cube.json
@@ -267,6 +267,16 @@
       },
       "returntype" : "raw"
     }
+  }, {
+    "name" : "GVM_PERCENTILE",
+    "function" : {
+      "expression" : "PERCENTILE",
+      "parameter" : {
+        "type" : "column",
+        "value" : "TEST_KYLIN_FACT.PRICE"
+      },
+      "returntype" : "percentile(100)"
+    }
   } ],
   "dictionaries": [ {
     "column": "TEST_KYLIN_FACT.TEST_COUNT_DISTINCT_BITMAP",
@@ -358,7 +368,7 @@
       "name" : "f3",
       "columns" : [ {
         "qualifier" : "m",
-        "measure_refs" : [ "TEST_EXTENDED_COLUMN", "TRANS_ID_RAW", "PRICE_RAW", "CAL_DT_RAW", "BUYER_CONTACT", "SELLER_CONTACT" ]
+        "measure_refs" : [ "TEST_EXTENDED_COLUMN", "TRANS_ID_RAW", "PRICE_RAW", "CAL_DT_RAW", "BUYER_CONTACT", "SELLER_CONTACT", "GVM_PERCENTILE" ]
       } ]
     } ]
   },

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/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 82d0873..02134d4 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
@@ -192,17 +192,17 @@ public class ITKylinQueryTest extends KylinTestBase {
     public void testSnowflakeQuery() throws Exception {
         execAndCompQuery(getQueryFolderPrefix() + "src/test/resources/query/sql_snowflake", null, true);
     }
-    
+
     @Test
     public void testDateTimeQuery() throws Exception {
         execAndCompQuery(getQueryFolderPrefix() + "src/test/resources/query/sql_datetime", null, true);
     }
-    
+
     @Test
     public void testExtendedColumnQuery() throws Exception {
         execAndCompQuery(getQueryFolderPrefix() + "src/test/resources/query/sql_extended_column", null, true);
     }
-    
+
     @Test
     public void testLikeQuery() throws Exception {
         execAndCompQuery(getQueryFolderPrefix() + "src/test/resources/query/sql_like", null, true);
@@ -270,8 +270,7 @@ public class ITKylinQueryTest extends KylinTestBase {
             this.batchExecuteQuery(getQueryFolderPrefix() + "src/test/resources/query/sql_intersect_count");
         }
     }
-    
-    
+
     @Test
     public void testMultiModelQuery() throws Exception {
         if ("left".equalsIgnoreCase(joinType)) {
@@ -280,7 +279,7 @@ public class ITKylinQueryTest extends KylinTestBase {
             joinType = "left";
         }
     }
-    
+
     @Test
     public void testDimDistinctCountQuery() throws Exception {
         execAndCompQuery(getQueryFolderPrefix() + "src/test/resources/query/sql_distinct_dim", null, true);
@@ -392,10 +391,15 @@ public class ITKylinQueryTest extends KylinTestBase {
         // compare the result
         Assert.assertEquals(expectVersion, queriedVersion);
     }
-    
+
     @Test
     public void testSelectStarColumnCount() throws Exception {
         execAndCompColumnCount("select * from test_kylin_fact limit 10", 11);
         execAndCompColumnCount("select * from test_kylin_fact", 11);
     }
+
+    @Test
+    public void testPercentileQuery() throws Exception {
+        batchExecuteQuery("src/test/resources/query/sql_percentile");
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/kylin-it/src/test/resources/query/sql_percentile/query01.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_percentile/query01.sql b/kylin-it/src/test/resources/query/sql_percentile/query01.sql
new file mode 100644
index 0000000..4f6d573
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_percentile/query01.sql
@@ -0,0 +1,2 @@
+select seller_id, percentile(price, 0.5) from test_kylin_fact
+group by seller_id
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e391103..e8ccb6c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -102,6 +102,7 @@
         <supercsv.version>2.4.0</supercsv.version>
         <cors.version>2.5</cors.version>
         <tomcat.version>8.5.9</tomcat.version>
+        <t-digest.version>3.1</t-digest.version>
 
         <!-- REST Service -->
         <spring.framework.version>3.2.17.RELEASE</spring.framework.version>
@@ -658,6 +659,11 @@
                 <version>${roaring.version}</version>
             </dependency>
             <dependency>
+                <groupId>com.tdunning</groupId>
+                <artifactId>t-digest</artifactId>
+                <version>${t-digest.version}</version>
+            </dependency>
+            <dependency>
                 <groupId>cglib</groupId>
                 <artifactId>cglib</artifactId>
                 <version>${cglib.version}</version>

http://git-wip-us.apache.org/repos/asf/kylin/blob/95d44121/storage-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/storage-hbase/pom.xml b/storage-hbase/pom.xml
index eb2c104..3aea531 100644
--- a/storage-hbase/pom.xml
+++ b/storage-hbase/pom.xml
@@ -121,6 +121,7 @@
                                     <include>org.apache.kylin:kylin-core-cube</include>
                                     <include>com.ning:compress-lzf</include>
                                     <include>org.roaringbitmap:RoaringBitmap</include>
+                                    <include>com.tdunning:t-digest</include>
                                     <!-- below for inverted index only -->
                                     <include>com.n3twork.druid:extendedset</include>
                                     <include>org.apache.commons:commons-lang3</include>


[17/21] kylin git commit: KYLIN-2400 remove byte[] interface in dictionary

Posted by li...@apache.org.
KYLIN-2400 remove byte[] interface in dictionary

Signed-off-by: Li Yang <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/e769bfa0
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/e769bfa0
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/e769bfa0

Branch: refs/heads/master-cdh5.7
Commit: e769bfa004e9d548c5a042f02765cc093ce70ec1
Parents: e20e2b2
Author: xiefan46 <95...@qq.com>
Authored: Mon Jan 9 19:11:39 2017 +0800
Committer: Li Yang <li...@apache.org>
Committed: Tue Jan 17 10:54:22 2017 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/util/Dictionary.java    | 46 ++++----------------
 .../apache/kylin/dict/AppendTrieDictionary.java | 19 ++++----
 .../org/apache/kylin/dict/CacheDictionary.java  | 13 ++++--
 .../apache/kylin/dict/DateStrDictionary.java    | 33 --------------
 .../apache/kylin/dict/DictionaryGenerator.java  |  2 +-
 .../org/apache/kylin/dict/NumberDictionary.java | 30 +------------
 .../kylin/dict/NumberDictionaryBuilder.java     |  8 ----
 .../apache/kylin/dict/TimeStrDictionary.java    | 29 ------------
 .../org/apache/kylin/dict/TrieDictionary.java   |  7 ++-
 .../kylin/dict/TrieDictionaryBuilder.java       |  2 +-
 .../apache/kylin/dict/TrieDictionaryForest.java | 27 +++---------
 .../kylin/dict/TrieDictionaryForestBuilder.java |  2 +-
 .../kylin/dict/AppendTrieDictionaryTest.java    | 10 ++---
 .../kylin/dict/DateStrDictionaryTest.java       |  3 --
 .../MultipleDictionaryValueEnumeratorTest.java  | 14 ------
 .../apache/kylin/dict/NumberDictionaryTest.java |  4 +-
 .../kylin/dict/TimeStrDictionaryTest.java       |  9 +++-
 .../dict/TrieDictionaryForestBenchmark.java     | 14 ++----
 .../kylin/dict/TrieDictionaryForestTest.java    | 11 ++---
 .../apache/kylin/dict/TrieDictionaryTest.java   | 19 ++++----
 .../kylin/measure/bitmap/BitmapMeasureType.java | 11 +++--
 .../kylin/measure/raw/RawMeasureType.java       |  7 ++-
 .../storage/gtrecord/DictGridTableTest.java     |  3 +-
 .../engine/mr/steps/MergeCuboidMapper.java      | 11 ++---
 .../mr/steps/NumberDictionaryForestTest.java    | 17 +++-----
 25 files changed, 92 insertions(+), 259 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java b/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
index 1e172bc..9d675f9 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
@@ -94,8 +94,13 @@ abstract public class Dictionary<T> implements Serializable {
     final public int getIdFromValue(T value, int roundingFlag) throws IllegalArgumentException {
         if (isNullObjectForm(value))
             return nullId();
-        else
-            return getIdFromValueImpl(value, roundingFlag);
+        else {
+            int id = getIdFromValueImpl(value, roundingFlag);
+            if(id == -1){
+                throw new IllegalArgumentException("Value : " + value + "no exists");
+            }
+            return id;
+        }
     }
 
     final public boolean containsValue(T value) throws IllegalArgumentException {
@@ -136,9 +141,7 @@ abstract public class Dictionary<T> implements Serializable {
      * Convenient form of
      * <code>getIdFromValueBytes(value, offset, len, 0)</code>
      */
-    final public int getIdFromValueBytes(byte[] value, int offset, int len) throws IllegalArgumentException {
-        return getIdFromValueBytes(value, offset, len, 0);
-    }
+
 
     /**
      * A lower level API, return ID integer from raw value bytes. In case of not found 
@@ -153,31 +156,6 @@ abstract public class Dictionary<T> implements Serializable {
      *             if value is not found in dictionary and rounding is off;
      *             or if rounding cannot find a smaller or bigger ID
      */
-    final public int getIdFromValueBytes(byte[] value, int offset, int len, int roundingFlag) throws IllegalArgumentException {
-        if (isNullByteForm(value, offset, len))
-            return nullId();
-        else {
-            int id = getIdFromValueBytesImpl(value, offset, len, roundingFlag);
-            if (id == -1)
-                throw new IllegalArgumentException("Value '" + Bytes.toString(value, offset, len) + "' (" + Bytes.toStringBinary(value, offset, len) + ") not exists!");
-            return id;
-        }
-    }
-
-    protected boolean isNullByteForm(byte[] value, int offset, int len) {
-        return value == null;
-    }
-
-    abstract protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag);
-
-    final public byte[] getValueBytesFromId(int id) {
-        if (isNullId(id))
-            return BytesUtil.EMPTY_BYTE_ARRAY;
-        else
-            return getValueBytesFromIdImpl(id);
-    }
-
-    abstract protected byte[] getValueBytesFromIdImpl(int id);
 
     /**
      * A lower level API, get byte values from ID, return the number of bytes
@@ -189,14 +167,6 @@ abstract public class Dictionary<T> implements Serializable {
      * @throws IllegalArgumentException
      *             if ID is not found in dictionary
      */
-    final public int getValueBytesFromId(int id, byte[] returnValue, int offset) throws IllegalArgumentException {
-        if (isNullId(id))
-            return -1;
-        else
-            return getValueBytesFromIdImpl(id, returnValue, offset);
-    }
-
-    abstract protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset);
 
     abstract public void dump(PrintStream out);
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
index b797167..3aef967 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
@@ -97,6 +97,8 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
         enableCache();
     }
 
+
+
     public void initParams(String baseDir, int baseId, int maxId, int maxValueLength, int nValues, BytesConverter bytesConverter) throws IOException {
         this.baseDir = baseDir;
         this.baseId = baseId;
@@ -1113,7 +1115,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
     }
 
     @Override
-    protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) {
+    protected int getIdFromValueBytesWithoutCache(byte[] value, int offset, int len, int roundingFlag) {
         if (dictSliceMap.isEmpty()) {
             return -1;
         }
@@ -1129,6 +1131,11 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
     }
 
     @Override
+    protected byte[] getValueBytesFromIdWithoutCache(int id) {
+        throw new UnsupportedOperationException("AppendTrieDictionary can't retrive value from id");
+    }
+
+    @Override
     public int getMinId() {
         return baseId;
     }
@@ -1149,16 +1156,6 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
     }
 
 
-    @Override
-    protected byte[] getValueBytesFromIdImpl(int id) {
-        throw new UnsupportedOperationException("AppendTrieDictionary can't retrive value from id");
-    }
-
-    @Override
-    protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) {
-        throw new UnsupportedOperationException("AppendTrieDictionary can't retrive value from id");
-    }
-
     public void flushIndex(CachedTreeMap dictSliceMap, boolean keepAppend) throws IOException {
         try (FSDataOutputStream indexOut = dictSliceMap.openIndexOutput()) {
             indexOut.writeInt(baseId);

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java
index b2bad53..156971d 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java
@@ -52,13 +52,13 @@ public abstract class CacheDictionary<T> extends Dictionary<T> {
                 if (id != null)
                     return id.intValue();
                 byte[] valueBytes = bytesConvert.convertToBytes(value);
-                id = getIdFromValueBytes(valueBytes, 0, valueBytes.length, roundingFlag);
+                id = getIdFromValueBytesWithoutCache(valueBytes, 0, valueBytes.length, roundingFlag);
                 cache.put(value, id);
                 return id;
             }
         }
         byte[] valueBytes = bytesConvert.convertToBytes(value);
-        return getIdFromValueBytes(valueBytes, 0, valueBytes.length, roundingFlag);
+        return getIdFromValueBytesWithoutCache(valueBytes, 0, valueBytes.length, roundingFlag);
     }
 
     //id --> value
@@ -70,13 +70,13 @@ public abstract class CacheDictionary<T> extends Dictionary<T> {
                 int seq = calcSeqNoFromId(id);
                 if (cache[seq] != null)
                     return (T) cache[seq];
-                byte[] valueBytes = getValueBytesFromIdImpl(id);
+                byte[] valueBytes = getValueBytesFromIdWithoutCache(id);
                 T value = bytesConvert.convertFromBytes(valueBytes, 0, valueBytes.length);
                 cache[seq] = value;
                 return value;
             }
         }
-        byte[] valueBytes = getValueBytesFromIdImpl(id);
+        byte[] valueBytes = getValueBytesFromIdWithoutCache(id);
         return bytesConvert.convertFromBytes(valueBytes, 0, valueBytes.length);
     }
 
@@ -99,4 +99,9 @@ public abstract class CacheDictionary<T> extends Dictionary<T> {
         this.valueToIdCache = null;
         this.idToValueCache = null;
     }
+
+    abstract protected byte[] getValueBytesFromIdWithoutCache(int id);
+
+    abstract protected int getIdFromValueBytesWithoutCache(byte[] valueBytes, int offset, int length, int roundingFlag);
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
index 29bbee2..56b4994 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
@@ -26,7 +26,6 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.PrintStream;
-import java.io.UnsupportedEncodingException;
 import java.util.Date;
 
 import org.apache.commons.lang.StringUtils;
@@ -84,10 +83,6 @@ public class DateStrDictionary extends Dictionary<String> {
         return pattern.length();
     }
 
-    @Override
-    protected boolean isNullByteForm(byte[] value, int offset, int len) {
-        return value == null || len == 0;
-    }
 
     @Override
     final protected int getIdFromValueImpl(String value, int roundFlag) {
@@ -107,34 +102,6 @@ public class DateStrDictionary extends Dictionary<String> {
         return dateToString(new Date(millis), pattern);
     }
 
-    @Override
-    final protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) {
-        try {
-            return getIdFromValue(new String(value, offset, len, "ISO-8859-1"));
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException(e); // never happen
-        }
-    }
-
-    @Override
-    final protected byte[] getValueBytesFromIdImpl(int id) {
-        String date = getValueFromId(id);
-        byte[] bytes;
-        try {
-            bytes = date.getBytes("ISO-8859-1");
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException(e); // never happen
-        }
-        return bytes;
-    }
-
-    @Override
-    final protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) {
-        byte[] bytes = getValueBytesFromIdImpl(id);
-        System.arraycopy(bytes, 0, returnValue, offset, bytes.length);
-        return bytes.length;
-    }
-
     private int calcIdFromSeqNo(int seq) {
         return seq < 0 ? seq : baseId + seq;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
index 6f4f2c4..c23e7b5 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
@@ -227,7 +227,7 @@ public class DictionaryGenerator {
         @Override
         public void init(DictionaryInfo info, int baseId) throws IOException {
             this.baseId = baseId;
-            this.builder = new NumberDictionaryBuilder(new StringBytesConverter());
+            this.builder = new NumberDictionaryBuilder(new NumberDictionaryForestBuilder.Number2BytesConverter());
         }
         
         @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
index 9458e9d..c55937d 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
@@ -175,37 +175,9 @@ public class NumberDictionary<T> extends TrieDictionary<T> {
     }
 
     @Override
-    protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) {
-        NumberBytesCodec codec = getCodec();
-        codec.encodeNumber(value, offset, len);
-        return super.getIdFromValueBytesImpl(codec.buf, codec.bufOffset, codec.bufLen, roundingFlag);
-    }
-
-    @Override
     protected boolean isNullObjectForm(T value) {
         return value == null || value.equals("");
     }
+    
 
-    @Override
-    protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) {
-        NumberBytesCodec codec = getCodec();
-        codec.bufOffset = 0;
-        codec.bufLen = super.getValueBytesFromIdImpl(id, codec.buf, 0);
-        return codec.decodeNumber(returnValue, offset);
-    }
-
-
-    public static void main(String[] args) throws Exception {
-        NumberDictionaryBuilder<String> b = new NumberDictionaryBuilder<String>(new StringBytesConverter());
-        b.addValue("10");
-        b.addValue("100");
-        b.addValue("40");
-        b.addValue("7");
-        TrieDictionary<String> dict = b.build(0);
-
-        //dict.enableIdToValueBytesCache();
-        for (int i = 0; i <= dict.getMaxId(); i++) {
-            System.out.println(Bytes.toString(dict.getValueBytesFromId(i)));
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java
index 68a05d4..b9e94414 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java
@@ -18,7 +18,6 @@
 
 package org.apache.kylin.dict;
 
-import org.apache.kylin.common.util.Bytes;
 
 /**
  * Use <code>NumberDictionaryForestBuilder</code> instead.
@@ -28,18 +27,11 @@ import org.apache.kylin.common.util.Bytes;
 @Deprecated
 public class NumberDictionaryBuilder<T> extends TrieDictionaryBuilder<T> {
 
-    NumberDictionary.NumberBytesCodec codec = new NumberDictionary.NumberBytesCodec(NumberDictionary.MAX_DIGITS_BEFORE_DECIMAL_POINT);
 
     public NumberDictionaryBuilder(BytesConverter<T> bytesConverter) {
         super(bytesConverter);
     }
 
-    @Override
-    void addValue(byte[] value) {
-        codec.encodeNumber(value, 0, value.length);
-        byte[] copy = Bytes.copy(codec.buf, codec.bufOffset, codec.bufLen);
-        super.addValue(copy);
-    }
 
     public NumberDictionary<T> build(int baseId) {
         byte[] trieBytes = buildTrieBytes(baseId);

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java
index eabc9f1..d075ce1 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java
@@ -22,7 +22,6 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.PrintStream;
-import java.io.UnsupportedEncodingException;
 
 import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.common.util.Dictionary;
@@ -85,34 +84,6 @@ public class TimeStrDictionary extends Dictionary<String> {
     }
 
     @Override
-    final protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) {
-        try {
-            return getIdFromValue(new String(value, offset, len, "ISO-8859-1"));
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException(e); // never happen
-        }
-    }
-
-    @Override
-    final protected byte[] getValueBytesFromIdImpl(int id) {
-        String date = getValueFromId(id);
-        byte[] bytes;
-        try {
-            bytes = date.getBytes("ISO-8859-1");
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException(e); // never happen
-        }
-        return bytes;
-    }
-
-    @Override
-    final protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) {
-        byte[] bytes = getValueBytesFromIdImpl(id);
-        System.arraycopy(bytes, 0, returnValue, offset, bytes.length);
-        return bytes.length;
-    }
-
-    @Override
     public void dump(PrintStream out) {
         out.println(this.toString());
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
index 8e7f5dc..9b84734 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
@@ -141,7 +141,7 @@ public class TrieDictionary<T> extends CacheDictionary<T> {
 
 
     @Override
-    protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) {
+    protected int getIdFromValueBytesWithoutCache(byte[] value, int offset, int len, int roundingFlag) {
         int seq = lookupSeqNoFromValue(headSize, value, offset, offset + len, roundingFlag);
         int id = calcIdFromSeqNo(seq);
         if (id < 0)
@@ -236,7 +236,7 @@ public class TrieDictionary<T> extends CacheDictionary<T> {
 
 
     @Override
-    protected byte[] getValueBytesFromIdImpl(int id) {
+    protected byte[] getValueBytesFromIdWithoutCache(int id) {
         byte[] buf = new byte[maxValueLength];
         int len = getValueBytesFromIdImpl(id, buf, 0);
 
@@ -249,7 +249,6 @@ public class TrieDictionary<T> extends CacheDictionary<T> {
         }
     }
 
-    @Override
     protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) {
         int seq = calcSeqNoFromId(id);
         return lookupValueFromSeqNo(headSize, seq, returnValue, offset);
@@ -425,7 +424,7 @@ public class TrieDictionary<T> extends CacheDictionary<T> {
 
         //dict2.enableIdToValueBytesCache();
         for (int i = 0; i <= dict.getMaxId(); i++) {
-            System.out.println(Bytes.toString(dict.getValueBytesFromId(i)));
+            System.out.println(Bytes.toString(dict.getValueBytesFromIdWithoutCache(i)));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
index 598865b..102c49e 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
@@ -74,7 +74,7 @@ public class TrieDictionaryBuilder<T> {
     // ============================================================================
 
     private Node root;
-    private BytesConverter<T> bytesConverter;
+    protected BytesConverter<T> bytesConverter;
 
     public TrieDictionaryBuilder(BytesConverter<T> bytesConverter) {
         this.root = new Node(new byte[0], false);

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
index 69b4b9c..d1b5a99 100755
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
@@ -104,15 +104,7 @@ public class TrieDictionaryForest<T> extends CacheDictionary<T> {
     }
 
     @Override
-    protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) throws IllegalArgumentException {
-
-        int result = _getIdFromValueBytesImpl(value, offset, len, roundingFlag);
-        //logger.info("{} => {}, rounding {}", bytesConvert.convertFromBytes(value, offset, len), result, roundingFlag);
-        return result;
-    }
-
-    // id = tree_inner_offset + accumulate_offset + baseId
-    protected int _getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) throws IllegalArgumentException {
+    protected int getIdFromValueBytesWithoutCache(byte[] value, int offset, int len, int roundingFlag) throws IllegalArgumentException {
         int index;
         if (trees.size() == 1) {
             index = 0;
@@ -136,27 +128,20 @@ public class TrieDictionaryForest<T> extends CacheDictionary<T> {
             }
         }
         TrieDictionary<T> tree = trees.get(index);
-        int id = tree.getIdFromValueBytes(value, offset, len, roundingFlag);
+        int id = tree.getIdFromValueBytesWithoutCache(value, offset, len, roundingFlag);
+        if(id == -1)
+            throw new IllegalArgumentException("Value '" + Bytes.toString(value, offset, len) + "' (" + Bytes.toStringBinary(value, offset, len) + ") not exists!");
         id = id + accuOffset.get(index);
         id += baseId;
         return id;
     }
 
     @Override
-    protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) throws IllegalArgumentException {
-        int index = (trees.size() == 1) ? 0 : findIndexById(id);
-        int treeInnerOffset = getTreeInnerOffset(id, index);
-        TrieDictionary<T> tree = trees.get(index);
-        int size = tree.getValueBytesFromIdImpl(treeInnerOffset, returnValue, offset);
-        return size;
-    }
-
-    @Override
-    protected byte[] getValueBytesFromIdImpl(int id) throws IllegalArgumentException {
+    protected byte[] getValueBytesFromIdWithoutCache(int id) throws IllegalArgumentException {
         int index = (trees.size() == 1) ? 0 : findIndexById(id);
         int treeInnerOffset = getTreeInnerOffset(id, index);
         TrieDictionary<T> tree = trees.get(index);
-        byte[] result = tree.getValueBytesFromId(treeInnerOffset);
+        byte[] result = tree.getValueBytesFromIdWithoutCache(treeInnerOffset);
         return result;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForestBuilder.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForestBuilder.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForestBuilder.java
index af2e302..69da472 100755
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForestBuilder.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForestBuilder.java
@@ -131,7 +131,7 @@ public class TrieDictionaryForestBuilder<T> {
         trees.add(tree);
         int minId = tree.getMinId();
         accuOffset.add(curOffset);
-        byte[] valueBytes = tree.getValueBytesFromId(minId);
+        byte[] valueBytes = tree.getValueBytesFromIdWithoutCache(minId);
         valueDivide.add(new ByteArray(valueBytes, 0, valueBytes.length));
         curOffset += (tree.getMaxId() + 1);
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
index 921925c..18913d0 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
@@ -180,7 +180,7 @@ public class AppendTrieDictionaryTest extends LocalFileMetadataTestCase {
         for (; checkIndex < firstAppend; checkIndex++) {
             String str = strList.get(checkIndex);
             byte[] bytes = converter.convertToBytes(str);
-            int id = dict.getIdFromValueBytesImpl(bytes, 0, bytes.length, 0);
+            int id = dict.getIdFromValueBytesWithoutCache(bytes, 0, bytes.length, 0);
             assertNotEquals(String.format("Value %s not exist", str), -1, id);
             assertFalse(String.format("Id %d for %s should be empty, but is %s", id, str, checkMap.get(id)), checkMap.containsKey(id) && !str.equals(checkMap.get(id)));
             checkMap.put(id, str);
@@ -200,7 +200,7 @@ public class AppendTrieDictionaryTest extends LocalFileMetadataTestCase {
         for (; checkIndex < secondAppend; checkIndex++) {
             String str = strList.get(checkIndex);
             byte[] bytes = converter.convertToBytes(str);
-            int id = dict.getIdFromValueBytesImpl(bytes, 0, bytes.length, 0);
+            int id = dict.getIdFromValueBytesWithoutCache(bytes, 0, bytes.length, 0);
             assertNotEquals(String.format("Value %s not exist", str), -1, id);
             if (checkIndex < firstAppend) {
                 assertEquals("Except id " + id + " for " + str + " but " + checkMap.get(id), str, checkMap.get(id));
@@ -224,7 +224,7 @@ public class AppendTrieDictionaryTest extends LocalFileMetadataTestCase {
         for (; checkIndex < strList.size(); checkIndex++) {
             String str = strList.get(checkIndex);
             byte[] bytes = converter.convertToBytes(str);
-            int id = dict.getIdFromValueBytesImpl(bytes, 0, bytes.length, 0);
+            int id = dict.getIdFromValueBytesWithoutCache(bytes, 0, bytes.length, 0);
             assertNotEquals(String.format("Value %s not exist", str), -1, id);
             if (checkIndex < secondAppend) {
                 assertEquals("Except id " + id + " for " + str + " but " + checkMap.get(id), str, checkMap.get(id));
@@ -237,7 +237,7 @@ public class AppendTrieDictionaryTest extends LocalFileMetadataTestCase {
         if (notfound != null) {
             for (String s : notfound) {
                 byte[] bytes = converter.convertToBytes(s);
-                int id = dict.getIdFromValueBytesImpl(bytes, 0, bytes.length, 0);
+                int id = dict.getIdFromValueBytesWithoutCache(bytes, 0, bytes.length, 0);
                 assertEquals(-1, id);
             }
         }
@@ -245,7 +245,7 @@ public class AppendTrieDictionaryTest extends LocalFileMetadataTestCase {
         dict = testSerialize(dict, converter);
         for (String str : strList) {
             byte[] bytes = converter.convertToBytes(str);
-            int id = dict.getIdFromValueBytesImpl(bytes, 0, bytes.length, 0);
+            int id = dict.getIdFromValueBytesWithoutCache(bytes, 0, bytes.length, 0);
             assertNotEquals(String.format("Value %s not exist", str), -1, id);
             assertEquals("Except id " + id + " for " + str + " but " + checkMap.get(id), str, checkMap.get(id));
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/test/java/org/apache/kylin/dict/DateStrDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/DateStrDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/DateStrDictionaryTest.java
index ffc8ceb..c6d2e05 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/DateStrDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/DateStrDictionaryTest.java
@@ -66,9 +66,6 @@ public class DateStrDictionaryTest {
     public void testNull() {
         int nullId = dict.getIdFromValue(null);
         assertNull(dict.getValueFromId(nullId));
-        int nullId2 = dict.getIdFromValueBytes(null, 0, 0);
-        assertEquals(dict.getValueBytesFromId(nullId2, null, 0), -1);
-        assertEquals(nullId, nullId2);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java
index 73e0935..2e90bcf 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java
@@ -134,20 +134,6 @@ public class MultipleDictionaryValueEnumeratorTest {
             return "" + values[id];
         }
 
-        @Override
-        protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) {
-            return 0;
-        }
-
-        @Override
-        protected byte[] getValueBytesFromIdImpl(int id) {
-            return null;
-        }
-
-        @Override
-        protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) {
-            return 0;
-        }
 
         @Override
         public void dump(PrintStream out) {}

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
index 23e925c..38f2648 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
@@ -59,7 +59,7 @@ public class NumberDictionaryTest extends LocalFileMetadataTestCase {
 
     @Test
     public void testMinMax() {
-        NumberDictionaryBuilder<String> builder = new NumberDictionaryBuilder<String>(new StringBytesConverter());
+        NumberDictionaryBuilder<String> builder = new NumberDictionaryBuilder<String>(new NumberDictionaryForestBuilder.Number2BytesConverter());
         builder.addValue("" + Long.MAX_VALUE);
         builder.addValue("" + Long.MIN_VALUE);
         NumberDictionary<String> dict = builder.build(0);
@@ -123,7 +123,7 @@ public class NumberDictionaryTest extends LocalFileMetadataTestCase {
         int n = 100;
 
         Set<BigDecimal> set = Sets.newHashSet();
-        NumberDictionaryBuilder<String> builder = new NumberDictionaryBuilder<String>(new StringBytesConverter());
+        NumberDictionaryBuilder<String> builder = new NumberDictionaryBuilder<String>(new NumberDictionaryForestBuilder.Number2BytesConverter());
         for (int i = 0; i < n; i++) {
             String num = randNumber();
             if (set.add(new BigDecimal(num))) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java
index ce07a86..f981942 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java
@@ -23,6 +23,8 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.junit.Assert.fail;
+
 /**
  */
 public class TimeStrDictionaryTest {
@@ -62,7 +64,12 @@ public class TimeStrDictionaryTest {
 
     @Test
     public void testIllegal() {
-        Assert.assertEquals(-1, dict.getIdFromValue("2038-01-19 03:14:07"));
+        try{
+            dict.getIdFromValue("2038-01-19 03:14:07");
+            fail("should throw exception");
+        }catch (IllegalArgumentException e){
+            //correct
+        }
     }
 
     public void encodeDecode(String origin) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestBenchmark.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestBenchmark.java b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestBenchmark.java
index 0b4c0e3..adc1074 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestBenchmark.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestBenchmark.java
@@ -25,7 +25,6 @@ import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.List;
 import java.util.Random;
 import java.util.UUID;
 
@@ -123,7 +122,8 @@ public class TrieDictionaryForestBenchmark {
         int step = 1;
         for (int i = 0; i < testTimes; i++) {
             for (int j = 0; j < cardnality; j++) {
-                step |= dict.getValueBytesFromId(j).length;
+                //step |= dict.getValueBytesFromId(j).length;
+                step |= dict.getValueFromId(j).length();
             }
         }
         return System.currentTimeMillis() - startTime;
@@ -135,8 +135,7 @@ public class TrieDictionaryForestBenchmark {
         byte[] returnValue = new byte[2048];
         for (int i = 0; i < testTimes; i++) {
             for (int j = 0; j < cardnality; j++) {
-                int size = dict.getValueBytesFromId(j, returnValue, 0);
-                step |= size;
+                step |= dict.getValueFromId(j).length();
             }
         }
         return System.currentTimeMillis() - startTime;
@@ -154,16 +153,11 @@ public class TrieDictionaryForestBenchmark {
     }
 
     private long runQueryIdByValueBytes(ArrayList<String> rawData, Dictionary<String> dict, int cardnality, int testTimes) {
-        List<byte[]> testBytes = new ArrayList<>();
-        StringBytesConverter converter = new StringBytesConverter();
-        for (int i = 0; i < cardnality; i++) {
-            testBytes.add(converter.convertToBytes(rawData.get(i)));
-        }
         long startTime = System.currentTimeMillis();
         int step = 1;
         for (int i = 0; i < testTimes; i++) {
             for (int j = 0; j < cardnality; j++) {
-                step |= dict.getIdFromValueBytes(testBytes.get(j), 0, testBytes.get(j).length);
+                step |= dict.getIdFromValue(rawData.get(j));
             }
         }
         return System.currentTimeMillis() - startTime;

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java
index 68cf301..c7fb9c4 100755
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java
@@ -641,7 +641,7 @@ public class TrieDictionaryForestTest {
         System.out.println("max memory:" + Runtime.getRuntime().maxMemory());
         System.gc();
         Thread.currentThread().sleep(1000);
-        NumberDictionaryBuilder<String> b = new NumberDictionaryBuilder<>(new StringBytesConverter());
+        NumberDictionaryBuilder<String> b = new NumberDictionaryBuilder<>(new NumberDictionaryForestBuilder.Number2BytesConverter());
         int k = 0;
         while (true) {
             b.addValue(k + "");
@@ -820,7 +820,7 @@ public class TrieDictionaryForestTest {
         for (int i = 0; i < times; i++) {
             for (int j = 0; j < n; j++) {
                 //System.out.println("looking for value:"+new String(array[j]));
-                keep |= dict.getIdFromValueBytes(array[j], 0, array[j].length);
+                keep |= dict.getIdFromValueBytesWithoutCache(array[j], 0, array[j].length, 0);
             }
         }
         long timeValueToIdByDict = System.currentTimeMillis() - start;
@@ -845,7 +845,7 @@ public class TrieDictionaryForestTest {
         start = System.currentTimeMillis();
         for (int i = 0; i < times; i++) {
             for (int j = 0; j < n; j++) {
-                keep |= dict.getValueBytesFromId(j, valueBytes, 0);
+                keep |= dict.getValueBytesFromIdWithoutCache(j).length;
             }
         }
         long timeIdToValueByDict = System.currentTimeMillis() - start;
@@ -884,7 +884,7 @@ public class TrieDictionaryForestTest {
             for (String s : notFound) {
                 try {
                     int nullId = dict.getIdFromValue(s);
-                    //System.out.println("null value id:" + nullId);
+                    System.out.println("null value id:" + nullId);
                     fail("For not found value '" + s + "', IllegalArgumentException is expected");
                 } catch (IllegalArgumentException e) {
                     // good
@@ -905,9 +905,6 @@ public class TrieDictionaryForestTest {
         // test null value
         int nullId = dict.getIdFromValue(null);
         assertNull(dict.getValueFromId(nullId));
-        int nullId2 = dict.getIdFromValueBytes(null, 0, 0);
-        assertEquals(dict.getValueBytesFromId(nullId2, null, 0), -1);
-        assertEquals(nullId, nullId2);
     }
 
     private Map<String, Integer> rightIdMap(int baseId, ArrayList<String> strs) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryTest.java
index a87d7cb..22a93a0 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryTest.java
@@ -47,7 +47,7 @@ public class TrieDictionaryTest {
         int count = (int) (Integer.MAX_VALUE * 0.8 / 64);
         benchmarkStringDictionary(new RandomStrings(count));
     }
-    
+
     private static class RandomStrings implements Iterable<String> {
         final private int size;
 
@@ -55,7 +55,7 @@ public class TrieDictionaryTest {
             this.size = size;
             System.out.println("size = " + size);
         }
-        
+
         @Override
         public Iterator<String> iterator() {
             return new Iterator<String>() {
@@ -71,11 +71,11 @@ public class TrieDictionaryTest {
                 public String next() {
                     if (hasNext() == false)
                         throw new NoSuchElementException();
-                    
+
                     i++;
                     if (i % 1000000 == 0)
                         System.out.println(i);
-                    
+
                     return nextString();
                 }
 
@@ -225,7 +225,7 @@ public class TrieDictionaryTest {
         TrieDictionaryBuilder<String> b = newDictBuilder(str);
         b.stats().print();
         TrieDictionary<String> dict = b.build(0);
-        
+
         TreeSet<String> set = new TreeSet<String>();
         for (String s : str) {
             set.add(s);
@@ -282,7 +282,7 @@ public class TrieDictionaryTest {
         start = System.currentTimeMillis();
         for (int i = 0; i < times; i++) {
             for (int j = 0; j < n; j++) {
-                keep |= dict.getIdFromValueBytes(array[j], 0, array[j].length);
+                keep |= dict.getIdFromValueBytesWithoutCache(array[j], 0, array[j].length, 0);
             }
         }
         long timeValueToIdByDict = System.currentTimeMillis() - start;
@@ -304,12 +304,12 @@ public class TrieDictionaryTest {
         start = System.currentTimeMillis();
         for (int i = 0; i < times; i++) {
             for (int j = 0; j < n; j++) {
-                keep |= dict.getValueBytesFromId(j, valueBytes, 0);
+                keep |= dict.getValueBytesFromIdWithoutCache(j).length;
             }
         }
         long timeIdToValueByDict = System.currentTimeMillis() - start;
         System.out.println(timeIdToValueByDict);
-        
+
         return keep;
     }
 
@@ -350,9 +350,6 @@ public class TrieDictionaryTest {
         // test null value
         int nullId = dict.getIdFromValue(null);
         assertNull(dict.getValueFromId(nullId));
-        int nullId2 = dict.getIdFromValueBytes(null, 0, 0);
-        assertEquals(dict.getValueBytesFromId(nullId2, null, 0), -1);
-        assertEquals(nullId, nullId2);
     }
 
     private static TrieDictionary<String> testSerialize(TrieDictionary<String> dict) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
index b6f1975..de2a34a 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
@@ -18,6 +18,8 @@
 
 package org.apache.kylin.measure.bitmap;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
@@ -38,8 +40,6 @@ import org.apache.kylin.metadata.realization.SQLDigest.SQLCall;
 
 import com.google.common.collect.ImmutableMap;
 
-import static com.google.common.base.Preconditions.checkArgument;
-
 /**
  * Created by sunyerui on 15/12/10.
  */
@@ -133,14 +133,13 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
                 Dictionary<String> mergedDict = newDicts.get(colRef);
 
                 MutableBitmapCounter retValue = new MutableBitmapCounter();
-                byte[] literal = new byte[sourceDict.getSizeOfValue()];
                 for (int id : value) {
                     int newId;
-                    int size = sourceDict.getValueBytesFromId(id, literal, 0);
-                    if (size < 0) {
+                    String v = sourceDict.getValueFromId(id);
+                    if (v == null) {
                         newId = mergedDict.nullId();
                     } else {
-                        newId = mergedDict.getIdFromValueBytes(literal, 0, size);
+                        newId = mergedDict.getIdFromValue(v);
                     }
                     retValue.add(newId);
                 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
index a5bb06b..3b6e659 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
@@ -133,17 +133,16 @@ public class RawMeasureType extends MeasureType<List<ByteArray>> {
 
                 int valueSize = value.size();
                 byte[] newIdBuf = new byte[valueSize * mergedDict.getSizeOfId()];
-                byte[] literal = new byte[sourceDict.getSizeOfValue()];
 
                 int bufOffset = 0;
                 for (ByteArray c : value) {
                     int oldId = BytesUtil.readUnsigned(c.array(), c.offset(), c.length());
                     int newId;
-                    int size = sourceDict.getValueBytesFromId(oldId, literal, 0);
-                    if (size < 0) {
+                    String v = sourceDict.getValueFromId(oldId);
+                    if (v == null) {
                         newId = mergedDict.nullId();
                     } else {
-                        newId = mergedDict.getIdFromValueBytes(literal, 0, size);
+                        newId = mergedDict.getIdFromValue(v);
                     }
                     BytesUtil.writeUnsigned(newId, newIdBuf, bufOffset, mergedDict.getSizeOfId());
                     c.set(newIdBuf, bufOffset, mergedDict.getSizeOfId());

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
----------------------------------------------------------------------
diff --git a/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java b/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
index fcd0182..ae186e2 100644
--- a/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
+++ b/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
@@ -35,6 +35,7 @@ import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.gridtable.CubeCodeSystem;
 import org.apache.kylin.dict.NumberDictionaryBuilder;
+import org.apache.kylin.dict.NumberDictionaryForestBuilder;
 import org.apache.kylin.dict.StringBytesConverter;
 import org.apache.kylin.dict.TrieDictionaryBuilder;
 import org.apache.kylin.dimension.DictionaryDimEnc;
@@ -601,7 +602,7 @@ public class DictGridTableTest extends LocalFileMetadataTestCase {
 
     @SuppressWarnings("rawtypes")
     private static Dictionary newDictionaryOfInteger() {
-        NumberDictionaryBuilder<String> builder = new NumberDictionaryBuilder<>(new StringBytesConverter());
+        NumberDictionaryBuilder<String> builder = new NumberDictionaryBuilder<>(new NumberDictionaryForestBuilder.Number2BytesConverter());
         builder.addValue("10");
         builder.addValue("20");
         builder.addValue("30");

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
index a79d5aa..047e2b1 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
@@ -186,8 +186,8 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
             if (this.checkNeedMerging(col)) {
                 // if dictionary on fact table column, needs rewrite
                 DictionaryManager dictMgr = DictionaryManager.getInstance(config);
-                Dictionary<?> sourceDict = dictMgr.getDictionary(sourceCubeSegment.getDictResPath(col));
-                Dictionary<?> mergedDict = dictMgr.getDictionary(mergedCubeSegment.getDictResPath(col));
+                Dictionary<String> sourceDict = dictMgr.getDictionary(sourceCubeSegment.getDictResPath(col));
+                Dictionary<String> mergedDict = dictMgr.getDictionary(mergedCubeSegment.getDictResPath(col));
 
                 while (sourceDict.getSizeOfValue() > newKeyBodyBuf.length - bufOffset || //
                         mergedDict.getSizeOfValue() > newKeyBodyBuf.length - bufOffset || //
@@ -200,11 +200,12 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
                 int idInSourceDict = BytesUtil.readUnsigned(splittedByteses[useSplit].value, 0, splittedByteses[useSplit].length);
                 int idInMergedDict;
 
-                int size = sourceDict.getValueBytesFromId(idInSourceDict, newKeyBodyBuf, bufOffset);
-                if (size < 0) {
+                //int size = sourceDict.getValueBytesFromId(idInSourceDict, newKeyBodyBuf, bufOffset);
+                String v = sourceDict.getValueFromId(idInSourceDict);
+                if (v == null) {
                     idInMergedDict = mergedDict.nullId();
                 } else {
-                    idInMergedDict = mergedDict.getIdFromValueBytes(newKeyBodyBuf, bufOffset, size);
+                    idInMergedDict = mergedDict.getIdFromValue(v);
                 }
 
                 BytesUtil.writeUnsigned(idInMergedDict, newKeyBodyBuf, bufOffset, mergedDict.getSizeOfId());

http://git-wip-us.apache.org/repos/asf/kylin/blob/e769bfa0/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java
index f6664bf..72793f5 100644
--- a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java
@@ -38,7 +38,6 @@ import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.dict.NumberDictionary;
 import org.apache.kylin.dict.NumberDictionaryBuilder;
 import org.apache.kylin.dict.NumberDictionaryForestBuilder;
-import org.apache.kylin.dict.StringBytesConverter;
 import org.apache.kylin.dict.TrieDictionaryForest;
 import org.apache.kylin.engine.mr.steps.SelfDefineSortableKey.TypeFlag;
 import org.junit.Ignore;
@@ -121,7 +120,7 @@ public class NumberDictionaryForestTest {
         TrieDictionaryForest<String> dict = b.build();
         dict.dump(System.out);
 
-        NumberDictionaryBuilder<String> b2 = new NumberDictionaryBuilder<>(new StringBytesConverter());
+        NumberDictionaryBuilder<String> b2 = new NumberDictionaryBuilder<>(new NumberDictionaryForestBuilder.Number2BytesConverter());
         for (String str : testData)
             b2.addValue(str);
         NumberDictionary<String> dict2 = b2.build(0);
@@ -148,22 +147,20 @@ public class NumberDictionaryForestTest {
         
         assertTrue(dict1.getSizeOfId() == dict2.getSizeOfId());
         assertTrue(dict1.getSizeOfValue() == dict2.getSizeOfValue());
-        
+
         byte[] buf = new byte[dict1.getSizeOfValue()];
-        
+
         {
-            int len = dict1.getValueBytesFromId(0, buf, 0);
-            int newId = dict2.getIdFromValueBytes(buf, 0, len);
+            int newId = dict2.getIdFromValue(dict1.getValueFromId(0));
             assertTrue(newId == 0);
         }
         {
-            int len = dict1.getValueBytesFromId(1, buf, 0);
-            int newId = dict2.getIdFromValueBytes(buf, 0, len);
+
+            int newId = dict2.getIdFromValue(dict1.getValueFromId(1));
             assertTrue(newId == 2);
         }
         {
-            int len = dict1.getValueBytesFromId(2, buf, 0);
-            int newId = dict2.getIdFromValueBytes(buf, 0, len);
+            int newId = dict2.getIdFromValue(dict1.getValueFromId(2));
             assertTrue(newId == 4);
         }
     }


[09/21] kylin git commit: KYLIN-2304 Only copy latest version dict for global dict

Posted by li...@apache.org.
KYLIN-2304 Only copy latest version dict for global dict


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

Branch: refs/heads/master-cdh5.7
Commit: eb18ac59fa74b3f2f1f0163f7728739a1027a233
Parents: 4b00ec2
Author: kangkaisen <ka...@live.com>
Authored: Sun Dec 18 20:09:03 2016 +0800
Committer: kangkaisen <ka...@163.com>
Committed: Sat Jan 14 19:07:59 2017 +0800

----------------------------------------------------------------------
 .../apache/kylin/dict/AppendTrieDictionary.java  | 19 +++++++++++++------
 .../org/apache/kylin/dict/CachedTreeMap.java     |  2 +-
 2 files changed, 14 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/eb18ac59/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
index 5029bc4..80403cc 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
@@ -1173,13 +1173,15 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
 
     @Override
     public AppendTrieDictionary copyToAnotherMeta(KylinConfig srcConfig, KylinConfig dstConfig) throws IOException {
+        //copy appendDict
         Configuration conf = new Configuration();
-        AppendTrieDictionary newDict = new AppendTrieDictionary();
-        newDict.initParams(baseDir.replaceFirst(srcConfig.getHdfsWorkingDirectory(), dstConfig.getHdfsWorkingDirectory()), baseId, maxId, maxValueLength, nValues, bytesConvert);
-        newDict.initDictSliceMap((CachedTreeMap)dictSliceMap);
-        logger.info("Copy AppendDict from {} to {}", this.baseDir, newDict.baseDir);
-        Path srcPath = new Path(this.baseDir);
-        Path dstPath = new Path(newDict.baseDir);
+
+        Path base = new Path(baseDir);
+        FileSystem srcFs = FileSystem.get(base.toUri(), conf);
+        Path srcPath = CachedTreeMap.getLatestVersion(conf, srcFs, base);
+        Path dstPath = new Path(srcPath.toString().replaceFirst(srcConfig.getHdfsWorkingDirectory(), dstConfig.getHdfsWorkingDirectory()));
+        logger.info("Copy appendDict from {} to {}", srcPath, dstPath);
+
         FileSystem dstFs = FileSystem.get(dstPath.toUri(), conf);
         if (dstFs.exists(dstPath)) {
             logger.info("Delete existing AppendDict {}", dstPath);
@@ -1187,6 +1189,11 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
         }
         FileUtil.copy(FileSystem.get(srcPath.toUri(), conf), srcPath, FileSystem.get(dstPath.toUri(), conf), dstPath, false, true, conf);
 
+        // init new AppendTrieDictionary
+        AppendTrieDictionary newDict = new AppendTrieDictionary();
+        newDict.initParams(baseDir.replaceFirst(srcConfig.getHdfsWorkingDirectory(), dstConfig.getHdfsWorkingDirectory()), baseId, maxId, maxValueLength, nValues, bytesConverter);
+        newDict.initDictSliceMap((CachedTreeMap) dictSliceMap);
+
         return newDict;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/eb18ac59/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java b/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java
index 3be4fd7..cc23261 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java
@@ -225,7 +225,7 @@ public class CachedTreeMap<K extends WritableComparable, V extends Writable> ext
         return getLatestVersion(conf, fs, baseDir).toUri().getPath();
     }
 
-    private static Path getLatestVersion(Configuration conf, FileSystem fs, Path baseDir) throws IOException {
+    public static Path getLatestVersion(Configuration conf, FileSystem fs, Path baseDir) throws IOException {
         String[] versions = listAllVersions(fs, baseDir);
         if (versions.length > 0) {
             return new Path(versions[versions.length - 1]);


[13/21] kylin git commit: KYLIN-2395 disable tomcat jar scanner to avoid lots of trivial warning messages

Posted by li...@apache.org.
KYLIN-2395 disable tomcat jar scanner to avoid lots of trivial warning messages

Signed-off-by: Billy Liu <bi...@apache.org>


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

Branch: refs/heads/master-cdh5.7
Commit: 4bc31de965ae67717bc4e05e0c7e91c957297999
Parents: 61d4302
Author: etherge <et...@163.com>
Authored: Mon Jan 16 12:05:07 2017 +0800
Committer: Billy Liu <bi...@apache.org>
Committed: Mon Jan 16 13:49:39 2017 +0800

----------------------------------------------------------------------
 build/deploy/context.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/4bc31de9/build/deploy/context.xml
----------------------------------------------------------------------
diff --git a/build/deploy/context.xml b/build/deploy/context.xml
index e8b4cf1..da4777b 100644
--- a/build/deploy/context.xml
+++ b/build/deploy/context.xml
@@ -28,4 +28,5 @@
     <Manager pathname="" />
     -->
     <Loader loaderClass="org.apache.kylin.ext.CustomizedWebappClassloader"/>
+    <JarScanner scanClassPath="false" />
 </Context>


[07/21] kylin git commit: minor, refactor package scripts abour download tomcat and spark

Posted by li...@apache.org.
minor, refactor package scripts abour download tomcat and spark


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

Branch: refs/heads/master-cdh5.7
Commit: 20c329423f75df63bcc3d6dac7883fe12288f668
Parents: d19533c
Author: lidongsjtu <li...@apache.org>
Authored: Sat Jan 14 16:19:59 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Sat Jan 14 16:19:59 2017 +0800

----------------------------------------------------------------------
 build/script/download-spark.sh  | 10 ++++--
 build/script/download-tomcat.sh | 13 +++++---
 build/script/functions.sh       | 60 ------------------------------------
 3 files changed, 15 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/20c32942/build/script/download-spark.sh
----------------------------------------------------------------------
diff --git a/build/script/download-spark.sh b/build/script/download-spark.sh
index ad9651d..3ea7d08 100755
--- a/build/script/download-spark.sh
+++ b/build/script/download-spark.sh
@@ -20,10 +20,13 @@
 dir=$(dirname ${0})
 cd ${dir}/../..
 
-source build/script/functions.sh
-
 rm -rf build/spark
 
+alias md5cmd="md5sum"
+if [[ `uname -a` =~ "Darwin" ]]; then
+    alias md5cmd="md5 -q"
+fi
+
 spark_version="1.6.3"
 spark_pkg_md5="ce8a2e7529aac0f0175194061769dbd4"
 
@@ -32,7 +35,7 @@ then
     echo "no binary file found"
     wget --directory-prefix=build/ http://archive.apache.org/dist/spark/spark-${spark_version}/spark-${spark_version}-bin-hadoop2.6.tgz || echo "Download spark failed"
 else
-    if [ `calMd5 build/spark-${spark_version}-bin-hadoop2.6.tgz | awk '{print $1}'` != "${spark_pkg_md5}" ]
+    if [ `md5cmd build/spark-${spark_version}-bin-hadoop2.6.tgz | awk '{print $1}'` != "${spark_pkg_md5}" ]
     then
         echo "md5 check failed"
         rm build/spark-${spark_version}-bin-hadoop2.6.tgz
@@ -40,6 +43,7 @@ else
 
     fi
 fi
+unalias md5cmd
 
 tar -zxvf build/spark-${spark_version}-bin-hadoop2.6.tgz -C build/   || { exit 1; }
 mv build/spark-${spark_version}-bin-hadoop2.6 build/spark

http://git-wip-us.apache.org/repos/asf/kylin/blob/20c32942/build/script/download-tomcat.sh
----------------------------------------------------------------------
diff --git a/build/script/download-tomcat.sh b/build/script/download-tomcat.sh
index c97c3ce..403d87b 100755
--- a/build/script/download-tomcat.sh
+++ b/build/script/download-tomcat.sh
@@ -27,16 +27,19 @@ if [[ `uname -a` =~ "Darwin" ]]; then
     alias md5cmd="md5 -q"
 fi
 
-if [ ! -f "build/apache-tomcat-8.5.9.tar.gz" ]
+tomcat_pkg_version="8.5.9"
+tomcat_pkg_md5="b41270a64b7774c964e4bec813eea2ed"
+
+if [ ! -f "build/apache-tomcat-${tomcat_pkg_version}.tar.gz" ]
 then
     echo "no binary file found"
-    wget --directory-prefix=build/ http://archive.apache.org/dist/tomcat/tomcat-8/v8.5.9/bin/apache-tomcat-8.5.9.tar.gz || echo "download tomcat failed"
+    wget --directory-prefix=build/ http://archive.apache.org/dist/tomcat/tomcat-8/v${tomcat_pkg_version}/bin/apache-tomcat-${tomcat_pkg_version}.tar.gz || echo "Download tomcat failed"
 else
-    if [ `md5cmd build/apache-tomcat-8.5.9.tar.gz | awk '{print $1}'` != "b41270a64b7774c964e4bec813eea2ed" ]
+    if [ `md5cmd build/apache-tomcat-${tomcat_pkg_version}.tar.gz | awk '{print $1}'` != "${tomcat_pkg_md5}" ]
     then
         echo "md5 check failed"
-        rm build/apache-tomcat-8.5.9.tar.gz
-        wget --directory-prefix=build/ http://archive.apache.org/dist/tomcat/tomcat-8/v8.5.9/bin/apache-tomcat-8.5.9.tar.g || echo "download tomcat failed"
+        rm build/apache-tomcat-${tomcat_pkg_version}.tar.gz
+        wget --directory-prefix=build/ http://archive.apache.org/dist/tomcat/tomcat-8/v${tomcat_pkg_version}/bin/apache-tomcat-${tomcat_pkg_version}.tar.gz || echo "download tomcat failed"
     fi
 fi
 unalias md5cmd

http://git-wip-us.apache.org/repos/asf/kylin/blob/20c32942/build/script/functions.sh
----------------------------------------------------------------------
diff --git a/build/script/functions.sh b/build/script/functions.sh
deleted file mode 100755
index 2eed617..0000000
--- a/build/script/functions.sh
+++ /dev/null
@@ -1,60 +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.
-#
-
-function checkCommandExits() {
-    echo "Checking ${1}..."
-    if [ -z "$(command -v ${1})" ]
-    then
-        echo "Please install ${1} first so that Kylin packaging can proceed"
-        exit 1
-    else
-        echo "${1} check passed"
-    fi
-}
-
-function exportProjectVersions() {
-    if [ -z "${kylin_versoin}" ]; then
-        export kylin_version=`mvn org.apache.maven.plugins:maven-help-plugin:2.1.1:evaluate -Dexpression=project.version -f kylin | grep -Ev '(^\[|Download\w+:)'`
-        echo "Apache Kylin Version: ${kylin_version}"
-    fi
-    if [ -z "${release_version}" ]; then
-        export release_version=$kap_version
-    fi
-}
-
-function detectOSType() {
-    OS_TYPE="linux"
-    if [[ `uname -a` =~ "Darwin" ]]; then
-        OS_TYPE="mac"
-    elif [[ `uname -a` =~ "Cygwin" ]]; then
-        OS_TYPE="windows"
-    fi
-    echo $OS_TYPE
-}
-
-function calMd5() {
-    OS_TYPE=`detectOSType`
-    if [[ "$OS_TYPE" == "mac" ]]; then
-        md5 -q $1
-    elif [[ "$OS_TYPE" == "windows" ]]; then
-        md5sum $1
-    else
-        md5sum $1
-    fi
-}
\ No newline at end of file


[21/21] kylin git commit: KYLIN-1672 support kylin on cdh 5.7

Posted by li...@apache.org.
KYLIN-1672 support kylin on cdh 5.7

Signed-off-by: Li Yang <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/cec70b82
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/cec70b82
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/cec70b82

Branch: refs/heads/master-cdh5.7
Commit: cec70b82e60731f1f414825555b32abd59a38577
Parents: a34f815
Author: Lynne Jiang <ly...@hotmail.com>
Authored: Mon May 16 03:33:27 2016 -0700
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Jan 17 11:30:20 2017 +0800

----------------------------------------------------------------------
 .../kylin/engine/mr/steps/MockupMapContext.java |  13 +-
 examples/test_case_data/sandbox/core-site.xml   | 146 +++---
 examples/test_case_data/sandbox/hbase-site.xml  | 162 ++----
 examples/test_case_data/sandbox/hdfs-site.xml   | 259 ++--------
 examples/test_case_data/sandbox/mapred-site.xml | 398 ++++++---------
 examples/test_case_data/sandbox/yarn-site.xml   | 496 ++-----------------
 pom.xml                                         |  16 +-
 server/pom.xml                                  |  36 ++
 .../storage/hbase/steps/MockupMapContext.java   |  29 +-
 tool/pom.xml                                    |  12 +
 10 files changed, 430 insertions(+), 1137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/cec70b82/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
index dfbba14..bc9601f 100644
--- a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
@@ -44,10 +44,10 @@ import org.apache.kylin.engine.mr.common.BatchConstants;
 
 /**
  * @author yangli9
- * 
+ *
  */
 @SuppressWarnings({ "rawtypes", "unchecked" })
-public class MockupMapContext implements MapContext{
+public class MockupMapContext implements MapContext {
 
     private Configuration hconf;
 
@@ -60,7 +60,7 @@ public class MockupMapContext implements MapContext{
         return new WrappedMapper().getMapContext(new MockupMapContext(hconf, outKV));
     }
 
-    public MockupMapContext(Configuration hconf, Object[] outKV){
+    public MockupMapContext(Configuration hconf, Object[] outKV) {
         this.hconf = hconf;
         this.outKV = outKV;
     }
@@ -175,6 +175,11 @@ public class MockupMapContext implements MapContext{
     }
 
     @Override
+    public boolean userClassesTakesPrecedence() {
+        throw new NotImplementedException();
+    }
+
+    @Override
     public Class<? extends InputFormat<?, ?>> getInputFormatClass() throws ClassNotFoundException {
         throw new NotImplementedException();
     }
@@ -318,4 +323,4 @@ public class MockupMapContext implements MapContext{
     public RawComparator<?> getCombinerKeyGroupingComparator() {
         throw new NotImplementedException();
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/cec70b82/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 9aa588c..6162406 100644
--- a/examples/test_case_data/sandbox/core-site.xml
+++ b/examples/test_case_data/sandbox/core-site.xml
@@ -14,152 +14,146 @@
   See the License for the specific language governing permissions and
   limitations under the License.
 -->
+<!--Autogenerated by Cloudera Manager-->
 <configuration>
-
     <property>
         <name>fs.defaultFS</name>
-        <value>hdfs://sandbox.hortonworks.com:8020</value>
-        <final>true</final>
+        <value>hdfs://quickstart.cloudera:8020</value>
     </property>
-
     <property>
         <name>fs.trash.interval</name>
-        <value>360</value>
+        <value>1</value>
     </property>
-
     <property>
-        <name>ha.failover-controller.active-standby-elector.zk.op.retries</name>
-        <value>120</value>
+        <name>io.compression.codecs</name>
+        <value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec,org.apache.hadoop.io.compress.DeflateCodec,org.apache.hadoop.io.compress.SnappyCodec,org.apache.hadoop.io.compress.Lz4Codec</value>
     </property>
-
     <property>
-        <name>hadoop.http.authentication.simple.anonymous.allowed</name>
-        <value>true</value>
+        <name>hadoop.security.authentication</name>
+        <value>simple</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.falcon.groups</name>
-        <value>users</value>
+        <name>hadoop.security.authorization</name>
+        <value>false</value>
+    </property>
+    <property>
+        <name>hadoop.rpc.protection</name>
+        <value>authentication</value>
+    </property>
+    <property>
+        <name>hadoop.security.auth_to_local</name>
+        <value>DEFAULT</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.falcon.hosts</name>
+        <name>hadoop.proxyuser.oozie.hosts</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.hbase.groups</name>
-        <value>users</value>
+        <name>hadoop.proxyuser.oozie.groups</name>
+        <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.hbase.hosts</name>
+        <name>hadoop.proxyuser.mapred.hosts</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.hcat.groups</name>
+        <name>hadoop.proxyuser.mapred.groups</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.hcat.hosts</name>
-        <value>sandbox.hortonworks.com</value>
+        <name>hadoop.proxyuser.flume.hosts</name>
+        <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.hive.groups</name>
-        <value>users</value>
+        <name>hadoop.proxyuser.flume.groups</name>
+        <value>*</value>
+    </property>
+    <property>
+        <name>hadoop.proxyuser.HTTP.hosts</name>
+        <value>*</value>
+    </property>
+    <property>
+        <name>hadoop.proxyuser.HTTP.groups</name>
+        <value>*</value>
     </property>
-
     <property>
         <name>hadoop.proxyuser.hive.hosts</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.hue.groups</name>
+        <name>hadoop.proxyuser.hive.groups</name>
         <value>*</value>
     </property>
-
     <property>
         <name>hadoop.proxyuser.hue.hosts</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.oozie.groups</name>
+        <name>hadoop.proxyuser.hue.groups</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.oozie.hosts</name>
-        <value>sandbox.hortonworks.com</value>
+        <name>hadoop.proxyuser.httpfs.hosts</name>
+        <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.root.groups</name>
+        <name>hadoop.proxyuser.httpfs.groups</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.proxyuser.root.hosts</name>
+        <name>hadoop.proxyuser.hdfs.groups</name>
         <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.security.auth_to_local</name>
-        <value>DEFAULT</value>
+        <name>hadoop.proxyuser.hdfs.hosts</name>
+        <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.security.authentication</name>
-        <value>simple</value>
+        <name>hadoop.proxyuser.yarn.hosts</name>
+        <value>*</value>
     </property>
-
     <property>
-        <name>hadoop.security.authorization</name>
-        <value>false</value>
+        <name>hadoop.proxyuser.yarn.groups</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>
+        <name>hadoop.security.group.mapping</name>
+        <value>org.apache.hadoop.security.ShellBasedUnixGroupsMapping</value>
     </property>
-
     <property>
-        <name>io.file.buffer.size</name>
-        <value>131072</value>
+        <name>hadoop.security.instrumentation.requires.admin</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>io.serializations</name>
-        <value>org.apache.hadoop.io.serializer.WritableSerialization</value>
+        <name>net.topology.script.file.name</name>
+        <value>/etc/hadoop/conf.cloudera.yarn/topology.py</value>
     </property>
-
     <property>
-        <name>ipc.client.connect.max.retries</name>
-        <value>50</value>
+        <name>io.file.buffer.size</name>
+        <value>65536</value>
     </property>
-
     <property>
-        <name>ipc.client.connection.maxidletime</name>
-        <value>30000</value>
+        <name>hadoop.ssl.enabled</name>
+        <value>false</value>
+    </property>
+    <property>
+        <name>hadoop.ssl.require.client.cert</name>
+        <value>false</value>
+        <final>true</final>
     </property>
-
     <property>
-        <name>ipc.client.idlethreshold</name>
-        <value>8000</value>
+        <name>hadoop.ssl.keystores.factory.class</name>
+        <value>org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory</value>
+        <final>true</final>
     </property>
-
     <property>
-        <name>ipc.server.tcpnodelay</name>
-        <value>true</value>
+        <name>hadoop.ssl.server.conf</name>
+        <value>ssl-server.xml</value>
+        <final>true</final>
     </property>
-
     <property>
-        <name>mapreduce.jobtracker.webinterface.trusted</name>
-        <value>false</value>
+        <name>hadoop.ssl.client.conf</name>
+        <value>ssl-client.xml</value>
+        <final>true</final>
     </property>
-
 </configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/cec70b82/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 734908e..58c6223 100644
--- a/examples/test_case_data/sandbox/hbase-site.xml
+++ b/examples/test_case_data/sandbox/hbase-site.xml
@@ -15,180 +15,104 @@
   limitations under the License.
 -->
 <configuration>
-
-    <property>
-        <name>dfs.domain.socket.path</name>
-        <value>/var/lib/hadoop-hdfs/dn_socket</value>
-    </property>
-
     <property>
-        <name>hbase.client.keyvalue.maxsize</name>
-        <value>10485760</value>
-    </property>
-
-    <property>
-        <name>hbase.client.scanner.caching</name>
-        <value>100</value>
+        <name>hbase.rootdir</name>
+        <value>hdfs://quickstart.cloudera:8020/hbase</value>
     </property>
-
     <property>
-        <name>hbase.cluster.distributed</name>
+        <name>hbase.replication</name>
         <value>true</value>
     </property>
-
-    <property>
-        <name>hbase.coprocessor.master.classes</name>
-        <value>com.xasecure.authorization.hbase.XaSecureAuthorizationCoprocessor</value>
-    </property>
-
     <property>
-        <name>hbase.coprocessor.region.classes</name>
-        <value>com.xasecure.authorization.hbase.XaSecureAuthorizationCoprocessor</value>
+        <name>hbase.client.write.buffer</name>
+        <value>2097152</value>
     </property>
-
-    <property>
-        <name>hbase.defaults.for.version.skip</name>
-        <value>true</value>
-    </property>
-
     <property>
-        <name>hbase.hregion.majorcompaction</name>
-        <value>604800000</value>
+        <name>hbase.client.pause</name>
+        <value>100</value>
     </property>
-
     <property>
-        <name>hbase.hregion.majorcompaction.jitter</name>
-        <value>0.50</value>
+        <name>hbase.client.retries.number</name>
+        <value>35</value>
     </property>
-
     <property>
-        <name>hbase.hregion.max.filesize</name>
-        <value>10737418240</value>
+        <name>hbase.client.scanner.caching</name>
+        <value>100</value>
     </property>
-
     <property>
-        <name>hbase.hregion.memstore.block.multiplier</name>
-        <value>4</value>
+        <name>hbase.client.keyvalue.maxsize</name>
+        <value>10485760</value>
     </property>
-
     <property>
-        <name>hbase.hregion.memstore.flush.size</name>
-        <value>134217728</value>
-    </property>
-
-    <property>
-        <name>hbase.hregion.memstore.mslab.enabled</name>
+        <name>hbase.ipc.client.allowsInterrupt</name>
         <value>true</value>
     </property>
-
     <property>
-        <name>hbase.hstore.blockingStoreFiles</name>
+        <name>hbase.client.primaryCallTimeout.get</name>
         <value>10</value>
     </property>
-
-    <property>
-        <name>hbase.hstore.compactionThreshold</name>
-        <value>3</value>
-    </property>
-
     <property>
-        <name>hbase.local.dir</name>
-        <value>${hbase.tmp.dir}/local</value>
+        <name>hbase.client.primaryCallTimeout.multiget</name>
+        <value>10</value>
     </property>
-
     <property>
-        <name>hbase.master.info.bindAddress</name>
-        <value>0.0.0.0</value>
+        <name>hbase.regionserver.thrift.http</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>hbase.master.info.port</name>
-        <value>60010</value>
+        <name>hbase.thrift.support.proxyuser</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>hbase.master.port</name>
+        <name>hbase.rpc.timeout</name>
         <value>60000</value>
     </property>
-
-    <property>
-        <name>hbase.regionserver.global.memstore.lowerLimit</name>
-        <value>0.38</value>
-    </property>
-
     <property>
-        <name>hbase.regionserver.global.memstore.upperLimit</name>
-        <value>0.4</value>
-    </property>
-
-    <property>
-        <name>hbase.regionserver.handler.count</name>
-        <value>60</value>
+        <name>hbase.snapshot.enabled</name>
+        <value>true</value>
     </property>
-
     <property>
-        <name>hbase.regionserver.info.port</name>
-        <value>60030</value>
+        <name>hbase.snapshot.master.timeoutMillis</name>
+        <value>60000</value>
     </property>
-
     <property>
-        <name>hbase.rootdir</name>
-        <value>hdfs://sandbox.hortonworks.com:8020/apps/hbase/data</value>
+        <name>hbase.snapshot.region.timeout</name>
+        <value>60000</value>
     </property>
-
     <property>
-        <name>hbase.rpc.protection</name>
-        <value>PRIVACY</value>
+        <name>hbase.snapshot.master.timeout.millis</name>
+        <value>60000</value>
     </property>
-
     <property>
         <name>hbase.security.authentication</name>
         <value>simple</value>
     </property>
-
     <property>
-        <name>hbase.security.authorization</name>
-        <value>true</value>
+        <name>hbase.rpc.protection</name>
+        <value>authentication</value>
     </property>
-
     <property>
-        <name>hbase.superuser</name>
-        <value>hbase</value>
+        <name>zookeeper.session.timeout</name>
+        <value>60000</value>
     </property>
-
     <property>
-        <name>hbase.tmp.dir</name>
-        <value>/hadoop/hbase</value>
+        <name>zookeeper.znode.parent</name>
+        <value>/hbase</value>
     </property>
-
     <property>
-        <name>hbase.zookeeper.property.clientPort</name>
-        <value>2181</value>
+        <name>zookeeper.znode.rootserver</name>
+        <value>root-region-server</value>
     </property>
-
     <property>
         <name>hbase.zookeeper.quorum</name>
-        <value>sandbox.hortonworks.com</value>
+        <value>quickstart.cloudera</value>
     </property>
-
     <property>
-        <name>hbase.zookeeper.useMulti</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>hfile.block.cache.size</name>
-        <value>0.40</value>
-    </property>
-
-    <property>
-        <name>zookeeper.session.timeout</name>
-        <value>30000</value>
+        <name>hbase.zookeeper.property.clientPort</name>
+        <value>2181</value>
     </property>
-
     <property>
-        <name>zookeeper.znode.parent</name>
-        <value>/hbase-unsecure</value>
+        <name>hbase.rest.ssl.enabled</name>
+        <value>false</value>
     </property>
-
 </configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/cec70b82/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 1175fff..05854bd 100644
--- a/examples/test_case_data/sandbox/hdfs-site.xml
+++ b/examples/test_case_data/sandbox/hdfs-site.xml
@@ -15,271 +15,68 @@
   limitations under the License.
 -->
 <configuration>
-
-    <property>
-        <name>dfs.block.access.token.enable</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>dfs.block.size</name>
-        <value>34217472</value>
-    </property>
-
-    <property>
-        <name>dfs.blockreport.initialDelay</name>
-        <value>120</value>
-    </property>
-
-    <property>
-        <name>dfs.blocksize</name>
-        <value>134217728</value>
-    </property>
-
-    <property>
-        <name>dfs.client.read.shortcircuit</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>dfs.client.read.shortcircuit.streams.cache.size</name>
-        <value>4096</value>
-    </property>
-
-    <property>
-        <name>dfs.cluster.administrators</name>
-        <value>hdfs</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.address</name>
-        <value>0.0.0.0:50010</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.balance.bandwidthPerSec</name>
-        <value>6250000</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.data.dir</name>
-        <value>/hadoop/hdfs/data</value>
-        <final>true</final>
-    </property>
-
-    <property>
-        <name>dfs.datanode.data.dir.perm</name>
-        <value>750</value>
-    </property>
-
     <property>
-        <name>dfs.datanode.du.reserved</name>
-        <value>1073741824</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.failed.volumes.tolerated</name>
-        <value>0</value>
-        <final>true</final>
-    </property>
-
-    <property>
-        <name>dfs.datanode.http.address</name>
-        <value>0.0.0.0:50075</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.https.address</name>
-        <value>0.0.0.0:50475</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.ipc.address</name>
-        <value>0.0.0.0:8010</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.max.transfer.threads</name>
-        <value>1024</value>
-    </property>
-
-    <property>
-        <name>dfs.datanode.max.xcievers</name>
-        <value>1024</value>
-    </property>
-
-    <property>
-        <name>dfs.domain.socket.path</name>
-        <value>/var/lib/hadoop-hdfs/dn_socket</value>
-    </property>
-
-    <property>
-        <name>dfs.heartbeat.interval</name>
-        <value>3</value>
+        <name>dfs.namenode.name.dir</name>
+        <value>file:///var/lib/hadoop-hdfs/cache/hdfs/dfs/name</value>
     </property>
-
     <property>
-        <name>dfs.hosts.exclude</name>
-        <value>/etc/hadoop/conf/dfs.exclude</value>
+        <name>dfs.namenode.servicerpc-address</name>
+        <value>quickstart.cloudera:8022</value>
     </property>
-
     <property>
-        <name>dfs.http.policy</name>
-        <value>HTTP_ONLY</value>
+        <name>dfs.https.address</name>
+        <value>quickstart.cloudera:50470</value>
     </property>
-
     <property>
         <name>dfs.https.port</name>
         <value>50470</value>
     </property>
-
-    <property>
-        <name>dfs.journalnode.edits.dir</name>
-        <value>/hadoop/hdfs/journalnode</value>
-    </property>
-
-    <property>
-        <name>dfs.journalnode.http-address</name>
-        <value>0.0.0.0:8480</value>
-    </property>
-
-    <property>
-        <name>dfs.journalnode.https-address</name>
-        <value>0.0.0.0:8481</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.accesstime.precision</name>
-        <value>3600000</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.avoid.read.stale.datanode</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.avoid.write.stale.datanode</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.checkpoint.dir</name>
-        <value>/hadoop/hdfs/namesecondary</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.checkpoint.edits.dir</name>
-        <value>${dfs.namenode.checkpoint.dir}</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.checkpoint.period</name>
-        <value>21600</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.checkpoint.txns</name>
-        <value>1000000</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.handler.count</name>
-        <value>100</value>
-    </property>
-
     <property>
         <name>dfs.namenode.http-address</name>
-        <value>sandbox.hortonworks.com:50070</value>
-        <final>true</final>
-    </property>
-
-    <property>
-        <name>dfs.namenode.https-address</name>
-        <value>sandbox.hortonworks.com:50470</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.name.dir</name>
-        <value>/hadoop/hdfs/namenode</value>
-        <final>true</final>
-    </property>
-
-    <property>
-        <name>dfs.namenode.name.dir.restore</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>dfs.namenode.safemode.threshold-pct</name>
-        <value>1.0f</value>
+        <value>quickstart.cloudera:50070</value>
     </property>
-
     <property>
-        <name>dfs.namenode.secondary.http-address</name>
-        <value>sandbox.hortonworks.com:50090</value>
+        <name>dfs.replication</name>
+        <value>1</value>
     </property>
-
     <property>
-        <name>dfs.namenode.stale.datanode.interval</name>
-        <value>30000</value>
+        <name>dfs.blocksize</name>
+        <value>134217728</value>
     </property>
-
     <property>
-        <name>dfs.namenode.startup.delay.block.deletion.sec</name>
-        <value>3600</value>
+        <name>dfs.client.use.datanode.hostname</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>dfs.namenode.write.stale.datanode.ratio</name>
-        <value>1.0f</value>
+        <name>fs.permissions.umask-mode</name>
+        <value>022</value>
     </property>
-
     <property>
-        <name>dfs.nfs.exports.allowed.hosts</name>
-        <value>* rw</value>
+        <name>dfs.namenode.acls.enabled</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>dfs.nfs3.dump.dir</name>
-        <value>/tmp/.hdfs-nfs</value>
+        <name>dfs.client.use.legacy.blockreader</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>dfs.permissions.enabled</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>dfs.permissions.superusergroup</name>
-        <value>hdfs</value>
+        <name>dfs.client.read.shortcircuit</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>dfs.replication</name>
-        <value>1</value>
+        <name>dfs.domain.socket.path</name>
+        <value>/var/run/hdfs-sockets/dn</value>
     </property>
-
     <property>
-        <name>dfs.replication.max</name>
-        <value>50</value>
+        <name>dfs.client.read.shortcircuit.skip.checksum</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>dfs.support.append</name>
-        <value>true</value>
-        <final>true</final>
+        <name>dfs.client.domain.socket.data.traffic</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>dfs.webhdfs.enabled</name>
+        <name>dfs.datanode.hdfs-blocks-metadata.enabled</name>
         <value>true</value>
-        <final>true</final>
     </property>
-
-    <property>
-        <name>fs.permissions.umask-mode</name>
-        <value>022</value>
-    </property>
-
 </configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/cec70b82/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 e90f594..c9b1ca4 100644
--- a/examples/test_case_data/sandbox/mapred-site.xml
+++ b/examples/test_case_data/sandbox/mapred-site.xml
@@ -15,241 +15,165 @@
   limitations under the License.
 -->
 <configuration>
-
-    <property>
-        <name>io.sort.mb</name>
-        <value>128</value>
-    </property>
-
-    <property>
-        <name>mapred.child.java.opts</name>
-        <value>-Xmx200m</value>
-    </property>
-
-    <property>
-        <name>mapreduce.map.memory.mb</name>
-        <value>512</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.memory.mb</name>
-        <value>512</value>
-    </property>
-
-    <property>
-        <name>mapreduce.admin.map.child.java.opts</name>
-        <value>-server -XX:NewRatio=8 -Djava.net.preferIPv4Stack=true -Dhdp.version=${hdp.version}</value>
-    </property>
-
-    <property>
-        <name>mapreduce.admin.reduce.child.java.opts</name>
-        <value>-server -XX:NewRatio=8 -Djava.net.preferIPv4Stack=true -Dhdp.version=${hdp.version}</value>
-    </property>
-
-    <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>
-    </property>
-
-    <property>
-        <name>mapreduce.am.max-attempts</name>
-        <value>2</value>
-    </property>
-
-    <property>
-        <name>mapreduce.application.classpath</name>
-        <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>
-        <name>mapreduce.application.framework.path</name>
-        <value>/hdp/apps/${hdp.version}/mapreduce/mapreduce.tar.gz#mr-framework</value>
-    </property>
-
-    <property>
-        <name>mapreduce.cluster.administrators</name>
-        <value>hadoop</value>
-    </property>
-
-    <property>
-        <name>mapreduce.framework.name</name>
-        <value>yarn</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>1</value>
-    </property>
-
-    <property>
-        <name>mapreduce.jobhistory.address</name>
-        <value>sandbox.hortonworks.com:10020</value>
-    </property>
-
-    <property>
-        <name>mapreduce.jobhistory.bind-host</name>
-        <value>0.0.0.0</value>
-    </property>
-
-    <property>
-        <name>mapreduce.jobhistory.done-dir</name>
-        <value>/mr-history/done</value>
-    </property>
-
-    <property>
-        <name>mapreduce.jobhistory.intermediate-done-dir</name>
-        <value>/mr-history/tmp</value>
-    </property>
-
-    <property>
-        <name>mapreduce.jobhistory.webapp.address</name>
-        <value>sandbox.hortonworks.com:19888</value>
-    </property>
-
-    <property>
-        <name>mapreduce.map.java.opts</name>
-        <value>-Xmx512m</value>
-    </property>
-
-    <property>
-        <name>mapreduce.map.log.level</name>
-        <value>INFO</value>
-    </property>
-
-    <property>
-        <name>mapreduce.map.memory.mb</name>
-        <value>512</value>
-    </property>
-
-    <property>
-        <name>mapreduce.map.output.compress</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>mapreduce.map.sort.spill.percent</name>
-        <value>0.7</value>
-    </property>
-
-    <property>
-        <name>mapreduce.map.speculative</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>mapreduce.output.fileoutputformat.compress</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>mapreduce.output.fileoutputformat.compress.type</name>
-        <value>BLOCK</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.input.buffer.percent</name>
-        <value>0.0</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.java.opts</name>
-        <value>-Xmx200m</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.log.level</name>
-        <value>INFO</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.memory.mb</name>
-        <value>512</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.shuffle.fetch.retry.enabled</name>
-        <value>1</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.shuffle.fetch.retry.interval-ms</name>
-        <value>1000</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.shuffle.fetch.retry.timeout-ms</name>
-        <value>30000</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.shuffle.input.buffer.percent</name>
-        <value>0.7</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.shuffle.merge.percent</name>
-        <value>0.66</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.shuffle.parallelcopies</name>
-        <value>30</value>
-    </property>
-
-    <property>
-        <name>mapreduce.reduce.speculative</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>mapreduce.shuffle.port</name>
-        <value>13562</value>
-    </property>
-
-    <property>
-        <name>mapreduce.task.io.sort.factor</name>
-        <value>100</value>
-    </property>
-
-    <property>
-        <name>mapreduce.task.io.sort.mb</name>
-        <value>128</value>
-    </property>
-
-    <property>
-        <name>mapreduce.task.timeout</name>
-        <value>300000</value>
-    </property>
-
-    <property>
-        <name>yarn.app.mapreduce.am.admin-command-opts</name>
-        <value>-Dhdp.version=${hdp.version}</value>
-    </property>
-
-    <property>
-        <name>yarn.app.mapreduce.am.command-opts</name>
-        <value>-Xmx512m</value>
-    </property>
-
-    <property>
-        <name>yarn.app.mapreduce.am.log.level</name>
-        <value>INFO</value>
-    </property>
-
-    <property>
-        <name>yarn.app.mapreduce.am.resource.mb</name>
-        <value>512</value>
-    </property>
-
-    <property>
-        <name>yarn.app.mapreduce.am.staging-dir</name>
-        <value>/user</value>
-    </property>
-
+<property>
+    <name>mapreduce.job.split.metainfo.maxsize</name>
+    <value>10000000</value>
+</property>
+<property>
+    <name>mapreduce.job.counters.max</name>
+    <value>120</value>
+</property>
+<property>
+    <name>mapreduce.output.fileoutputformat.compress</name>
+    <value>false</value>
+</property>
+<property>
+    <name>mapreduce.output.fileoutputformat.compress.type</name>
+    <value>BLOCK</value>
+</property>
+<property>
+    <name>mapreduce.output.fileoutputformat.compress.codec</name>
+    <value>org.apache.hadoop.io.compress.DefaultCodec</value>
+</property>
+<property>
+    <name>mapreduce.map.output.compress.codec</name>
+    <value>org.apache.hadoop.io.compress.SnappyCodec</value>
+</property>
+<property>
+    <name>mapreduce.map.output.compress</name>
+    <value>true</value>
+</property>
+<property>
+    <name>zlib.compress.level</name>
+    <value>DEFAULT_COMPRESSION</value>
+</property>
+<property>
+    <name>mapreduce.task.io.sort.factor</name>
+    <value>64</value>
+</property>
+<property>
+    <name>mapreduce.map.sort.spill.percent</name>
+    <value>0.8</value>
+</property>
+<property>
+    <name>mapreduce.reduce.shuffle.parallelcopies</name>
+    <value>10</value>
+</property>
+<property>
+    <name>mapreduce.task.timeout</name>
+    <value>600000</value>
+</property>
+<property>
+    <name>mapreduce.client.submit.file.replication</name>
+    <value>1</value>
+</property>
+<property>
+    <name>mapreduce.job.reduces</name>
+    <value>1</value>
+</property>
+<property>
+    <name>mapreduce.task.io.sort.mb</name>
+    <value>16</value>
+</property>
+<property>
+    <name>mapreduce.map.speculative</name>
+    <value>false</value>
+</property>
+<property>
+    <name>mapreduce.reduce.speculative</name>
+    <value>false</value>
+</property>
+<property>
+    <name>mapreduce.job.reduce.slowstart.completedmaps</name>
+    <value>0.8</value>
+</property>
+<property>
+    <name>mapreduce.jobhistory.address</name>
+    <value>quickstart.cloudera:10020</value>
+</property>
+<property>
+    <name>mapreduce.jobhistory.webapp.address</name>
+    <value>quickstart.cloudera:19888</value>
+</property>
+<property>
+    <name>mapreduce.jobhistory.webapp.https.address</name>
+    <value>quickstart.cloudera:19890</value>
+</property>
+<property>
+    <name>mapreduce.jobhistory.admin.address</name>
+    <value>quickstart.cloudera:10033</value>
+</property>
+<property>
+    <name>mapreduce.framework.name</name>
+    <value>yarn</value>
+</property>
+<property>
+    <name>yarn.app.mapreduce.am.staging-dir</name>
+    <value>/user</value>
+</property>
+<property>
+    <name>mapreduce.am.max-attempts</name>
+    <value>2</value>
+</property>
+<property>
+    <name>yarn.app.mapreduce.am.resource.mb</name>
+    <value>128</value>
+</property>
+<property>
+    <name>yarn.app.mapreduce.am.resource.cpu-vcores</name>
+    <value>1</value>
+</property>
+<property>
+    <name>mapreduce.job.ubertask.enable</name>
+    <value>false</value>
+</property>
+<property>
+    <name>yarn.app.mapreduce.am.command-opts</name>
+    <value>-Djava.net.preferIPv4Stack=true -Xmx52428800</value>
+</property>
+<property>
+    <name>mapreduce.map.java.opts</name>
+    <value>-Djava.net.preferIPv4Stack=true -Xmx52428800</value>
+</property>
+<property>
+    <name>mapreduce.reduce.java.opts</name>
+    <value>-Djava.net.preferIPv4Stack=true -Xmx52428800</value>
+</property>
+<property>
+    <name>yarn.app.mapreduce.am.admin.user.env</name>
+    <value>LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native:$JAVA_LIBRARY_PATH</value>
+</property>
+<property>
+    <name>mapreduce.map.memory.mb</name>
+    <value>128</value>
+</property>
+<property>
+    <name>mapreduce.map.cpu.vcores</name>
+    <value>1</value>
+</property>
+<property>
+    <name>mapreduce.reduce.memory.mb</name>
+    <value>128</value>
+</property>
+<property>
+    <name>mapreduce.reduce.cpu.vcores</name>
+    <value>1</value>
+</property>
+<property>
+    <name>mapreduce.job.heap.memory-mb.ratio</name>
+    <value>0.8</value>
+</property>
+<property>
+    <name>mapreduce.application.classpath</name>
+    <value>/tmp/kylin/*,/usr/lib/hadoop-mapreduce/lib/*,/etc/hadoop/conf:/usr/lib/hadoop/lib/*:/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/lib/*:/usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop-yarn/.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*,/usr/lib/hbase/hbase-common.jar,/etc/hive/conf</value>
+</property>
+<property>
+    <name>mapreduce.admin.user.env</name>
+    <value>LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native:$JAVA_LIBRARY_PATH</value>
+</property>
+<property>
+    <name>mapreduce.shuffle.max.connections</name>
+    <value>80</value>
+</property>
 </configuration>
+

http://git-wip-us.apache.org/repos/asf/kylin/blob/cec70b82/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 8256158..8988d4a 100644
--- a/examples/test_case_data/sandbox/yarn-site.xml
+++ b/examples/test_case_data/sandbox/yarn-site.xml
@@ -15,520 +15,128 @@
   limitations under the License.
 -->
 <configuration>
-
-    <property>
-        <name>hadoop.registry.rm.enabled</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>hadoop.registry.zk.quorum</name>
-        <value>sandbox.hortonworks.com:2181</value>
-    </property>
-
     <property>
         <name>yarn.acl.enable</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>yarn.admin.acl</name>
-        <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.client.nodemanager-connect.max-wait-ms</name>
-        <value>60000</value>
-    </property>
-
-    <property>
-        <name>yarn.client.nodemanager-connect.retry-interval-ms</name>
-        <value>10000</value>
-    </property>
-
-    <property>
-        <name>yarn.http.policy</name>
-        <value>HTTP_ONLY</value>
-    </property>
-
-    <property>
-        <name>yarn.log-aggregation-enable</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>yarn.log-aggregation.retain-seconds</name>
-        <value>2592000</value>
-    </property>
-
-    <property>
-        <name>yarn.log.server.url</name>
-        <value>http://sandbox.hortonworks.com:19888/jobhistory/logs</value>
-    </property>
-
-    <property>
-        <name>yarn.node-labels.fs-store.retry-policy-spec</name>
-        <value>2000, 500</value>
-    </property>
-
-    <property>
-        <name>yarn.node-labels.fs-store.root-dir</name>
-        <value>/system/yarn/node-labels</value>
-    </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>
-
-    <property>
-        <name>yarn.nodemanager.admin-env</name>
-        <value>MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.aux-services</name>
-        <value>mapreduce_shuffle</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.aux-services.mapreduce_shuffle.class</name>
-        <value>org.apache.hadoop.mapred.ShuffleHandler</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.bind-host</name>
-        <value>0.0.0.0</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.container-executor.class</name>
-        <value>org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.container-monitor.interval-ms</name>
-        <value>3000</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.delete.debug-delay-sec</name>
-        <value>0</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage</name>
-        <value>90</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb</name>
-        <value>1000</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.disk-health-checker.min-healthy-disks</name>
-        <value>0.25</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.health-checker.interval-ms</name>
-        <value>135000</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.health-checker.script.timeout-ms</name>
-        <value>60000</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.linux-container-executor.cgroups.hierarchy</name>
-        <value>hadoop-yarn</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.linux-container-executor.cgroups.mount</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.linux-container-executor.group</name>
-        <value>hadoop</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.linux-container-executor.resources-handler.class</name>
-        <value>org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.local-dirs</name>
-        <value>/hadoop/yarn/local</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.log-aggregation.compression-type</name>
-        <value>gz</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.log-aggregation.debug-enabled</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.log-aggregation.num-log-files-per-app</name>
-        <value>30</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds</name>
-        <value>-1</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.log-dirs</name>
-        <value>/hadoop/yarn/log</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.log.retain-second</name>
-        <value>604800</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.pmem-check-enabled</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.recovery.dir</name>
-        <value>/var/log/hadoop-yarn/nodemanager/recovery-state</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.recovery.enabled</name>
         <value>true</value>
     </property>
-
-    <property>
-        <name>yarn.nodemanager.remote-app-log-dir</name>
-        <value>/app-logs</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.remote-app-log-dir-suffix</name>
-        <value>logs</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.resource.cpu-vcores</name>
-        <value>8</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.resource.memory-mb</name>
-        <value>9216</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.resource.percentage-physical-cpu-limit</name>
-        <value>100</value>
-    </property>
-
-    <property>
-        <name>yarn.nodemanager.vmem-check-enabled</name>
-        <value>false</value>
-    </property>
-
     <property>
-        <name>yarn.nodemanager.vmem-pmem-ratio</name>
-        <value>10</value>
+        <name>yarn.admin.acl</name>
+        <value>*</value>
     </property>
-
     <property>
         <name>yarn.resourcemanager.address</name>
-        <value>sandbox.hortonworks.com:8050</value>
+        <value>quickstart.cloudera:8032</value>
     </property>
-
     <property>
         <name>yarn.resourcemanager.admin.address</name>
-        <value>sandbox.hortonworks.com:8141</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.am.max-attempts</name>
-        <value>2</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.bind-host</name>
-        <value>0.0.0.0</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.connect.max-wait.ms</name>
-        <value>900000</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.connect.retry-interval.ms</name>
-        <value>30000</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.fs.state-store.retry-policy-spec</name>
-        <value>2000, 500</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.fs.state-store.uri</name>
-        <value></value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.ha.enabled</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.hostname</name>
-        <value>sandbox.hortonworks.com</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.nodes.exclude-path</name>
-        <value>/etc/hadoop/conf/yarn.exclude</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.recovery.enabled</name>
-        <value>true</value>
+        <value>quickstart.cloudera:8033</value>
     </property>
-
-    <property>
-        <name>yarn.resourcemanager.resource-tracker.address</name>
-        <value>sandbox.hortonworks.com:8025</value>
-    </property>
-
     <property>
         <name>yarn.resourcemanager.scheduler.address</name>
-        <value>sandbox.hortonworks.com:8030</value>
+        <value>quickstart.cloudera:8030</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.scheduler.class</name>
-        <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.state-store.max-completed-applications</name>
-        <value>${yarn.resourcemanager.max-completed-applications}</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.store.class</name>
-        <value>org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size</name>
-        <value>10</value>
-    </property>
-
-    <property>
-        <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>
-        <value>true</value>
+        <name>yarn.resourcemanager.resource-tracker.address</name>
+        <value>quickstart.cloudera:8031</value>
     </property>
-
     <property>
         <name>yarn.resourcemanager.webapp.address</name>
-        <value>sandbox.hortonworks.com:8088</value>
+        <value>quickstart.cloudera:8088</value>
     </property>
-
-    <property>
-        <name>yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled</name>
-        <value>false</value>
-    </property>
-
     <property>
         <name>yarn.resourcemanager.webapp.https.address</name>
-        <value>localhost:8090</value>
+        <value>quickstart.cloudera:8090</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.webapp.proxyuser.hcat.groups</name>
-        <value>*</value>
+        <name>yarn.resourcemanager.client.thread-count</name>
+        <value>50</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.webapp.proxyuser.hcat.hosts</name>
-        <value>*</value>
+        <name>yarn.resourcemanager.scheduler.client.thread-count</name>
+        <value>50</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.webapp.proxyuser.oozie.groups</name>
-        <value>*</value>
+        <name>yarn.resourcemanager.admin.client.thread-count</name>
+        <value>1</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.webapp.proxyuser.oozie.hosts</name>
-        <value>*</value>
+        <name>yarn.scheduler.minimum-allocation-mb</name>
+        <value>1</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.work-preserving-recovery.enabled</name>
-        <value>true</value>
+        <name>yarn.scheduler.increment-allocation-mb</name>
+        <value>512</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms</name>
-        <value>10000</value>
+        <name>yarn.scheduler.maximum-allocation-mb</name>
+        <value>2816</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.zk-acl</name>
-        <value>world:anyone:rwcda</value>
+        <name>yarn.scheduler.minimum-allocation-vcores</name>
+        <value>1</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.zk-address</name>
-        <value>localhost:2181</value>
+        <name>yarn.scheduler.increment-allocation-vcores</name>
+        <value>1</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.zk-num-retries</name>
-        <value>1000</value>
+        <name>yarn.scheduler.maximum-allocation-vcores</name>
+        <value>2</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.zk-retry-interval-ms</name>
+        <name>yarn.resourcemanager.amliveliness-monitor.interval-ms</name>
         <value>1000</value>
     </property>
-
     <property>
-        <name>yarn.resourcemanager.zk-state-store.parent-path</name>
-        <value>/rmstore</value>
+        <name>yarn.am.liveness-monitor.expiry-interval-ms</name>
+        <value>600000</value>
     </property>
-
-    <property>
-        <name>yarn.resourcemanager.zk-timeout-ms</name>
-        <value>10000</value>
-    </property>
-
-    <property>
-        <name>yarn.scheduler.maximum-allocation-mb</name>
-        <value>9216</value>
-    </property>
-
     <property>
-        <name>yarn.scheduler.minimum-allocation-mb</name>
-        <value>1536</value>
+        <name>yarn.resourcemanager.am.max-attempts</name>
+        <value>2</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.address</name>
-        <value>sandbox.hortonworks.com:10200</value>
+        <name>yarn.resourcemanager.container.liveness-monitor.interval-ms</name>
+        <value>600000</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.bind-host</name>
-        <value>0.0.0.0</value>
+        <name>yarn.resourcemanager.nm.liveness-monitor.interval-ms</name>
+        <value>1000</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.client.max-retries</name>
-        <value>30</value>
+        <name>yarn.nm.liveness-monitor.expiry-interval-ms</name>
+        <value>600000</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.client.retry-interval-ms</name>
-        <value>1000</value>
+        <name>yarn.resourcemanager.resource-tracker.client.thread-count</name>
+        <value>50</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.enabled</name>
-        <value>true</value>
+        <name>yarn.application.classpath</name>
+        <value>$HADOOP_CLIENT_CONF_DIR,$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/*,$HADOOP_COMMON_HOME/lib/*,$HADOOP_HDFS_HOME/*,$HADOOP_HDFS_HOME/lib/*,$HADOOP_YARN_HOME/*,$HADOOP_YARN_HOME/lib/*</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.generic-application-history.store-class</name>
-        <value>org.apache.hadoop.yarn.server.applicationhistoryservice.NullApplicationHistoryStore</value>
+        <name>yarn.resourcemanager.scheduler.class</name>
+        <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.http-authentication.simple.anonymous.allowed</name>
+        <name>yarn.scheduler.fair.user-as-default-queue</name>
         <value>true</value>
     </property>
-
-    <property>
-        <name>yarn.timeline-service.http-authentication.type</name>
-        <value>simple</value>
-    </property>
-
     <property>
-        <name>yarn.timeline-service.leveldb-timeline-store.path</name>
-        <value>/hadoop/yarn/timeline</value>
+        <name>yarn.scheduler.fair.preemption</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.leveldb-timeline-store.read-cache-size</name>
-        <value>104857600</value>
+        <name>yarn.scheduler.fair.sizebasedweight</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size</name>
-        <value>10000</value>
+        <name>yarn.scheduler.fair.assignmultiple</name>
+        <value>false</value>
     </property>
-
     <property>
-        <name>yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size</name>
+        <name>yarn.resourcemanager.max-completed-applications</name>
         <value>10000</value>
     </property>
-
-    <property>
-        <name>yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms</name>
-        <value>300000</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.store-class</name>
-        <value>org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.ttl-enable</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.ttl-ms</name>
-        <value>2678400000</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.webapp.address</name>
-        <value>sandbox.hortonworks.com:8188</value>
-    </property>
-
-    <property>
-        <name>yarn.timeline-service.webapp.https.address</name>
-        <value>sandbox.hortonworks.com:8190</value>
-    </property>
-
 </configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/cec70b82/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1cea2d5..b51d4c0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -46,19 +46,19 @@
         <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-cdh5.7.0</hadoop2.version>
+        <yarn.version>2.6.0-cdh5.7.0</yarn.version>
 
         <!-- Hive versions -->
-        <hive.version>1.2.1</hive.version>
-        <hive-hcatalog.version>1.2.1</hive-hcatalog.version>
+        <hive.version>1.1.0-cdh5.7.0</hive.version>
+        <hive-hcatalog.version>1.1.0-cdh5.7.0</hive-hcatalog.version>
 
         <!-- HBase versions -->
-        <hbase-hadoop2.version>1.1.1</hbase-hadoop2.version>
+        <hbase-hadoop2.version>1.2.0-cdh5.7.0</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>
+        <zookeeper.version>3.4.5-cdh5.7.0</zookeeper.version>
         <curator.version>2.7.1</curator.version>
         <jackson.version>2.2.4</jackson.version>
         <jsr305.version>3.0.1</jsr305.version>
@@ -813,6 +813,10 @@
             <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>

http://git-wip-us.apache.org/repos/asf/kylin/blob/cec70b82/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index 454ed6b..42a9562 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -46,6 +46,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
 
@@ -112,6 +116,10 @@
                     <groupId>javax.servlet</groupId>
                     <artifactId>servlet-api</artifactId>
                 </exclusion>
+	        <exclusion>
+		    <groupId>com.google.protobuf</groupId>
+		    <artifactId>protobuf-java</artifactId>
+	        </exclusion>
                 <exclusion>
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
@@ -131,6 +139,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -146,6 +158,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -161,6 +177,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -176,6 +196,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -199,6 +223,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -214,6 +242,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -273,6 +305,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/kylin/blob/cec70b82/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java
index d953d9c..b168064 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/MockupMapContext.java
@@ -23,7 +23,6 @@ import java.net.URI;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configuration.IntegerRanges;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.mapreduce.Counter;
@@ -44,10 +43,10 @@ import org.apache.kylin.engine.mr.common.BatchConstants;
 
 /**
  * @author yangli9
- * 
+ *
  */
 @SuppressWarnings({ "rawtypes", "unchecked" })
-public class MockupMapContext implements  MapContext {
+public class MockupMapContext implements MapContext {
 
     private Configuration hconf;
 
@@ -60,7 +59,7 @@ public class MockupMapContext implements  MapContext {
         return new WrappedMapper().getMapContext(new MockupMapContext(hconf, outKV));
     }
 
-    public MockupMapContext(Configuration hconf, Object[] outKV){
+    public MockupMapContext(Configuration hconf, Object[] outKV) {
         this.hconf = hconf;
         this.outKV = outKV;
     }
@@ -110,11 +109,6 @@ public class MockupMapContext implements  MapContext {
     }
 
     @Override
-    public float getProgress() {
-        throw new NotImplementedException();
-    }
-
-    @Override
     public Counter getCounter(Enum<?> counterName) {
         throw new NotImplementedException();
     }
@@ -175,6 +169,11 @@ public class MockupMapContext implements  MapContext {
     }
 
     @Override
+    public boolean userClassesTakesPrecedence() {
+        throw new NotImplementedException();
+    }
+
+    @Override
     public Class<? extends InputFormat<?, ?>> getInputFormatClass() throws ClassNotFoundException {
         throw new NotImplementedException();
     }
@@ -225,11 +224,6 @@ public class MockupMapContext implements  MapContext {
     }
 
     @Override
-    public boolean getTaskCleanupNeeded() {
-        throw new NotImplementedException();
-    }
-
-    @Override
     public boolean getProfileEnabled() {
         throw new NotImplementedException();
     }
@@ -240,11 +234,6 @@ public class MockupMapContext implements  MapContext {
     }
 
     @Override
-    public IntegerRanges getProfileTaskRange(boolean isMap) {
-        throw new NotImplementedException();
-    }
-
-    @Override
     public String getUser() {
         throw new NotImplementedException();
     }
@@ -318,4 +307,4 @@ public class MockupMapContext implements  MapContext {
     public RawComparator<?> getCombinerKeyGroupingComparator() {
         throw new NotImplementedException();
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/cec70b82/tool/pom.xml
----------------------------------------------------------------------
diff --git a/tool/pom.xml b/tool/pom.xml
index 919a903..6820f0f 100644
--- a/tool/pom.xml
+++ b/tool/pom.xml
@@ -56,6 +56,18 @@
 
         <!--Env-->
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-api</artifactId>
+            <version>${yarn.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-common</artifactId>
+            <version>${yarn.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-client</artifactId>
             <scope>provided</scope>


[18/21] kylin git commit: KYLIN-2400 code review

Posted by li...@apache.org.
KYLIN-2400 code review


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

Branch: refs/heads/master-cdh5.7
Commit: 576d2dd352b11f428db1d6308e35350a2fca122e
Parents: e769bfa
Author: Li Yang <li...@apache.org>
Authored: Tue Jan 17 11:06:26 2017 +0800
Committer: Li Yang <li...@apache.org>
Committed: Tue Jan 17 11:06:26 2017 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/util/Dictionary.java    | 42 +++-----------------
 .../apache/kylin/dict/AppendTrieDictionary.java | 39 +++++++-----------
 .../apache/kylin/dict/DictionaryGenerator.java  |  2 +-
 .../kylin/dict/NumberDictionaryBuilder.java     | 10 ++---
 .../dict/NumberDictionaryForestBuilder.java     |  7 +++-
 .../apache/kylin/dict/TrieDictionaryForest.java |  2 +-
 .../apache/kylin/dict/NumberDictionaryTest.java |  4 +-
 .../kylin/dict/TimeStrDictionaryTest.java       |  8 ++--
 .../kylin/dict/TrieDictionaryForestTest.java    |  2 +-
 .../kylin/measure/raw/RawMeasureType.java       |  3 ++
 .../storage/gtrecord/DictGridTableTest.java     |  3 +-
 .../mr/steps/NumberDictionaryForestTest.java    |  2 +-
 12 files changed, 43 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/576d2dd3/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java b/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
index 9d675f9..4b5c1c4 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
@@ -94,13 +94,12 @@ abstract public class Dictionary<T> implements Serializable {
     final public int getIdFromValue(T value, int roundingFlag) throws IllegalArgumentException {
         if (isNullObjectForm(value))
             return nullId();
-        else {
-            int id = getIdFromValueImpl(value, roundingFlag);
-            if(id == -1){
-                throw new IllegalArgumentException("Value : " + value + "no exists");
-            }
-            return id;
+
+        int id = getIdFromValueImpl(value, roundingFlag);
+        if (id == -1) {
+            throw new IllegalArgumentException("Value : " + value + " not exists");
         }
+        return id;
     }
 
     final public boolean containsValue(T value) throws IllegalArgumentException {
@@ -137,37 +136,6 @@ abstract public class Dictionary<T> implements Serializable {
 
     abstract protected T getValueFromIdImpl(int id);
 
-    /**
-     * Convenient form of
-     * <code>getIdFromValueBytes(value, offset, len, 0)</code>
-     */
-
-
-    /**
-     * A lower level API, return ID integer from raw value bytes. In case of not found 
-     * <p>
-     * - if roundingFlag=0, throw IllegalArgumentException; <br>
-     * - if roundingFlag<0, the closest smaller ID integer if exist; <br>
-     * - if roundingFlag>0, the closest bigger ID integer if exist. <br>
-     * <p>
-     * Bypassing the cache layer, this could be significantly slower than getIdFromValue(T value).
-     * 
-     * @throws IllegalArgumentException
-     *             if value is not found in dictionary and rounding is off;
-     *             or if rounding cannot find a smaller or bigger ID
-     */
-
-    /**
-     * A lower level API, get byte values from ID, return the number of bytes
-     * written. Bypassing the cache layer, this could be significantly slower
-     * than getIdFromValue(T value).
-     *
-     * @return size of value bytes, 0 if empty string, -1 if null
-     *
-     * @throws IllegalArgumentException
-     *             if ID is not found in dictionary
-     */
-
     abstract public void dump(PrintStream out);
 
     public int nullId() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/576d2dd3/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
index 3aef967..faffcc0 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
@@ -91,14 +91,11 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
 
     volatile private TreeMap<DictSliceKey, DictSlice> dictSliceMap;
 
-
     // Constructor both for build and deserialize
     public AppendTrieDictionary() {
         enableCache();
     }
 
-
-
     public void initParams(String baseDir, int baseId, int maxId, int maxValueLength, int nValues, BytesConverter bytesConverter) throws IOException {
         this.baseDir = baseDir;
         this.baseId = baseId;
@@ -111,8 +108,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
     public void initDictSliceMap(CachedTreeMap dictMap) throws IOException {
         int maxVersions = KylinConfig.getInstanceFromEnv().getAppendDictMaxVersions();
         long versionTTL = KylinConfig.getInstanceFromEnv().getAppendDictVersionTTL();
-        CachedTreeMap newDictSliceMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder().maxSize(1).baseDir(baseDir)
-            .immutable(true).maxVersions(maxVersions).versionTTL(versionTTL).keyClazz(DictSliceKey.class).valueClazz(DictSlice.class).build();
+        CachedTreeMap newDictSliceMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder().maxSize(1).baseDir(baseDir).immutable(true).maxVersions(maxVersions).versionTTL(versionTTL).keyClazz(DictSliceKey.class).valueClazz(DictSlice.class).build();
         newDictSliceMap.loadEntry(dictMap);
         this.dictSliceMap = newDictSliceMap;
     }
@@ -210,7 +206,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
                 this.sizeChildOffset = headIn.read();
                 this.sizeOfId = headIn.read();
 
-                this.childOffsetMask = ~(((long)(BIT_IS_LAST_CHILD | BIT_IS_END_OF_VALUE)) << ((sizeChildOffset - 1) * 8));
+                this.childOffsetMask = ~(((long) (BIT_IS_LAST_CHILD | BIT_IS_END_OF_VALUE)) << ((sizeChildOffset - 1) * 8));
                 this.firstByteOffset = sizeChildOffset + 1; // the offset from begin of node to its first value byte
             } catch (Exception e) {
                 if (e instanceof RuntimeException)
@@ -229,7 +225,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
                 if (checkFlag(nodeOffset, BIT_IS_END_OF_VALUE)) {
                     break;
                 }
-                nodeOffset = headSize + (int)(BytesUtil.readLong(trieBytes, nodeOffset, sizeChildOffset) & childOffsetMask);
+                nodeOffset = headSize + (int) (BytesUtil.readLong(trieBytes, nodeOffset, sizeChildOffset) & childOffsetMask);
                 if (nodeOffset == headSize) {
                     break;
                 }
@@ -271,7 +267,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
                 }
 
                 // find a child to continue
-                int c = headSize + (int)(BytesUtil.readLong(trieBytes, n, sizeChildOffset) & childOffsetMask);
+                int c = headSize + (int) (BytesUtil.readLong(trieBytes, n, sizeChildOffset) & childOffsetMask);
                 if (c == headSize) // has no children
                     return -1;
                 byte inpByte = inp[o];
@@ -310,7 +306,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
             DictNode root = null;
             while (true) {
                 int p = n + firstByteOffset;
-                int childOffset = (int)(BytesUtil.readLong(trieBytes, n, sizeChildOffset) & childOffsetMask);
+                int childOffset = (int) (BytesUtil.readLong(trieBytes, n, sizeChildOffset) & childOffsetMask);
                 int parLen = BytesUtil.readUnsigned(trieBytes, p - 1, 1);
                 boolean isEndOfValue = checkFlag(n, BIT_IS_END_OF_VALUE);
 
@@ -357,13 +353,13 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
                     lastChild = false;
                 }
                 int p = offset + firstByteOffset;
-                int childOffset = (int)(BytesUtil.readLong(trieBytes, offset, sizeChildOffset) & childOffsetMask);
+                int childOffset = (int) (BytesUtil.readLong(trieBytes, offset, sizeChildOffset) & childOffsetMask);
                 int parLen = BytesUtil.readUnsigned(trieBytes, p - 1, 1);
                 boolean isEndOfValue = checkFlag(offset, BIT_IS_END_OF_VALUE);
 
                 // Copy value overflow, the data is corrupted
                 if (trieBytes.length < p + parLen) {
-                   return false;
+                    return false;
                 }
 
                 // Check id is fine
@@ -401,7 +397,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
                 throw new IllegalArgumentException("Wrong file type (magic does not match)");
 
             DataInputStream headIn = new DataInputStream(//
-                new ByteArrayInputStream(headPartial, HEAD_SIZE_I, headPartial.length - HEAD_SIZE_I));
+                    new ByteArrayInputStream(headPartial, HEAD_SIZE_I, headPartial.length - HEAD_SIZE_I));
             int headSize = headIn.readShort();
             int bodyLen = headIn.readInt();
             headIn.close();
@@ -665,8 +661,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
 
             // nValueBytes
             if (n.part.length > 255)
-                throw new RuntimeException("Value length is " + n.part.length
-                    + " and larger than 255: " + Bytes.toStringBinary(n.part));
+                throw new RuntimeException("Value length is " + n.part.length + " and larger than 255: " + Bytes.toStringBinary(n.part));
             BytesUtil.writeUnsigned(n.part.length, trieBytes, o, 1);
             o++;
 
@@ -785,7 +780,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
                 int t = s.mbpn_sizeValueTotal + s.mbpn_nNodes * (s.mbpn_sizeNoValueBytes + s.mbpn_sizeChildOffset - 1);
                 // *4 because 2 MSB of offset is used for isEndOfValue & isEndChild flag
                 // expand t to long before *4, avoiding exceed Integer.MAX_VALUE
-                if (BytesUtil.sizeForValue((long)t * 4) <= s.mbpn_sizeChildOffset - 1) {
+                if (BytesUtil.sizeForValue((long) t * 4) <= s.mbpn_sizeChildOffset - 1) {
                     s.mbpn_sizeChildOffset--;
                     s.mbpn_footprint = t;
                 } else
@@ -886,8 +881,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
                 builder = new Builder<>(resourcePath, null, dictDir, 0, 0, 0, new StringBytesConverter(), null);
             } else {
                 logger.info("GlobalDict {} exist, append value", resourcePath);
-                builder = new Builder<>(resourcePath, dictToUse, dictToUse.baseDir, dictToUse.maxId, dictToUse.maxValueLength,
-                    dictToUse.nValues, dictToUse.bytesConvert, dictToUse.writeDictMap());
+                builder = new Builder<>(resourcePath, dictToUse, dictToUse.baseDir, dictToUse.maxId, dictToUse.maxValueLength, dictToUse.nValues, dictToUse.bytesConvert, dictToUse.writeDictMap());
             }
 
             return builder;
@@ -926,8 +920,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
             int maxVersions = KylinConfig.getInstanceFromEnv().getAppendDictMaxVersions();
             long versionTTL = KylinConfig.getInstanceFromEnv().getAppendDictVersionTTL();
             // create a new cached map with baseDir
-            mutableDictSliceMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder().maxSize(1).baseDir(baseDir)
-                .maxVersions(maxVersions).versionTTL(versionTTL).keyClazz(DictSliceKey.class).valueClazz(DictNode.class).immutable(false).build();
+            mutableDictSliceMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder().maxSize(1).baseDir(baseDir).maxVersions(maxVersions).versionTTL(versionTTL).keyClazz(DictSliceKey.class).valueClazz(DictNode.class).immutable(false).build();
             if (dictMapBytes != null) {
                 ((Writable) mutableDictSliceMap).readFields(new DataInputStream(new ByteArrayInputStream(dictMapBytes)));
             }
@@ -1105,7 +1098,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
 
         public synchronized AppendTrieDictionary<T> build(int baseId) throws IOException {
             boolean keepAppend = releaseInstance(resourcePath);
-            CachedTreeMap dictSliceMap = (CachedTreeMap)mutableDictSliceMap;
+            CachedTreeMap dictSliceMap = (CachedTreeMap) mutableDictSliceMap;
             dict.initParams(baseDir, baseId, maxId, maxValueLength, nValues, bytesConverter);
             dict.flushIndex(dictSliceMap, keepAppend);
             dict.initDictSliceMap(dictSliceMap);
@@ -1155,7 +1148,6 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
         return maxValueLength;
     }
 
-
     public void flushIndex(CachedTreeMap dictSliceMap, boolean keepAppend) throws IOException {
         try (FSDataOutputStream indexOut = dictSliceMap.openIndexOutput()) {
             indexOut.writeInt(baseId);
@@ -1220,14 +1212,12 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
             initParams(baseDir, baseId, maxId, maxValueLength, nValues, converter);
 
             // Create instance for deserialize data, and update to map in dict
-            CachedTreeMap dictMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder()
-                .baseDir(baseDir).immutable(true).keyClazz(DictSliceKey.class).valueClazz(DictSlice.class).build();
+            CachedTreeMap dictMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder().baseDir(baseDir).immutable(true).keyClazz(DictSliceKey.class).valueClazz(DictSlice.class).build();
             dictMap.readFields(input);
             initDictSliceMap(dictMap);
         }
     }
 
-
     @Override
     public void dump(PrintStream out) {
         out.println("Total " + nValues + " values, " + (dictSliceMap == null ? 0 : dictSliceMap.size()) + " slice");
@@ -1261,4 +1251,3 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
     }
 
 }
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/576d2dd3/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
index c23e7b5..61a0664 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
@@ -227,7 +227,7 @@ public class DictionaryGenerator {
         @Override
         public void init(DictionaryInfo info, int baseId) throws IOException {
             this.baseId = baseId;
-            this.builder = new NumberDictionaryBuilder(new NumberDictionaryForestBuilder.Number2BytesConverter());
+            this.builder = new NumberDictionaryBuilder();
         }
         
         @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/576d2dd3/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java
index b9e94414..288e38f 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java
@@ -25,17 +25,17 @@ package org.apache.kylin.dict;
  * @author yangli9
  */
 @Deprecated
-public class NumberDictionaryBuilder<T> extends TrieDictionaryBuilder<T> {
+public class NumberDictionaryBuilder extends TrieDictionaryBuilder<String> {
 
 
-    public NumberDictionaryBuilder(BytesConverter<T> bytesConverter) {
-        super(bytesConverter);
+    public NumberDictionaryBuilder() {
+        super(new NumberDictionaryForestBuilder.Number2BytesConverter());
     }
 
 
-    public NumberDictionary<T> build(int baseId) {
+    public NumberDictionary build(int baseId) {
         byte[] trieBytes = buildTrieBytes(baseId);
-        NumberDictionary2<T> r = new NumberDictionary2<T>(trieBytes);
+        NumberDictionary2 r = new NumberDictionary2(trieBytes);
         return r;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/576d2dd3/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java
index 7c750f4..b072599 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java
@@ -18,6 +18,8 @@
 
 package org.apache.kylin.dict;
 
+import java.io.Serializable;
+
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.dict.NumberDictionary.NumberBytesCodec;
 
@@ -26,8 +28,9 @@ import org.apache.kylin.dict.NumberDictionary.NumberBytesCodec;
  */
 public class NumberDictionaryForestBuilder extends TrieDictionaryForestBuilder<String> {
 
-    public static class Number2BytesConverter implements BytesConverter<String>, java.io.Serializable {
-
+    public static class Number2BytesConverter implements BytesConverter<String>, Serializable {
+        private static final long serialVersionUID = 1L;
+        
         static final int MAX_DIGITS_BEFORE_DECIMAL_POINT = NumberDictionary.MAX_DIGITS_BEFORE_DECIMAL_POINT;
         static final transient ThreadLocal<NumberBytesCodec> LOCAL = new ThreadLocal<NumberBytesCodec>();
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/576d2dd3/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
index d1b5a99..04292d2 100755
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
@@ -129,7 +129,7 @@ public class TrieDictionaryForest<T> extends CacheDictionary<T> {
         }
         TrieDictionary<T> tree = trees.get(index);
         int id = tree.getIdFromValueBytesWithoutCache(value, offset, len, roundingFlag);
-        if(id == -1)
+        if (id == -1)
             throw new IllegalArgumentException("Value '" + Bytes.toString(value, offset, len) + "' (" + Bytes.toStringBinary(value, offset, len) + ") not exists!");
         id = id + accuOffset.get(index);
         id += baseId;

http://git-wip-us.apache.org/repos/asf/kylin/blob/576d2dd3/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
index 38f2648..1c04745 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
@@ -59,7 +59,7 @@ public class NumberDictionaryTest extends LocalFileMetadataTestCase {
 
     @Test
     public void testMinMax() {
-        NumberDictionaryBuilder<String> builder = new NumberDictionaryBuilder<String>(new NumberDictionaryForestBuilder.Number2BytesConverter());
+        NumberDictionaryBuilder builder = new NumberDictionaryBuilder();
         builder.addValue("" + Long.MAX_VALUE);
         builder.addValue("" + Long.MIN_VALUE);
         NumberDictionary<String> dict = builder.build(0);
@@ -123,7 +123,7 @@ public class NumberDictionaryTest extends LocalFileMetadataTestCase {
         int n = 100;
 
         Set<BigDecimal> set = Sets.newHashSet();
-        NumberDictionaryBuilder<String> builder = new NumberDictionaryBuilder<String>(new NumberDictionaryForestBuilder.Number2BytesConverter());
+        NumberDictionaryBuilder builder = new NumberDictionaryBuilder();
         for (int i = 0; i < n; i++) {
             String num = randNumber();
             if (set.add(new BigDecimal(num))) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/576d2dd3/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java
index f981942..e852d26 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java
@@ -18,13 +18,13 @@
 
 package org.apache.kylin.dict;
 
+import static org.junit.Assert.fail;
+
 import org.apache.kylin.common.util.DateFormat;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import static org.junit.Assert.fail;
-
 /**
  */
 public class TimeStrDictionaryTest {
@@ -64,10 +64,10 @@ public class TimeStrDictionaryTest {
 
     @Test
     public void testIllegal() {
-        try{
+        try {
             dict.getIdFromValue("2038-01-19 03:14:07");
             fail("should throw exception");
-        }catch (IllegalArgumentException e){
+        } catch (IllegalArgumentException e) {
             //correct
         }
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/576d2dd3/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java
index c7fb9c4..dd1f951 100755
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java
@@ -641,7 +641,7 @@ public class TrieDictionaryForestTest {
         System.out.println("max memory:" + Runtime.getRuntime().maxMemory());
         System.gc();
         Thread.currentThread().sleep(1000);
-        NumberDictionaryBuilder<String> b = new NumberDictionaryBuilder<>(new NumberDictionaryForestBuilder.Number2BytesConverter());
+        NumberDictionaryBuilder b = new NumberDictionaryBuilder();
         int k = 0;
         while (true) {
             b.addValue(k + "");

http://git-wip-us.apache.org/repos/asf/kylin/blob/576d2dd3/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
index 3b6e659..31f35e9 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
@@ -46,6 +46,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class RawMeasureType extends MeasureType<List<ByteArray>> {
+    private static final long serialVersionUID = 1L;
 
     private static final Logger logger = LoggerFactory.getLogger(RawMeasureType.class);
 
@@ -103,6 +104,8 @@ public class RawMeasureType extends MeasureType<List<ByteArray>> {
     @Override
     public MeasureIngester<List<ByteArray>> newIngester() {
         return new MeasureIngester<List<ByteArray>>() {
+            private static final long serialVersionUID = 1L;
+
             //encode measure value to dictionary
             @Override
             public List<ByteArray> valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/576d2dd3/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
----------------------------------------------------------------------
diff --git a/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java b/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
index ae186e2..7500b00 100644
--- a/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
+++ b/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
@@ -35,7 +35,6 @@ import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.gridtable.CubeCodeSystem;
 import org.apache.kylin.dict.NumberDictionaryBuilder;
-import org.apache.kylin.dict.NumberDictionaryForestBuilder;
 import org.apache.kylin.dict.StringBytesConverter;
 import org.apache.kylin.dict.TrieDictionaryBuilder;
 import org.apache.kylin.dimension.DictionaryDimEnc;
@@ -602,7 +601,7 @@ public class DictGridTableTest extends LocalFileMetadataTestCase {
 
     @SuppressWarnings("rawtypes")
     private static Dictionary newDictionaryOfInteger() {
-        NumberDictionaryBuilder<String> builder = new NumberDictionaryBuilder<>(new NumberDictionaryForestBuilder.Number2BytesConverter());
+        NumberDictionaryBuilder builder = new NumberDictionaryBuilder();
         builder.addValue("10");
         builder.addValue("20");
         builder.addValue("30");

http://git-wip-us.apache.org/repos/asf/kylin/blob/576d2dd3/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java
index 72793f5..1908992 100644
--- a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java
@@ -120,7 +120,7 @@ public class NumberDictionaryForestTest {
         TrieDictionaryForest<String> dict = b.build();
         dict.dump(System.out);
 
-        NumberDictionaryBuilder<String> b2 = new NumberDictionaryBuilder<>(new NumberDictionaryForestBuilder.Number2BytesConverter());
+        NumberDictionaryBuilder b2 = new NumberDictionaryBuilder();
         for (String str : testData)
             b2.addValue(str);
         NumberDictionary<String> dict2 = b2.build(0);


[08/21] kylin git commit: KYLIN-2238 Add query server scan threshold

Posted by li...@apache.org.
KYLIN-2238 Add query server scan threshold


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

Branch: refs/heads/master-cdh5.7
Commit: 4b00ec26de1e1cc9bf9b2e55a1b162d123114df8
Parents: 20c3294
Author: kangkaisen <ka...@live.com>
Authored: Tue Nov 29 19:51:39 2016 +0800
Committer: kangkaisen <ka...@163.com>
Committed: Sat Jan 14 19:00:09 2017 +0800

----------------------------------------------------------------------
 .../storage/gtrecord/SequentialCubeTupleIterator.java | 14 +++++++++++---
 1 file changed, 11 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/4b00ec26/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 ee868c7..c621215 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,6 +26,7 @@ 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.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -33,6 +34,7 @@ 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;
 
@@ -45,6 +47,8 @@ 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;
@@ -78,7 +82,7 @@ public class SequentialCubeTupleIterator implements ITupleIterator {
             tupleIterator = new SortedIteratorMergerWithLimit<ITuple>(transformed, context.getFinalPushDownLimit(), getTupleDimensionComparator(cuboid, returnTupleInfo)).getIterator();
         }
     }
-    
+
     public Comparator<ITuple> getTupleDimensionComparator(Cuboid cuboid, TupleInfo returnTupleInfo) {
         // dimensionIndexOnTuple is for SQL with limit
         List<Integer> temp = Lists.newArrayList();
@@ -92,7 +96,7 @@ public class SequentialCubeTupleIterator implements ITupleIterator {
         for (int i = 0; i < temp.size(); i++) {
             dimensionIndexOnTuple[i] = temp.get(i);
         }
-        
+
         return new Comparator<ITuple>() {
             @Override
             public int compare(ITuple o1, ITuple o2) {
@@ -137,7 +141,11 @@ public class SequentialCubeTupleIterator implements ITupleIterator {
 
     @Override
     public ITuple next() {
-        scanCount++;
+        // 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();
 


[16/21] kylin git commit: KYLIN-2396 fix ci

Posted by li...@apache.org.
KYLIN-2396 fix ci


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

Branch: refs/heads/master-cdh5.7
Commit: e20e2b2e82de5318edaf123b11d72267b7ef1ea1
Parents: 95d4412
Author: lidongsjtu <li...@apache.org>
Authored: Mon Jan 16 22:39:58 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Mon Jan 16 22:39:58 2017 +0800

----------------------------------------------------------------------
 .../src/test/java/org/apache/kylin/query/ITKylinQueryTest.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/e20e2b2e/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 02134d4..87ddcb8 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
@@ -400,6 +400,6 @@ public class ITKylinQueryTest extends KylinTestBase {
 
     @Test
     public void testPercentileQuery() throws Exception {
-        batchExecuteQuery("src/test/resources/query/sql_percentile");
+        batchExecuteQuery(getQueryFolderPrefix() + "src/test/resources/query/sql_percentile");
     }
 }


[14/21] kylin git commit: minor, fix null_counter datatype in TableExtDesc

Posted by li...@apache.org.
minor, fix null_counter datatype in TableExtDesc

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/master-cdh5.7
Commit: 4625b059ba5f55e1b938483297c18884aa17f6da
Parents: 4bc31de
Author: Cheng Wang <ch...@kyligence.io>
Authored: Mon Jan 16 17:10:40 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jan 16 17:18:05 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/metadata/model/TableExtDesc.java   | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/4625b059/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java
index 7ed3368..18fefc2 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java
@@ -200,8 +200,8 @@ public class TableExtDesc extends RootPersistentEntity {
         private String minLengthValue;
 
         @JsonProperty("null_count")
-        private int nullCount;
-        
+        private long nullCount;
+
         @JsonProperty("cardinality")
         private long cardinality;
 
@@ -260,11 +260,11 @@ public class TableExtDesc extends RootPersistentEntity {
             this.minLengthValue = minLenValue;
         }
 
-        public int getNullCount() {
+        public long getNullCount() {
             return nullCount;
         }
 
-        public void setNullCount(int nullCount) {
+        public void setNullCount(long nullCount) {
             this.nullCount = nullCount;
         }
     }


[19/21] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 c8410f9..e72859d 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,7 +44,8 @@ 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.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.KylinVersion;
@@ -81,7 +82,8 @@ public class DeployCoprocessorCLI {
         KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
         Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
         FileSystem fileSystem = FileSystem.get(hconf);
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(hconf);
+        Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
 
         String localCoprocessorJar;
         if ("default".equals(args[0])) {
@@ -165,10 +167,10 @@ public class DeployCoprocessorCLI {
     public static void deployCoprocessor(HTableDescriptor tableDesc) {
         try {
             initHTableCoprocessor(tableDesc);
-            logger.info("hbase table " + tableDesc.getName() + " deployed with coprocessor.");
+            logger.info("hbase table " + tableDesc.getTableName() + " deployed with coprocessor.");
 
         } catch (Exception ex) {
-            logger.error("Error deploying coprocessor on " + tableDesc.getName(), ex);
+            logger.error("Error deploying coprocessor on " + tableDesc.getTableName(), ex);
             logger.error("Will try creating the table without coprocessor.");
         }
     }
@@ -189,7 +191,7 @@ public class DeployCoprocessorCLI {
         desc.addCoprocessor(CubeEndpointClass, hdfsCoprocessorJar, 1001, null);
     }
 
-    public static boolean resetCoprocessor(String tableName, HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
+    public static boolean resetCoprocessor(String tableName, Admin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
         KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
         HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
 
@@ -204,7 +206,7 @@ public class DeployCoprocessorCLI {
         logger.info("reset coprocessor on " + tableName);
 
         logger.info("Disable " + tableName);
-        hbaseAdmin.disableTable(tableName);
+        hbaseAdmin.disableTable(TableName.valueOf(tableName));
 
         while (desc.hasCoprocessor(CubeObserverClassOld2)) {
             desc.removeCoprocessor(CubeObserverClassOld2);
@@ -230,16 +232,15 @@ public class DeployCoprocessorCLI {
             desc.setValue(IRealizationConstants.HTableGitTag, commitInfo);
         }
 
-        hbaseAdmin.modifyTable(tableName, desc);
+        hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
 
         logger.info("Enable " + tableName);
-        hbaseAdmin.enableTable(tableName);
+        hbaseAdmin.enableTable(TableName.valueOf(tableName));
 
         return true;
     }
 
-
-    private static List<String> resetCoprocessorOnHTables(final HBaseAdmin hbaseAdmin, final Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
+    private static List<String> resetCoprocessorOnHTables(final Admin 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());
@@ -260,12 +261,12 @@ public class DeployCoprocessorCLI {
 
     private static class ResetCoprocessorWorker implements Runnable {
         private final CountDownLatch countDownLatch;
-        private final HBaseAdmin hbaseAdmin;
+        private final Admin hbaseAdmin;
         private final Path hdfsCoprocessorJar;
         private final String tableName;
         private final List<String> processedTables;
 
-        public ResetCoprocessorWorker(CountDownLatch countDownLatch, HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar, String tableName, List<String> processedTables) {
+        public ResetCoprocessorWorker(CountDownLatch countDownLatch, Admin hbaseAdmin, Path hdfsCoprocessorJar, String tableName, List<String> processedTables) {
             this.countDownLatch = countDownLatch;
             this.hbaseAdmin = hbaseAdmin;
             this.hdfsCoprocessorJar = hdfsCoprocessorJar;
@@ -386,7 +387,7 @@ public class DeployCoprocessorCLI {
         return coprocessorDir;
     }
 
-    private static Set<String> getCoprocessorJarPaths(HBaseAdmin hbaseAdmin, List<String> tableNames) throws IOException {
+    private static Set<String> getCoprocessorJarPaths(Admin 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/a34f8151/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 61c73d5..1cdb2f8 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,10 +25,11 @@ 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;
@@ -235,9 +236,9 @@ public class ExtendCubeToHybridCLI {
         Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
         ProjectInstance project = store.getResource(projectResPath, ProjectInstance.class, projectSerializer);
         String projUUID = project.getUuid();
-        HTableInterface aclHtable = null;
+        Table aclHtable = null;
         try {
-            aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(kylinConfig.getMetadataUrlPrefix() + "_acl");
+            aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(TableName.valueOf(kylinConfig.getMetadataUrlPrefix() + "_acl"));
 
             // cube acl
             Result result = aclHtable.get(new Get(Bytes.toBytes(origCubeId)));
@@ -257,7 +258,6 @@ public class ExtendCubeToHybridCLI {
                     aclHtable.put(put);
                 }
             }
-            aclHtable.flushCommits();
         } finally {
             IOUtils.closeQuietly(aclHtable);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 86ba22f..dd5f8fa 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.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 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
 
-        HConnection conn = HBaseConnection.get(hbaseUrl);
+        Connection conn = HBaseConnection.get(hbaseUrl);
         createHTableIfNeeded(conn, TEST_TABLE);
         prepareData(conn);
 
@@ -91,10 +91,10 @@ public class GridTableHBaseBenchmark {
 
     }
 
-    private static void testColumnScan(HConnection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
+    private static void testColumnScan(Connection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
         Stats stats = new Stats("COLUMN_SCAN");
 
-        HTableInterface table = conn.getTable(TEST_TABLE);
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
             stats.markStart();
 
@@ -122,20 +122,20 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void testRowScanNoIndexFullScan(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanNoIndexFullScan(Connection conn, boolean[] hits) throws IOException {
         fullScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_FULL"));
     }
 
-    private static void testRowScanNoIndexSkipScan(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanNoIndexSkipScan(Connection conn, boolean[] hits) throws IOException {
         jumpScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_SKIP"));
     }
 
-    private static void testRowScanWithIndex(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanWithIndex(Connection conn, boolean[] hits) throws IOException {
         jumpScan(conn, hits, new Stats("ROW_SCAN_IDX"));
     }
 
-    private static void fullScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
-        HTableInterface table = conn.getTable(TEST_TABLE);
+    private static void fullScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
             stats.markStart();
 
@@ -156,11 +156,11 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void jumpScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
+    private static void jumpScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
 
         final int jumpThreshold = 6; // compensate for Scan() overhead, totally by experience
 
-        HTableInterface table = conn.getTable(TEST_TABLE);
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
 
             stats.markStart();
@@ -204,8 +204,8 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void prepareData(HConnection conn) throws IOException {
-        HTableInterface table = conn.getTable(TEST_TABLE);
+    private static void prepareData(Connection conn) throws IOException {
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
 
         try {
             // check how many rows existing
@@ -258,8 +258,8 @@ public class GridTableHBaseBenchmark {
         return bytes;
     }
 
-    private static void createHTableIfNeeded(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    private static void createHTableIfNeeded(Connection conn, String tableName) throws IOException {
+        Admin hbase = conn.getAdmin();
 
         try {
             boolean tableExist = false;

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 6749d6c..940d64a 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,9 +24,11 @@ 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.client.HBaseAdmin;
+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.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
@@ -55,8 +57,8 @@ public class HBaseClean extends AbstractApplication {
     private void cleanUp() {
         try {
             // get all kylin hbase tables
-            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-            HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+            Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+            Admin hbaseAdmin = conn.getAdmin();
             String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
             HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
             List<String> allTablesNeedToBeDropped = Lists.newArrayList();
@@ -71,12 +73,12 @@ public class HBaseClean extends AbstractApplication {
                 // drop tables
                 for (String htableName : allTablesNeedToBeDropped) {
                     logger.info("Deleting HBase table " + htableName);
-                    if (hbaseAdmin.tableExists(htableName)) {
-                        if (hbaseAdmin.isTableEnabled(htableName)) {
-                            hbaseAdmin.disableTable(htableName);
+                    if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
+                        if (hbaseAdmin.isTableEnabled(TableName.valueOf(htableName))) {
+                            hbaseAdmin.disableTable(TableName.valueOf(htableName));
                         }
 
-                        hbaseAdmin.deleteTable(htableName);
+                        hbaseAdmin.deleteTable(TableName.valueOf(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/a34f8151/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 937b65f..1daca0a 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,6 +23,7 @@ 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;
@@ -31,12 +32,15 @@ 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.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
+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.util.Bytes;
 import org.apache.kylin.common.util.Pair;
 import org.slf4j.Logger;
@@ -58,30 +62,31 @@ public class HBaseRegionSizeCalculator {
     /**
      * Computes size of each region for table and given column families.
      * */
-    public HBaseRegionSizeCalculator(HTable table) throws IOException {
-        this(table, new HBaseAdmin(table.getConfiguration()));
-    }
-
-    /** Constructor for unit testing */
-    HBaseRegionSizeCalculator(HTable table, HBaseAdmin hBaseAdmin) throws IOException {
+    public HBaseRegionSizeCalculator(String tableName, Connection hbaseConnection) throws IOException {
 
+        Table table = null;
+        Admin admin = null;
         try {
+            table = hbaseConnection.getTable(TableName.valueOf(tableName));
+            admin = hbaseConnection.getAdmin();
+
             if (!enabled(table.getConfiguration())) {
                 logger.info("Region size calculation disabled.");
                 return;
             }
 
-            logger.info("Calculating region sizes for table \"" + new String(table.getTableName()) + "\".");
+            logger.info("Calculating region sizes for table \"" + table.getName() + "\".");
 
             // Get regions for table.
-            Set<HRegionInfo> tableRegionInfos = table.getRegionLocations().keySet();
+            RegionLocator regionLocator = hbaseConnection.getRegionLocator(table.getName());
+            List<HRegionLocation> regionLocationList = regionLocator.getAllRegionLocations();
             Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
 
-            for (HRegionInfo regionInfo : tableRegionInfos) {
-                tableRegions.add(regionInfo.getRegionName());
+            for (HRegionLocation hRegionLocation : regionLocationList) {
+                tableRegions.add(hRegionLocation.getRegionInfo().getRegionName());
             }
 
-            ClusterStatus clusterStatus = hBaseAdmin.getClusterStatus();
+            ClusterStatus clusterStatus = admin.getClusterStatus();
             Collection<ServerName> servers = clusterStatus.getServers();
             final long megaByte = 1024L * 1024L;
 
@@ -105,7 +110,7 @@ public class HBaseRegionSizeCalculator {
                 }
             }
         } finally {
-            IOUtils.closeQuietly(hBaseAdmin);
+            IOUtils.closeQuietly(admin);
         }
 
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 266f7e7..a2f60d4 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,9 +23,10 @@ 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.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 
@@ -42,8 +43,8 @@ public class HBaseUsage {
         Map<String, List<String>> envs = Maps.newHashMap();
 
         // get all kylin hbase tables
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         for (HTableDescriptor desc : tableDescriptors) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 1db60fb..8dd2164 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.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.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 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 {
-        HConnection conn = getConnection();
-        HBaseAdmin hadmin = new HBaseAdmin(conn);
+        Connection conn = getConnection();
+        Admin hadmin = conn.getAdmin();
 
         try {
-            boolean tableExist = hadmin.tableExists(tableName);
+            boolean tableExist = hadmin.tableExists(TableName.valueOf(tableName));
             if (tableExist) {
                 logger.info("HTable '" + tableName + "' already exists");
                 return;
@@ -120,8 +120,8 @@ public class HbaseStreamingInput {
                 e.printStackTrace();
             }
 
-            HConnection conn = getConnection();
-            HTableInterface table = conn.getTable(tableName);
+            Connection conn = getConnection();
+            Table table = conn.getTable(TableName.valueOf(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.add(CF, QN, cell);
+                put.addColumn(CF, QN, cell);
                 buffer.add(put);
             }
             table.put(buffer);
@@ -172,8 +172,8 @@ public class HbaseStreamingInput {
             }
 
             Random r = new Random();
-            HConnection conn = getConnection();
-            HTableInterface table = conn.getTable(tableName);
+            Connection conn = getConnection();
+            Table table = conn.getTable(TableName.valueOf(tableName));
 
             long leftBound = getFirstKeyTime(table);
             long rightBound = System.currentTimeMillis();
@@ -208,7 +208,7 @@ public class HbaseStreamingInput {
         }
     }
 
-    private static long getFirstKeyTime(HTableInterface table) throws IOException {
+    private static long getFirstKeyTime(Table table) throws IOException {
         long startTime = 0;
 
         Scan scan = new Scan();
@@ -226,8 +226,8 @@ public class HbaseStreamingInput {
 
     }
 
-    private static HConnection getConnection() throws IOException {
-        return HConnectionManager.createConnection(HBaseConnection.getCurrentHBaseConfiguration());
+    private static Connection getConnection() throws IOException {
+        return HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
     }
 
     private static String formatTime(long time) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 ca1a060..ea05ab2 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,10 +23,11 @@ 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.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.engine.mr.common.BatchConstants;
@@ -50,8 +51,8 @@ public class HtableAlterMetadataCLI extends AbstractApplication {
     String metadataValue;
 
     private void alter() throws IOException {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
         HTableDescriptor table = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
 
         hbaseAdmin.disableTable(table.getTableName());

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 8ff5b0f..df4e912 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,10 +30,14 @@ 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.client.HBaseAdmin;
+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.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;
 
@@ -52,9 +56,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
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Admin hbaseAdmin = conn.getAdmin();
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         List<String> allTablesNeedToBeDropped = new ArrayList<String>();
@@ -73,12 +77,13 @@ public class OrphanHBaseCleanJob extends AbstractApplication {
             // drop tables
             for (String htableName : allTablesNeedToBeDropped) {
                 logger.info("Deleting HBase table " + htableName);
-                if (hbaseAdmin.tableExists(htableName)) {
-                    if (hbaseAdmin.isTableEnabled(htableName)) {
-                        hbaseAdmin.disableTable(htableName);
+                TableName tableName = TableName.valueOf(htableName);
+                if (hbaseAdmin.tableExists(tableName)) {
+                    if (hbaseAdmin.isTableEnabled(tableName)) {
+                        hbaseAdmin.disableTable(tableName);
                     }
 
-                    hbaseAdmin.deleteTable(htableName);
+                    hbaseAdmin.deleteTable(tableName);
                     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/a34f8151/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 1ea8e8d..bba6745 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,12 +22,13 @@ import java.io.IOException;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
-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.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 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;
@@ -59,12 +60,12 @@ public class PingHBaseCLI {
         Scan scan = new Scan();
         int limit = 20;
 
-        HConnection conn = null;
-        HTableInterface table = null;
+        Connection conn = null;
+        Table table = null;
         ResultScanner scanner = null;
         try {
-            conn = HConnectionManager.createConnection(hconf);
-            table = conn.getTable(hbaseTable);
+            conn = ConnectionFactory.createConnection(hconf);
+            table = conn.getTable(TableName.valueOf(hbaseTable));
             scanner = table.getScanner(scan);
             int count = 0;
             for (Result r : scanner) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 01edb1f..db516bb 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,11 +22,12 @@ import java.io.IOException;
 import java.util.Iterator;
 
 import org.apache.hadoop.conf.Configuration;
-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.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 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;
@@ -70,8 +71,8 @@ public class RowCounterCLI {
 
         logger.info("My Scan " + scan.toString());
 
-        HConnection conn = HConnectionManager.createConnection(conf);
-        HTableInterface tableInterface = conn.getTable(htableName);
+        Connection conn = ConnectionFactory.createConnection(conf);
+        Table tableInterface = conn.getTable(TableName.valueOf(htableName));
 
         Iterator<Result> iterator = tableInterface.getScanner(scan).iterator();
         int counter = 0;

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 d1a74ad..f3d7649 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,7 +40,9 @@ 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.client.HBaseAdmin;
+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.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.CliCommandExecutor;
@@ -57,6 +59,7 @@ 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;
 
@@ -77,7 +80,8 @@ public class StorageCleanupJob extends AbstractApplication {
     private void cleanUnusedHBaseTables(Configuration conf) throws IOException {
         CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
         // get all kylin hbase tables
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         List<String> allTablesNeedToBeDropped = new ArrayList<String>();
@@ -153,22 +157,22 @@ public class StorageCleanupJob extends AbstractApplication {
     }
 
     class DeleteHTableRunnable implements Callable {
-        HBaseAdmin hbaseAdmin;
+        Admin hbaseAdmin;
         String htableName;
 
-        DeleteHTableRunnable(HBaseAdmin hbaseAdmin, String htableName) {
+        DeleteHTableRunnable(Admin hbaseAdmin, String htableName) {
             this.hbaseAdmin = hbaseAdmin;
             this.htableName = htableName;
         }
 
         public Object call() throws Exception {
             logger.info("Deleting HBase table " + htableName);
-            if (hbaseAdmin.tableExists(htableName)) {
-                if (hbaseAdmin.isTableEnabled(htableName)) {
-                    hbaseAdmin.disableTable(htableName);
+            if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
+                if (hbaseAdmin.isTableEnabled(TableName.valueOf(htableName))) {
+                    hbaseAdmin.disableTable(TableName.valueOf(htableName));
                 }
 
-                hbaseAdmin.deleteTable(htableName);
+                hbaseAdmin.deleteTable(TableName.valueOf(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/a34f8151/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 e36f662..42a54c8 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,16 +24,18 @@ 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.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 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;
 
@@ -49,14 +51,15 @@ public class UpdateHTableHostCLI {
     private List<String> errorMsgs = Lists.newArrayList();
 
     private List<String> htables;
-    private HBaseAdmin hbaseAdmin;
+    private Admin hbaseAdmin;
     private KylinConfig kylinConfig;
     private String oldHostValue;
 
     public UpdateHTableHostCLI(List<String> htables, String oldHostValue) throws IOException {
         this.htables = htables;
         this.oldHostValue = oldHostValue;
-        this.hbaseAdmin = new HBaseAdmin(HBaseConnection.getCurrentHBaseConfiguration());
+        Connection conn = ConnectionFactory.createConnection(HBaseConfiguration.create());
+        hbaseAdmin = conn.getAdmin();
         this.kylinConfig = KylinConfig.getInstanceFromEnv();
     }
 
@@ -166,9 +169,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);
-            hbaseAdmin.modifyTable(tableName, desc);
-            hbaseAdmin.enableTable(tableName);
+            hbaseAdmin.disableTable(TableName.valueOf(tableName));
+            hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+            hbaseAdmin.enableTable(TableName.valueOf(tableName));
 
             updatedResources.add(tableName);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/a34f8151/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 5269195..48d3be8 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,6 +231,7 @@ 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))
@@ -447,11 +448,11 @@ public class CubeMigrationCLI {
             Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
             ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
             String projUUID = project.getUuid();
-            HTableInterface srcAclHtable = null;
-            HTableInterface destAclHtable = null;
+            Table srcAclHtable = null;
+            Table destAclHtable = null;
             try {
-                srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
-                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+                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));
 
                 // cube acl
                 Result result = srcAclHtable.get(new Get(Bytes.toBytes(cubeId)));
@@ -471,7 +472,6 @@ public class CubeMigrationCLI {
                         destAclHtable.put(put);
                     }
                 }
-                destAclHtable.flushCommits();
             } finally {
                 IOUtils.closeQuietly(srcAclHtable);
                 IOUtils.closeQuietly(destAclHtable);
@@ -537,13 +537,12 @@ public class CubeMigrationCLI {
         case COPY_ACL: {
             String cubeId = (String) opt.params[0];
             String modelId = (String) opt.params[1];
-            HTableInterface destAclHtable = null;
+            Table destAclHtable = null;
             try {
-                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(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);
             }
@@ -560,7 +559,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/a34f8151/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 19e5db0..f52fc3e 100644
--- a/tool/src/main/java/org/apache/kylin/tool/ExtendCubeToHybridCLI.java
+++ b/tool/src/main/java/org/apache/kylin/tool/ExtendCubeToHybridCLI.java
@@ -25,10 +25,11 @@ 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;
@@ -231,9 +232,9 @@ public class ExtendCubeToHybridCLI {
         Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
         ProjectInstance project = store.getResource(projectResPath, ProjectInstance.class, projectSerializer);
         String projUUID = project.getUuid();
-        HTableInterface aclHtable = null;
+        Table aclHtable = null;
         try {
-            aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(kylinConfig.getMetadataUrlPrefix() + "_acl");
+            aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(TableName.valueOf(kylinConfig.getMetadataUrlPrefix() + "_acl"));
 
             // cube acl
             Result result = aclHtable.get(new Get(Bytes.toBytes(origCubeId)));
@@ -253,7 +254,6 @@ public class ExtendCubeToHybridCLI {
                     aclHtable.put(put);
                 }
             }
-            aclHtable.flushCommits();
         } finally {
             IOUtils.closeQuietly(aclHtable);
         }