You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2017/02/09 09:37:28 UTC

[1/3] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API [Forced Update!]

Repository: kylin
Updated Branches:
  refs/heads/yang22-cdh5.7 123849433 -> 2a0fd52e5 (forced update)


http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/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/2f12bb4c/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/2f12bb4c/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 68c0a39..581de38 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/2f12bb4c/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 ------");

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/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 8f69c18..8f7430e 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.");
         }
     }
@@ -190,7 +192,7 @@ public class DeployCoprocessorCLI {
         desc.addCoprocessor(CubeObserverClass, hdfsCoprocessorJar, 1002, 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));
 
@@ -205,7 +207,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(CubeObserverClass)) {
             desc.removeCoprocessor(CubeObserverClass);
@@ -231,16 +233,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());
@@ -261,12 +262,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;
@@ -387,7 +388,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/2f12bb4c/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/2f12bb4c/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/2f12bb4c/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/2f12bb4c/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/2f12bb4c/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/2f12bb4c/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 e26c8e8..da13fa4 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;
@@ -119,8 +119,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
 
@@ -135,7 +135,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);
@@ -170,8 +170,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();
@@ -206,7 +206,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();
@@ -224,8 +224,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/2f12bb4c/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/2f12bb4c/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/2f12bb4c/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 e219c5a..8a93160 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;
@@ -58,12 +59,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/2f12bb4c/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/2f12bb4c/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 62af2c9..0784305 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/2f12bb4c/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/2f12bb4c/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
index c25b690..4695353 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.gridtable.StorageSideBehavior;
 import org.apache.kylin.metadata.datatype.LongMutable;
@@ -229,15 +230,8 @@ public class AggregateRegionObserverTest {
             return nextRaw(results);
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.InternalScanner#next(java.util
-         * .List, int)
-         */
         @Override
-        public boolean next(List<Cell> result, int limit) throws IOException {
+        public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
@@ -306,6 +300,11 @@ public class AggregateRegionObserverTest {
             return 0;
         }
 
+        @Override
+        public int getBatch() {
+            return 0;
+        }
+
         /*
          * (non-Javadoc)
          * 
@@ -322,16 +321,9 @@ public class AggregateRegionObserverTest {
             return i < input.size();
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#nextRaw(java.util
-         * .List, int)
-         */
         @Override
-        public boolean nextRaw(List<Cell> result, int limit) throws IOException {
-            return nextRaw(result);
+        public boolean nextRaw(List<Cell> list, ScannerContext scannerContext) throws IOException {
+            return false;
         }
 
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
index 1d85922..04e2e8b 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FilterList.Operator;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -136,7 +137,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
 
                         Put p = new Put(rk);
                         p.setDurability(Durability.SKIP_WAL);
-                        p.add(cf.getBytes(), cq, Bytes.toBytes(c));
+                        p.addColumn(cf.getBytes(), cq, Bytes.toBytes(c));
                         ht.put(p);
                     }
                 }
@@ -224,7 +225,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
         scan.addFamily(cf.getBytes());
         scan.setFilter(filter);
         List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(table.getBytes());
-        HRegion first = regions.get(0);
+        Region first = regions.get(0);
         first.getScanner(scan);
         RegionScanner scanner = first.getScanner(scan);
         List<Cell> results = new ArrayList<Cell>();

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/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 c8bff89..c0042f3 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/2f12bb4c/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);
         }


[3/3] 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/2a0fd52e
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/2a0fd52e
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/2a0fd52e

Branch: refs/heads/yang22-cdh5.7
Commit: 2a0fd52e53edd5d6d6d5bd722248c514c43f6c4e
Parents: 2f12bb4
Author: Lynne Jiang <ly...@hotmail.com>
Authored: Mon May 16 03:33:27 2016 -0700
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Feb 9 17:35:48 2017 +0800

----------------------------------------------------------------------
 dev-support/test_all_against_hdp_2_2_4_2_2.sh   |   0
 .../kylin/engine/mr/steps/MockupMapContext.java |  15 +-
 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   |  19 +-
 tool/pom.xml                                    |  12 +
 11 files changed, 428 insertions(+), 1131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/2a0fd52e/dev-support/test_all_against_hdp_2_2_4_2_2.sh
----------------------------------------------------------------------
diff --git a/dev-support/test_all_against_hdp_2_2_4_2_2.sh b/dev-support/test_all_against_hdp_2_2_4_2_2.sh
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/kylin/blob/2a0fd52e/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 847071d..9900465 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
@@ -77,6 +77,7 @@ public class MockupMapContext {
                     outKV[0] = key;
                     outKV[1] = value;
                 }
+
             }
 
             @Override
@@ -99,6 +100,7 @@ public class MockupMapContext {
                 throw new NotImplementedException();
             }
 
+
             @Override
             public float getProgress() {
                 throw new NotImplementedException();
@@ -195,17 +197,17 @@ public class MockupMapContext {
             }
 
             @Override
-            public RawComparator<?> getSortComparator() {
+            public boolean userClassesTakesPrecedence() {
                 throw new NotImplementedException();
             }
 
             @Override
-            public String getJar() {
+            public RawComparator<?> getSortComparator() {
                 throw new NotImplementedException();
             }
 
             @Override
-            public RawComparator<?> getGroupingComparator() {
+            public String getJar() {
                 throw new NotImplementedException();
             }
 
@@ -221,7 +223,7 @@ public class MockupMapContext {
 
             @Override
             public boolean getProfileEnabled() {
-                throw new NotImplementedException();
+                return false;
             }
 
             @Override
@@ -308,6 +310,11 @@ public class MockupMapContext {
             public RawComparator<?> getCombinerKeyGroupingComparator() {
                 throw new NotImplementedException();
             }
+
+            @Override
+            public RawComparator<?> getGroupingComparator() {
+                return null;
+            }
         });
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2a0fd52e/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/2a0fd52e/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/2a0fd52e/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/2a0fd52e/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/2a0fd52e/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/2a0fd52e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6d3425e..9b857b5 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.0.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>
@@ -814,6 +814,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/2a0fd52e/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index cf92fb1..20f4483 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -112,6 +112,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 +135,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 +154,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 +173,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 +192,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 +219,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 +238,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>
@@ -229,6 +257,10 @@
                     <groupId>javax.servlet.jsp</groupId>
                     <artifactId>jsp-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
 
@@ -262,6 +294,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/2a0fd52e/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 d5c3f60..5adf327 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
@@ -100,11 +100,6 @@ public class MockupMapContext {
             }
 
             @Override
-            public float getProgress() {
-                throw new NotImplementedException();
-            }
-
-            @Override
             public Counter getCounter(Enum<?> counterName) {
                 throw new NotImplementedException();
             }
@@ -165,6 +160,11 @@ public class MockupMapContext {
             }
 
             @Override
+            public boolean userClassesTakesPrecedence() {
+                return false;
+            }
+
+            @Override
             public Class<? extends InputFormat<?, ?>> getInputFormatClass() throws ClassNotFoundException {
                 throw new NotImplementedException();
             }
@@ -214,10 +214,6 @@ public class MockupMapContext {
                 throw new NotImplementedException();
             }
 
-            @Override
-            public boolean getTaskCleanupNeeded() {
-                throw new NotImplementedException();
-            }
 
             @Override
             public boolean getProfileEnabled() {
@@ -230,11 +226,6 @@ public class MockupMapContext {
             }
 
             @Override
-            public IntegerRanges getProfileTaskRange(boolean isMap) {
-                throw new NotImplementedException();
-            }
-
-            @Override
             public String getUser() {
                 throw new NotImplementedException();
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2a0fd52e/tool/pom.xml
----------------------------------------------------------------------
diff --git a/tool/pom.xml b/tool/pom.xml
index f4afdf5..8629b25 100644
--- a/tool/pom.xml
+++ b/tool/pom.xml
@@ -49,6 +49,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>


[2/3] 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/2f12bb4c
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/2f12bb4c
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/2f12bb4c

Branch: refs/heads/yang22-cdh5.7
Commit: 2f12bb4c6712f41231cfa3f1174647e56c02950c
Parents: 5ef2480
Author: shaofengshi <sh...@apache.org>
Authored: Wed Mar 23 17:07:05 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Feb 9 17:34:11 2017 +0800

----------------------------------------------------------------------
 examples/test_case_data/sandbox/hbase-site.xml  | 19 +---
 .../kylin/provision/BuildCubeWithEngine.java    | 12 +--
 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 +++----
 .../kylin/storage/hbase/HBaseStorage.java       |  3 +-
 .../storage/hbase/cube/SimpleHBaseStore.java    | 20 ++---
 .../hbase/cube/v1/CubeSegmentTupleIterator.java | 11 +--
 .../storage/hbase/cube/v1/CubeStorageQuery.java |  4 +-
 .../hbase/cube/v1/RegionScannerAdapter.java     | 10 ++-
 .../cube/v1/SerializedHBaseTupleIterator.java   |  4 +-
 .../observer/AggregateRegionObserver.java       |  4 +-
 .../observer/AggregationScanner.java            | 14 ++-
 .../observer/ObserverAggregationCache.java      | 10 ++-
 .../coprocessor/observer/ObserverEnabler.java   |  4 +-
 .../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   | 26 +++---
 .../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 ++--
 .../observer/AggregateRegionObserverTest.java   | 26 ++----
 .../v1/filter/TestFuzzyRowFilterV2EndToEnd.java |  5 +-
 .../org/apache/kylin/tool/CubeMigrationCLI.java | 19 ++--
 .../kylin/tool/ExtendCubeToHybridCLI.java       |  8 +-
 50 files changed, 418 insertions(+), 475 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/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/2f12bb4c/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 7007039..ba9511c 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
@@ -35,8 +35,7 @@ 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 +57,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;
@@ -430,10 +430,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/2f12bb4c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 51479c8..6d3425e 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.0.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/2f12bb4c/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/2f12bb4c/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/2f12bb4c/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/2f12bb4c/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/2f12bb4c/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/2f12bb4c/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 4657cd2..9115dcd 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
@@ -28,9 +28,7 @@ import java.util.Map;
 import java.util.Set;
 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.HadoopUtil;
 import org.apache.kylin.common.util.Pair;
@@ -406,33 +404,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/2f12bb4c/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..b07e698 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/2f12bb4c/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/2f12bb4c/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 335bfe7..53c95cb 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/2f12bb4c/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 6217350..1c45967 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/2f12bb4c/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
index 43b65cb..d36d722 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
@@ -18,7 +18,6 @@
 
 package org.apache.kylin.storage.hbase;
 
-import com.google.common.base.Preconditions;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.debug.BackdoorToggles;
 import org.apache.kylin.cube.CubeInstance;
@@ -36,6 +35,8 @@ import org.apache.kylin.storage.IStorageQuery;
 import org.apache.kylin.storage.hbase.steps.HBaseMROutput;
 import org.apache.kylin.storage.hbase.steps.HBaseMROutput2Transition;
 
+import com.google.common.base.Preconditions;
+
 @SuppressWarnings("unused")
 //used by reflection
 public class HBaseStorage implements IStorage {

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/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/2f12bb4c/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java
index 8ac3832..982a044 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java
@@ -25,11 +25,12 @@ import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.Set;
 
-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.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FuzzyRowFilter;
@@ -70,7 +71,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
     protected final List<RowValueDecoder> rowValueDecoders;
     private final StorageContext context;
     private final String tableName;
-    private final HTableInterface table;
+    private final Table table;
 
     protected CubeTupleConverter tupleConverter;
     protected final Iterator<HBaseKeyRange> rangeIterator;
@@ -88,7 +89,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
     private int advMeasureRowsRemaining;
     private int advMeasureRowIndex;
 
-    public CubeSegmentTupleIterator(CubeSegment cubeSeg, List<HBaseKeyRange> keyRanges, HConnection conn, //
+    public CubeSegmentTupleIterator(CubeSegment cubeSeg, List<HBaseKeyRange> keyRanges, Connection conn, //
             Set<TblColRef> dimensions, TupleFilter filter, Set<TblColRef> groupBy, //
             List<RowValueDecoder> rowValueDecoders, StorageContext context, TupleInfo returnTupleInfo) {
         this.cubeSeg = cubeSeg;
@@ -108,7 +109,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
         this.rangeIterator = keyRanges.iterator();
 
         try {
-            this.table = conn.getTable(tableName);
+            this.table = conn.getTable(TableName.valueOf(tableName));
         } catch (Throwable t) {
             throw new StorageException("Error when open connection to table " + tableName, t);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
index 1b08880..13c9c47 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
@@ -33,7 +33,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.Dictionary;
@@ -156,7 +156,7 @@ public class CubeStorageQuery implements IStorageQuery {
         setCoprocessor(groupsCopD, valueDecoders, context); // enable coprocessor if beneficial
         setLimit(filter, context);
 
-        HConnection conn = HBaseConnection.get(context.getConnUrl());
+        Connection conn = HBaseConnection.get(context.getConnUrl());
 
         // notice we're passing filterD down to storage instead of flatFilter
         return new SerializedHBaseTupleIterator(conn, scans, cubeInstance, dimensionsD, filterD, groupsCopD, valueDecoders, context, returnTupleInfo);

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java
index 8a20c65..3d30767 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 
 /**
  * @author yangli9
@@ -50,7 +51,7 @@ public class RegionScannerAdapter implements RegionScanner {
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
         return next(result);
     }
 
@@ -60,7 +61,7 @@ public class RegionScannerAdapter implements RegionScanner {
     }
 
     @Override
-    public boolean nextRaw(List<Cell> result, int limit) throws IOException {
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
         return next(result);
     }
 
@@ -94,4 +95,9 @@ public class RegionScannerAdapter implements RegionScanner {
         return Long.MAX_VALUE;
     }
 
+    @Override
+    public int getBatch() {
+        return -1;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
index c4f7367..0f2e2fb 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
@@ -25,7 +25,7 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.metadata.filter.TupleFilter;
@@ -57,7 +57,7 @@ public class SerializedHBaseTupleIterator implements ITupleIterator {
     private int scanCount;
     private ITuple next;
 
-    public SerializedHBaseTupleIterator(HConnection conn, List<HBaseKeyRange> segmentKeyRanges, CubeInstance cube, //
+    public SerializedHBaseTupleIterator(Connection conn, List<HBaseKeyRange> segmentKeyRanges, CubeInstance cube, //
             Set<TblColRef> dimensions, TupleFilter filter, Set<TblColRef> groupBy, List<RowValueDecoder> rowValueDecoders, //
             StorageContext context, TupleInfo returnTupleInfo) {
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java
index 7139ca7..7e25e4c 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.kylin.gridtable.StorageSideBehavior;
@@ -99,7 +99,7 @@ public class AggregateRegionObserver extends BaseRegionObserver {
         // start/end region operation & sync on scanner is suggested by the
         // javadoc of RegionScanner.nextRaw()
         // FIXME: will the lock still work when a iterator is returned? is it safe? Is readonly attribute helping here? by mhb
-        HRegion region = ctxt.getEnvironment().getRegion();
+        Region region = ctxt.getEnvironment().getRegion();
         region.startRegionOperation();
         try {
             synchronized (innerScanner) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
index a900ea1..d64f48f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
@@ -25,6 +25,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.gridtable.StorageSideBehavior;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
@@ -116,8 +117,8 @@ public class AggregationScanner implements RegionScanner {
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
-        return outerScanner.next(result, limit);
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return outerScanner.next(result, scannerContext);
     }
 
     @Override
@@ -126,8 +127,8 @@ public class AggregationScanner implements RegionScanner {
     }
 
     @Override
-    public boolean nextRaw(List<Cell> result, int limit) throws IOException {
-        return outerScanner.nextRaw(result, limit);
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return outerScanner.nextRaw(result, scannerContext);
     }
 
     @Override
@@ -160,6 +161,11 @@ public class AggregationScanner implements RegionScanner {
         return outerScanner.getMvccReadPoint();
     }
 
+    @Override
+    public int getBatch() {
+        return outerScanner.getBatch();
+    }
+
     private static class Stats {
         long inputRows = 0;
         long inputBytes = 0;

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
index 8404262..331e34d 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggregationCache;
@@ -112,7 +113,7 @@ public class ObserverAggregationCache extends AggregationCache {
         }
 
         @Override
-        public boolean next(List<Cell> result, int limit) throws IOException {
+        public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
@@ -122,7 +123,7 @@ public class ObserverAggregationCache extends AggregationCache {
         }
 
         @Override
-        public boolean nextRaw(List<Cell> result, int limit) throws IOException {
+        public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
@@ -161,6 +162,11 @@ public class ObserverAggregationCache extends AggregationCache {
             // AggregateRegionObserver.LOG.info("Kylin Scanner getMvccReadPoint()");
             return Long.MAX_VALUE;
         }
+
+        @Override
+        public int getBatch() {
+            return innerScanner.getBatch();
+        }
     }
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverEnabler.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverEnabler.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverEnabler.java
index 394b3e2..9fd33f5 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverEnabler.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverEnabler.java
@@ -23,9 +23,9 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.client.HTableInterface;
 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.RegionScanner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.debug.BackdoorToggles;
@@ -60,7 +60,7 @@ public class ObserverEnabler {
     static final Map<String, Boolean> CUBE_OVERRIDES = Maps.newConcurrentMap();
 
     public static ResultScanner scanWithCoprocessorIfBeneficial(CubeSegment segment, Cuboid cuboid, TupleFilter tupleFiler, //
-            Collection<TblColRef> groupBy, Collection<RowValueDecoder> rowValueDecoders, StorageContext context, HTableInterface table, Scan scan) throws IOException {
+            Collection<TblColRef> groupBy, Collection<RowValueDecoder> rowValueDecoders, StorageContext context, Table table, Scan scan) throws IOException {
 
         if (context.isCoprocessorEnabled() == false) {
             return table.getScanner(scan);

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/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/2f12bb4c/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/2f12bb4c/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 de53d0d..85812f8 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/2f12bb4c/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/2f12bb4c/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 eacff9f..d029b7a 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.HadoopUtil;
 import org.apache.kylin.common.util.HiveCmdBuilder;
 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 {
@@ -193,4 +195,4 @@ public class DeprecatedGCStep extends AbstractExecutable {
         return getParam(OLD_HIVE_TABLE);
     }
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f12bb4c/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();