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/06/05 05:23:38 UTC

[50/67] [abbrv] kylin git commit: KYLIN-2535 Use ResourceStore to manage ACL and saved queries

KYLIN-2535 Use ResourceStore to manage ACL and saved queries


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

Branch: refs/heads/master
Commit: afaa95a099d69105198b73fbf92622975dbf766c
Parents: 0523823
Author: Yang Li <li...@apache.org>
Authored: Tue May 30 00:40:50 2017 +0800
Committer: nichunen <zj...@sjtu.org>
Committed: Tue May 30 13:20:11 2017 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/KylinConfigBase.java    |  32 +-
 .../kylin/common/persistence/ResourceStore.java |   7 +-
 .../apache/kylin/common/util/ZooKeeperUtil.java |  59 +++
 .../common/persistence/ResourceStoreTest.java   |   3 +-
 .../hbase/ITAclTableMigrationToolTest.java      |  15 +-
 .../storage/hbase/ITHBaseResourceStoreTest.java |   8 +-
 .../rest/job/StorageCleanJobHbaseUtil.java      | 127 +++++++
 .../kylin/rest/job/StorageCleanupJob.java       | 113 +-----
 .../apache/kylin/rest/security/AclConstant.java |  47 +++
 .../kylin/rest/security/AclHBaseStorage.java    |  42 ---
 .../rest/security/MockAclHBaseStorage.java      |  83 -----
 .../rest/security/RealAclHBaseStorage.java      |  71 ----
 .../rest/service/AclTableMigrationTool.java     |  48 +--
 .../apache/kylin/rest/service/CubeService.java  |  32 +-
 .../kylin/rest/service/HBaseInfoUtil.java       |  58 +++
 .../kylin/rest/service/LegacyAclService.java    | 368 -------------------
 .../kylin/rest/service/LegacyUserService.java   | 237 ------------
 .../apache/kylin/rest/service/QueryService.java | 142 +++----
 .../src/main/resources/applicationContext.xml   |   7 -
 .../kylin/storage/hbase/HBaseResourceStore.java |  26 +-
 .../storage/hbase/util/ZookeeperJobLock.java    |   5 +-
 .../kylin/storage/hbase/util/ZookeeperUtil.java |  32 +-
 .../kylin/storage/hdfs/HDFSResourceStore.java   |   7 +-
 .../org/apache/kylin/tool/DiagnosisInfoCLI.java |  10 +-
 24 files changed, 500 insertions(+), 1079 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 7f366d8..77c2987 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.kylin.common.lock.DistributedLockFactory;
 import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.common.util.CliCommandExecutor;
+import org.apache.kylin.common.util.ZooKeeperUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -188,16 +189,16 @@ abstract public class KylinConfigBase implements Serializable {
     }
 
     private String cachedHdfsWorkingDirectory;
-    
+
     public String getHdfsWorkingDirectory() {
         if (cachedHdfsWorkingDirectory != null)
             return cachedHdfsWorkingDirectory;
-        
+
         String root = getRequired("kylin.env.hdfs-working-dir");
         Path path = new Path(root);
         if (!path.isAbsolute())
             throw new IllegalArgumentException("kylin.env.hdfs-working-dir must be absolute, but got " + root);
-        
+
         // make sure path is qualified
         try {
             FileSystem fs = path.getFileSystem(new Configuration());
@@ -205,19 +206,34 @@ abstract public class KylinConfigBase implements Serializable {
         } catch (IOException e) {
             throw new RuntimeException(e);
         }
-        
+
         // append metadata-url prefix
         root = new Path(path, StringUtils.replaceChars(getMetadataUrlPrefix(), ':', '-')).toString();
-        
+
         if (!root.endsWith("/"))
             root += "/";
-        
+
         cachedHdfsWorkingDirectory = root;
         if (cachedHdfsWorkingDirectory.startsWith("file:")) {
             cachedHdfsWorkingDirectory = cachedHdfsWorkingDirectory.replace("file:", "file://");
         }
         return cachedHdfsWorkingDirectory;
     }
+    
+    /**
+     * A comma separated list of host:port pairs, each corresponding to a ZooKeeper server
+     */
+    public String getZookeeperConnectString() {
+        String str = getOptional("kylin.env.zookeeper-connect-string");
+        if (str != null)
+            return str;
+        
+        str = ZooKeeperUtil.getZKConnectStringFromHBase();
+        if (str != null)
+            return str;
+        
+        throw new RuntimeException("Please set 'kylin.env.zookeeper-connect-string' in kylin.properties");
+    }
 
     // ============================================================================
     // METADATA
@@ -962,13 +978,13 @@ abstract public class KylinConfigBase implements Serializable {
     }
 
     public boolean isAdhocEnabled() {
-        return StringUtils.isNotEmpty(getAdHocRunnerClassName()); 
+        return StringUtils.isNotEmpty(getAdHocRunnerClassName());
     }
 
     public String getAdHocRunnerClassName() {
         return getOptional("kylin.query.ad-hoc.runner.class-name", "");
     }
-    
+
     public String getAdHocConverterClassName() {
         return getOptional("kylin.query.ad-hoc.converter.class-name", "org.apache.kylin.storage.adhoc.HiveAdhocConverter");
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
index 7fb93e7..8a84968 100644
--- a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
+++ b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
@@ -119,6 +119,9 @@ abstract public class ResourceStore {
         return listResourcesImpl(path);
     }
 
+    /**
+     * return null if given path is not a folder or not exists
+     */
     abstract protected NavigableSet<String> listResourcesImpl(String folderPath) throws IOException;
 
     public String createMetaStoreUUID() throws IOException {
@@ -201,6 +204,9 @@ abstract public class ResourceStore {
         }
     }
 
+    /**
+     * return empty list if given path is not a folder or not exists
+     */
     abstract protected List<RawResource> getAllResourcesImpl(String folderPath, long timeStart, long timeEndExclusive) throws IOException;
 
     /**
@@ -452,5 +458,4 @@ abstract public class ResourceStore {
 
         return metaDirURI;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/core-common/src/main/java/org/apache/kylin/common/util/ZooKeeperUtil.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/ZooKeeperUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/ZooKeeperUtil.java
new file mode 100644
index 0000000..66fed31
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/util/ZooKeeperUtil.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.common.util;
+
+import java.util.Arrays;
+
+import javax.annotation.Nullable;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+
+/**
+ * Use reflection to get zookeeper connect string from HBase configuration.
+ */
+public class ZooKeeperUtil {
+    private static final Logger logger = LoggerFactory.getLogger(ZooKeeperUtil.class);
+
+    public static String getZKConnectStringFromHBase() {
+        Configuration hconf = null;
+        try {
+            Class<? extends Object> hbaseConnClz = ClassUtil.forName("org.apache.kylin.storage.hbase.HBaseConnection", Object.class);
+            hconf = (Configuration) hbaseConnClz.getMethod("getCurrentHBaseConfiguration").invoke(null);
+        } catch (Throwable ex) {
+            logger.warn("Failed to get zookeeper connect string from HBase configuration", ex);
+            return null;
+        }
+        
+        final String serverList = hconf.get("hbase.zookeeper.quorum");
+        final String port = hconf.get("hbase.zookeeper.property.clientPort");
+        return StringUtils.join(Iterables.transform(Arrays.asList(serverList.split(",")), new Function<String, String>() {
+            @Nullable
+            @Override
+            public String apply(String input) {
+                return input + ":" + port;
+            }
+        }), ",");
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/core-common/src/test/java/org/apache/kylin/common/persistence/ResourceStoreTest.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/persistence/ResourceStoreTest.java b/core-common/src/test/java/org/apache/kylin/common/persistence/ResourceStoreTest.java
index 91a9dfd..f183e7c 100644
--- a/core-common/src/test/java/org/apache/kylin/common/persistence/ResourceStoreTest.java
+++ b/core-common/src/test/java/org/apache/kylin/common/persistence/ResourceStoreTest.java
@@ -55,7 +55,8 @@ public class ResourceStoreTest {
     }
 
     public static String mockUrl(String tag, KylinConfig kylinConfig) {
-        return kylinConfig.getMetadataUrlPrefix() + "@" + tag;
+        String str = kylinConfig.getMetadataUrlPrefix() + "@" + tag;
+        return str;
     }
 
     private static void testAStore(ResourceStore store) throws IOException {

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITAclTableMigrationToolTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITAclTableMigrationToolTest.java b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITAclTableMigrationToolTest.java
index 65d5b52..2cb671e 100644
--- a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITAclTableMigrationToolTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITAclTableMigrationToolTest.java
@@ -41,10 +41,9 @@ import org.apache.kylin.common.persistence.ResourceStoreTest;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.rest.security.AclHBaseStorage;
+import org.apache.kylin.rest.security.AclConstant;
 import org.apache.kylin.rest.service.AclService;
 import org.apache.kylin.rest.service.AclTableMigrationTool;
-import org.apache.kylin.rest.service.LegacyUserService;
 import org.apache.kylin.rest.service.UserGrantedAuthority;
 import org.apache.kylin.rest.service.UserService;
 import org.apache.kylin.rest.util.Serializer;
@@ -67,9 +66,9 @@ public class ITAclTableMigrationToolTest extends HBaseMetadataTestCase {
 
     private Logger logger = LoggerFactory.getLogger(ITAclTableMigrationToolTest.class);
 
-    private TableName aclTable = TableName.valueOf(STORE_WITH_OLD_TABLE + AclHBaseStorage.ACL_TABLE_NAME);
+    private TableName aclTable = TableName.valueOf(STORE_WITH_OLD_TABLE + AclConstant.ACL_TABLE_NAME);
 
-    private TableName userTable = TableName.valueOf(STORE_WITH_OLD_TABLE + AclHBaseStorage.USER_TABLE_NAME);
+    private TableName userTable = TableName.valueOf(STORE_WITH_OLD_TABLE + AclConstant.USER_TABLE_NAME);
 
     private Serializer<UserGrantedAuthority[]> ugaSerializer = new Serializer<UserGrantedAuthority[]>(UserGrantedAuthority[].class);
 
@@ -127,15 +126,15 @@ public class ITAclTableMigrationToolTest extends HBaseMetadataTestCase {
     private void createTestHTables() throws IOException {
         Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
         Admin hbaseAdmin = new HBaseAdmin(conf);
-        creatTable(hbaseAdmin, conf, aclTable, new String[] { AclHBaseStorage.ACL_INFO_FAMILY, AclHBaseStorage.ACL_ACES_FAMILY });
-        creatTable(hbaseAdmin, conf, userTable, new String[] { AclHBaseStorage.USER_AUTHORITY_FAMILY });
+        creatTable(hbaseAdmin, conf, aclTable, new String[] { AclConstant.ACL_INFO_FAMILY, AclConstant.ACL_ACES_FAMILY });
+        creatTable(hbaseAdmin, conf, userTable, new String[] { AclConstant.USER_AUTHORITY_FAMILY });
     }
 
     private void addRecordsToTable() throws Exception {
         Table htable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(userTable);
         Pair<byte[], byte[]> pair = getRandomUserRecord();
         Put put = new Put(pair.getKey());
-        put.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
+        put.addColumn(Bytes.toBytes(AclConstant.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclConstant.USER_AUTHORITY_COLUMN), pair.getSecond());
         htable.put(put);
     }
 
@@ -196,7 +195,7 @@ public class ITAclTableMigrationToolTest extends HBaseMetadataTestCase {
         UserGrantedAuthority[] serializing = new UserGrantedAuthority[authorities.size() + 1];
 
         // password is stored as the [0] authority
-        serializing[0] = new UserGrantedAuthority(LegacyUserService.PWD_PREFIX + "password");
+        serializing[0] = new UserGrantedAuthority(AclConstant.PWD_PREFIX + "password");
         int i = 1;
         for (GrantedAuthority a : authorities) {
             serializing[i++] = new UserGrantedAuthority(a.getAuthority());

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
index fd91397..d879fac 100644
--- a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
@@ -18,6 +18,9 @@
 
 package org.apache.kylin.storage.hbase;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -28,11 +31,10 @@ import org.apache.kylin.common.persistence.StringEntity;
 import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
+@Ignore
 public class ITHBaseResourceStoreTest extends HBaseMetadataTestCase {
 
     private KylinConfig kylinConfig;

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/server-base/src/main/java/org/apache/kylin/rest/job/StorageCleanJobHbaseUtil.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/job/StorageCleanJobHbaseUtil.java b/server-base/src/main/java/org/apache/kylin/rest/job/StorageCleanJobHbaseUtil.java
new file mode 100644
index 0000000..3728ea1
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/job/StorageCleanJobHbaseUtil.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *  
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *  
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.rest.job;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+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.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.realization.IRealizationConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StorageCleanJobHbaseUtil {
+
+    protected static final Logger logger = LoggerFactory.getLogger(StorageCleanJobHbaseUtil.class);
+
+    public static void cleanUnusedHBaseTables(boolean delete, int deleteTimeout) throws IOException {
+        Configuration conf = HBaseConfiguration.create();
+        CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+        // get all kylin hbase tables
+        try (HBaseAdmin hbaseAdmin = new HBaseAdmin(conf)) {
+            String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
+            HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
+            List<String> allTablesNeedToBeDropped = new ArrayList<String>();
+            for (HTableDescriptor desc : tableDescriptors) {
+                String host = desc.getValue(IRealizationConstants.HTableTag);
+                if (KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix().equalsIgnoreCase(host)) {
+                    //only take care htables that belongs to self, and created more than 2 days
+                    allTablesNeedToBeDropped.add(desc.getTableName().getNameAsString());
+                }
+            }
+
+            // remove every segment htable from drop list
+            for (CubeInstance cube : cubeMgr.listAllCubes()) {
+                for (CubeSegment seg : cube.getSegments()) {
+                    String tablename = seg.getStorageLocationIdentifier();
+                    if (allTablesNeedToBeDropped.contains(tablename)) {
+                        allTablesNeedToBeDropped.remove(tablename);
+                        logger.info("Exclude table " + tablename + " from drop list, as the table belongs to cube " + cube.getName() + " with status " + cube.getStatus());
+                    }
+                }
+            }
+
+            if (delete == true) {
+                // drop tables
+                ExecutorService executorService = Executors.newSingleThreadExecutor();
+                for (String htableName : allTablesNeedToBeDropped) {
+                    FutureTask futureTask = new FutureTask(new DeleteHTableRunnable(hbaseAdmin, htableName));
+                    executorService.execute(futureTask);
+                    try {
+                        futureTask.get(deleteTimeout, TimeUnit.MINUTES);
+                    } catch (TimeoutException e) {
+                        logger.warn("It fails to delete htable " + htableName + ", for it cost more than " + deleteTimeout + " minutes!");
+                        futureTask.cancel(true);
+                    } catch (Exception e) {
+                        e.printStackTrace();
+                        futureTask.cancel(true);
+                    }
+                }
+                executorService.shutdown();
+            } else {
+                System.out.println("--------------- Tables To Be Dropped ---------------");
+                for (String htableName : allTablesNeedToBeDropped) {
+                    System.out.println(htableName);
+                }
+                System.out.println("----------------------------------------------------");
+            }
+        } catch (IOException e) {
+            throw new IOException(e);
+        }
+    }
+
+    static class DeleteHTableRunnable implements Callable {
+        HBaseAdmin hbaseAdmin;
+        String htableName;
+
+        DeleteHTableRunnable(HBaseAdmin hbaseAdmin, String htableName) {
+            this.hbaseAdmin = hbaseAdmin;
+            this.htableName = htableName;
+        }
+
+        public Object call() throws Exception {
+            logger.info("Deleting HBase table " + htableName);
+            if (hbaseAdmin.tableExists(htableName)) {
+                if (hbaseAdmin.isTableEnabled(htableName)) {
+                    hbaseAdmin.disableTable(htableName);
+                }
+
+                hbaseAdmin.deleteTable(htableName);
+                logger.info("Deleted HBase table " + htableName);
+            } else {
+                logger.info("HBase table" + htableName + " does not exist");
+            }
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/server-base/src/main/java/org/apache/kylin/rest/job/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/job/StorageCleanupJob.java b/server-base/src/main/java/org/apache/kylin/rest/job/StorageCleanupJob.java
index d92107b..448e3c6 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/job/StorageCleanupJob.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/job/StorageCleanupJob.java
@@ -19,15 +19,10 @@
 package org.apache.kylin.rest.job;
 
 import java.io.IOException;
+import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.regex.Pattern;
 
 import javax.annotation.Nullable;
@@ -39,9 +34,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 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.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.CliCommandExecutor;
@@ -56,9 +48,8 @@ import org.apache.kylin.job.engine.JobEngineConfig;
 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.source.SourceFactory;
 import org.apache.kylin.source.ISourceMetadataExplorer;
+import org.apache.kylin.source.SourceFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -79,55 +70,16 @@ public class StorageCleanupJob extends AbstractApplication {
     protected boolean force = false;
     protected static ExecutableManager executableManager = ExecutableManager.getInstance(KylinConfig.getInstanceFromEnv());
 
-    private void cleanUnusedHBaseTables(Configuration conf) throws IOException {
-        CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
-        // get all kylin hbase tables
-        try (HBaseAdmin hbaseAdmin = new HBaseAdmin(conf)) {
-            String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
-            HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
-            List<String> allTablesNeedToBeDropped = new ArrayList<String>();
-            for (HTableDescriptor desc : tableDescriptors) {
-                String host = desc.getValue(IRealizationConstants.HTableTag);
-                if (KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix().equalsIgnoreCase(host)) {
-                    //only take care htables that belongs to self, and created more than 2 days
-                    allTablesNeedToBeDropped.add(desc.getTableName().getNameAsString());
-                }
-            }
-
-            // remove every segment htable from drop list
-            for (CubeInstance cube : cubeMgr.listAllCubes()) {
-                for (CubeSegment seg : cube.getSegments()) {
-                    String tablename = seg.getStorageLocationIdentifier();
-                    if (allTablesNeedToBeDropped.contains(tablename)) {
-                        allTablesNeedToBeDropped.remove(tablename);
-                        logger.info("Exclude table " + tablename + " from drop list, as the table belongs to cube " + cube.getName() + " with status " + cube.getStatus());
-                    }
-                }
-            }
-
-            if (delete == true) {
-                // drop tables
-                ExecutorService executorService = Executors.newSingleThreadExecutor();
-                for (String htableName : allTablesNeedToBeDropped) {
-                    FutureTask futureTask = new FutureTask(new DeleteHTableRunnable(hbaseAdmin, htableName));
-                    executorService.execute(futureTask);
-                    try {
-                        futureTask.get(deleteTimeout, TimeUnit.MINUTES);
-                    } catch (TimeoutException e) {
-                        logger.warn("It fails to delete htable " + htableName + ", for it cost more than " + deleteTimeout + " minutes!");
-                        futureTask.cancel(true);
-                    } catch (Exception e) {
-                        e.printStackTrace();
-                        futureTask.cancel(true);
-                    }
-                }
-                executorService.shutdown();
-            } else {
-                System.out.println("--------------- Tables To Be Dropped ---------------");
-                for (String htableName : allTablesNeedToBeDropped) {
-                    System.out.println(htableName);
-                }
-                System.out.println("----------------------------------------------------");
+    protected void cleanUnusedHBaseTables() throws IOException {
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+        if ("hbase".equals(config.getMetadataUrl().getScheme())) {
+            try {
+                // use reflection to isolate NoClassDef errors when HBase is not available
+                Class hbaseCleanUpUtil = Class.forName("org.apache.kylin.rest.job.StorageCleanJobHbaseUtil");
+                Method cleanUnusedHBaseTables = hbaseCleanUpUtil.getDeclaredMethod("cleanUnusedHBaseTables", boolean.class, int.class);
+                cleanUnusedHBaseTables.invoke(hbaseCleanUpUtil, delete, deleteTimeout);
+            } catch (Throwable e) {
+                throw new IOException(e);
             }
         }
     }
@@ -147,41 +99,13 @@ public class StorageCleanupJob extends AbstractApplication {
         logger.info("force option value: '" + optionsHelper.getOptionValue(OPTION_FORCE) + "'");
         delete = Boolean.parseBoolean(optionsHelper.getOptionValue(OPTION_DELETE));
         force = Boolean.parseBoolean(optionsHelper.getOptionValue(OPTION_FORCE));
-
-        Configuration conf = HBaseConfiguration.create();
-
-        cleanUnusedIntermediateHiveTable(conf);
-        cleanUnusedHdfsFiles(conf);
-        cleanUnusedHBaseTables(conf);
-
-    }
-
-    class DeleteHTableRunnable implements Callable {
-        HBaseAdmin hbaseAdmin;
-        String htableName;
-
-        DeleteHTableRunnable(HBaseAdmin hbaseAdmin, String htableName) {
-            this.hbaseAdmin = hbaseAdmin;
-            this.htableName = htableName;
-        }
-
-        public Object call() throws Exception {
-            logger.info("Deleting HBase table " + htableName);
-            if (hbaseAdmin.tableExists(htableName)) {
-                if (hbaseAdmin.isTableEnabled(htableName)) {
-                    hbaseAdmin.disableTable(htableName);
-                }
-
-                hbaseAdmin.deleteTable(htableName);
-                logger.info("Deleted HBase table " + htableName);
-            } else {
-                logger.info("HBase table" + htableName + " does not exist");
-            }
-            return null;
-        }
+        cleanUnusedIntermediateHiveTable();
+        cleanUnusedHdfsFiles();
+        cleanUnusedHBaseTables();
     }
 
-    private void cleanUnusedHdfsFiles(Configuration conf) throws IOException {
+    private void cleanUnusedHdfsFiles() throws IOException {
+        Configuration conf = HadoopUtil.getCurrentConfiguration();
         JobEngineConfig engineConfig = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
         CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
 
@@ -245,7 +169,8 @@ public class StorageCleanupJob extends AbstractApplication {
         }
     }
 
-    private void cleanUnusedIntermediateHiveTable(Configuration conf) throws Exception {
+    private void cleanUnusedIntermediateHiveTable() throws Exception {
+        Configuration conf = HadoopUtil.getCurrentConfiguration();
         final KylinConfig config = KylinConfig.getInstanceFromEnv();
         JobEngineConfig engineConfig = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
         final CliCommandExecutor cmdExec = config.getCliCommandExecutor();

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/server-base/src/main/java/org/apache/kylin/rest/security/AclConstant.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/AclConstant.java b/server-base/src/main/java/org/apache/kylin/rest/security/AclConstant.java
new file mode 100644
index 0000000..7b959e1
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/AclConstant.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.rest.security;
+
+/**
+ * Created by xiefan on 17-5-3.
+ */
+public interface AclConstant {
+
+    static final String ACL_INFO_FAMILY = "i";
+
+    static final String ACL_ACES_FAMILY = "a";
+
+    static final String ACL_TABLE_NAME = "_acl";
+
+    static final String USER_AUTHORITY_FAMILY = "a";
+
+    static final String USER_TABLE_NAME = "_user";
+
+    static final String USER_AUTHORITY_COLUMN = "c";
+
+    static String ACL_INFO_FAMILY_TYPE_COLUMN = "t";
+
+    static String ACL_INFO_FAMILY_OWNER_COLUMN = "o";
+
+    static String ACL_INFO_FAMILY_PARENT_COLUMN = "p";
+
+    static String ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN = "i";
+
+    static final String PWD_PREFIX = "PWD:";
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/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
deleted file mode 100644
index b595c72..0000000
--- a/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.rest.security;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.client.Table;
-
-/**
- */
-@Deprecated  //use ResourceStore interface instead.
-public interface AclHBaseStorage {
-
-    String ACL_INFO_FAMILY = "i";
-    String ACL_ACES_FAMILY = "a";
-    String ACL_TABLE_NAME = "_acl";
-
-    String USER_AUTHORITY_FAMILY = "a";
-    String USER_TABLE_NAME = "_user";
-    String USER_AUTHORITY_COLUMN = "c";
-
-    String prepareHBaseTable(Class<?> clazz) throws IOException;
-
-    Table getTable(String tableName) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/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
deleted file mode 100644
index ca49641..0000000
--- a/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.rest.security;
-
-import java.io.IOException;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.StorageURL;
-import org.apache.kylin.rest.service.LegacyAclService;
-import org.apache.kylin.rest.service.LegacyUserService;
-import org.apache.kylin.rest.service.QueryService;
-
-/**
- */
-@Deprecated
-public class MockAclHBaseStorage implements AclHBaseStorage {
-
-    private static final String aclTableName = "MOCK-ACL-TABLE";
-    private static final String userTableName = "MOCK-USER-TABLE";
-
-    private Table mockedAclTable;
-    private Table mockedUserTable;
-    private RealAclHBaseStorage realAcl;
-
-    public MockAclHBaseStorage() {
-        StorageURL metadataUrl = KylinConfig.getInstanceFromEnv().getMetadataUrl();
-        if (metadataUrl.getScheme().endsWith("hbase")) {
-            // hbase must be available since metadata is on it
-            // in this case, let us use a real ACL instead of mockup
-            realAcl = new RealAclHBaseStorage();
-        }
-    }
-
-    @Override
-    public String prepareHBaseTable(Class<?> clazz) throws IOException {
-        if (realAcl != null) {
-            return realAcl.prepareHBaseTable(clazz);
-        }
-
-        if (clazz == LegacyAclService.class) {
-            mockedAclTable = new MockHTable(aclTableName, ACL_INFO_FAMILY, ACL_ACES_FAMILY);
-            return aclTableName;
-        } else if (clazz == LegacyUserService.class) {
-            mockedUserTable = new MockHTable(userTableName, USER_AUTHORITY_FAMILY, QueryService.USER_QUERY_FAMILY);
-            return userTableName;
-        } else {
-            throw new IllegalStateException("prepareHBaseTable for unknown class: " + clazz);
-        }
-    }
-
-    @Override
-    public Table getTable(String tableName) throws IOException {
-        if (realAcl != null) {
-            return realAcl.getTable(tableName);
-        }
-
-        if (StringUtils.equals(tableName, aclTableName)) {
-            return mockedAclTable;
-        } else if (StringUtils.equals(tableName, userTableName)) {
-            return mockedUserTable;
-        } else {
-            throw new IllegalStateException("getTable failed" + tableName);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/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
deleted file mode 100644
index 98cef3a..0000000
--- a/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.rest.security;
-
-import java.io.IOException;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.StorageURL;
-import org.apache.kylin.rest.service.LegacyAclService;
-import org.apache.kylin.rest.service.LegacyUserService;
-import org.apache.kylin.rest.service.QueryService;
-import org.apache.kylin.storage.hbase.HBaseConnection;
-
-/**
- */
-@Deprecated
-public class RealAclHBaseStorage implements AclHBaseStorage {
-
-    private StorageURL hbaseUrl;
-    private String aclTableName;
-    private String userTableName;
-
-    @Override
-    public String prepareHBaseTable(Class<?> clazz) throws IOException {
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-        hbaseUrl = kylinConfig.getMetadataUrl();
-        String tableNameBase = hbaseUrl.getIdentifier();
-
-        if (clazz == LegacyAclService.class) {
-            aclTableName = tableNameBase + ACL_TABLE_NAME;
-            HBaseConnection.createHTableIfNeeded(hbaseUrl, aclTableName, ACL_INFO_FAMILY, ACL_ACES_FAMILY);
-            return aclTableName;
-        } else if (clazz == LegacyUserService.class) {
-            userTableName = tableNameBase + USER_TABLE_NAME;
-            HBaseConnection.createHTableIfNeeded(hbaseUrl, userTableName, USER_AUTHORITY_FAMILY, QueryService.USER_QUERY_FAMILY);
-            return userTableName;
-        } else {
-            throw new IllegalStateException("prepareHBaseTable for unknown class: " + clazz);
-        }
-    }
-
-    @Override
-    public Table getTable(String tableName) throws IOException {
-        if (StringUtils.equals(tableName, aclTableName)) {
-            return HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
-        } else if (StringUtils.equals(tableName, 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/afaa95a0/server-base/src/main/java/org/apache/kylin/rest/service/AclTableMigrationTool.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/AclTableMigrationTool.java b/server-base/src/main/java/org/apache/kylin/rest/service/AclTableMigrationTool.java
index d88ca74..e22e0dc 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/AclTableMigrationTool.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/AclTableMigrationTool.java
@@ -28,19 +28,18 @@ import java.util.Map;
 import java.util.NavigableMap;
 
 import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 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.KylinConfig;
+import org.apache.kylin.common.StorageURL;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.StringEntity;
 import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.rest.security.AclHBaseStorage;
+import org.apache.kylin.rest.security.AclConstant;
 import org.apache.kylin.rest.util.Serializer;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.storage.hbase.HBaseResourceStore;
@@ -74,16 +73,16 @@ public class AclTableMigrationTool {
             }
             logger.info("Start to migrate acl table data");
             ResourceStore store = ResourceStore.getStore(kylinConfig);
-            String userTableName = kylinConfig.getMetadataUrlPrefix() + AclHBaseStorage.USER_TABLE_NAME;
+            String userTableName = kylinConfig.getMetadataUrlPrefix() + AclConstant.USER_TABLE_NAME;
             //System.out.println("user table name : " + userTableName);
-            String aclTableName = kylinConfig.getMetadataUrlPrefix() + AclHBaseStorage.ACL_TABLE_NAME;
+            String aclTableName = kylinConfig.getMetadataUrlPrefix() + AclConstant.ACL_TABLE_NAME;
             if (needMigrateTable(aclTableName, store)) {
                 logger.info("Migrate table : {}", aclTableName);
-                migrate(store, AclHBaseStorage.ACL_TABLE_NAME, kylinConfig);
+                migrate(store, AclConstant.ACL_TABLE_NAME, kylinConfig);
             }
             if (needMigrateTable(userTableName, store)) {
                 logger.info("Migrate table : {}", userTableName);
-                migrate(store, AclHBaseStorage.USER_TABLE_NAME, kylinConfig);
+                migrate(store, AclConstant.USER_TABLE_NAME, kylinConfig);
             }
         }
     }
@@ -95,8 +94,8 @@ public class AclTableMigrationTool {
             return false;
         }
 
-        String userTableName = kylinConfig.getMetadataUrlPrefix() + AclHBaseStorage.USER_TABLE_NAME;
-        String aclTableName = kylinConfig.getMetadataUrlPrefix() + AclHBaseStorage.ACL_TABLE_NAME;
+        String userTableName = kylinConfig.getMetadataUrlPrefix() + AclConstant.USER_TABLE_NAME;
+        String aclTableName = kylinConfig.getMetadataUrlPrefix() + AclConstant.ACL_TABLE_NAME;
         if (needMigrateTable(aclTableName, store) || needMigrateTable(userTableName, store))
             return true;
         return false;
@@ -112,8 +111,8 @@ public class AclTableMigrationTool {
     private void migrate(ResourceStore store, String tableType, KylinConfig kylinConfig) throws IOException {
 
         switch (tableType) {
-        case AclHBaseStorage.ACL_TABLE_NAME:
-            String aclTableName = kylinConfig.getMetadataUrlPrefix() + AclHBaseStorage.ACL_TABLE_NAME;
+        case AclConstant.ACL_TABLE_NAME:
+            String aclTableName = kylinConfig.getMetadataUrlPrefix() + AclConstant.ACL_TABLE_NAME;
             convertToResourceStore(kylinConfig, aclTableName, store, new ResultConverter() {
                 @Override
                 public void convertResult(ResultScanner rs, ResourceStore store) throws IOException {
@@ -135,8 +134,8 @@ public class AclTableMigrationTool {
                 }
             });
             break;
-        case AclHBaseStorage.USER_TABLE_NAME:
-            String userTableName = kylinConfig.getMetadataUrlPrefix() + AclHBaseStorage.USER_TABLE_NAME;
+        case AclConstant.USER_TABLE_NAME:
+            String userTableName = kylinConfig.getMetadataUrlPrefix() + AclConstant.USER_TABLE_NAME;
 
             convertToResourceStore(kylinConfig, userTableName, store, new ResultConverter() {
                 @Override
@@ -162,9 +161,10 @@ public class AclTableMigrationTool {
     }
 
     private boolean checkTableExist(String tableName) throws IOException {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        Admin hbaseAdmin = new HBaseAdmin(conf);
-        return hbaseAdmin.tableExists(TableName.valueOf(tableName));
+        StorageURL metadataUrl = KylinConfig.getInstanceFromEnv().getMetadataUrl();
+        try (Admin admin = HBaseConnection.get(metadataUrl).getAdmin()) {
+            return admin.tableExists(TableName.valueOf(tableName));
+        }
     }
 
     private boolean isTableAlreadyMigrate(ResourceStore store, String tableName) throws IOException {
@@ -189,7 +189,7 @@ public class AclTableMigrationTool {
     }
 
     private DomainObjectInfo getDomainObjectInfoFromRs(Result result) {
-        String type = String.valueOf(result.getValue(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(LegacyAclService.ACL_INFO_FAMILY_TYPE_COLUMN)));
+        String type = String.valueOf(result.getValue(Bytes.toBytes(AclConstant.ACL_INFO_FAMILY), Bytes.toBytes(AclConstant.ACL_INFO_FAMILY_TYPE_COLUMN)));
         String id = String.valueOf(result.getRow());
         DomainObjectInfo newInfo = new DomainObjectInfo();
         newInfo.setId(id);
@@ -198,23 +198,23 @@ public class AclTableMigrationTool {
     }
 
     private DomainObjectInfo getParentDomainObjectInfoFromRs(Result result) throws IOException {
-        DomainObjectInfo parentInfo = domainObjSerializer.deserialize(result.getValue(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(LegacyAclService.ACL_INFO_FAMILY_PARENT_COLUMN)));
+        DomainObjectInfo parentInfo = domainObjSerializer.deserialize(result.getValue(Bytes.toBytes(AclConstant.ACL_INFO_FAMILY), Bytes.toBytes(AclConstant.ACL_INFO_FAMILY_PARENT_COLUMN)));
         return parentInfo;
     }
 
     private boolean getInheriting(Result result) {
-        boolean entriesInheriting = Bytes.toBoolean(result.getValue(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(LegacyAclService.ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN)));
+        boolean entriesInheriting = Bytes.toBoolean(result.getValue(Bytes.toBytes(AclConstant.ACL_INFO_FAMILY), Bytes.toBytes(AclConstant.ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN)));
         return entriesInheriting;
     }
 
     private SidInfo getOwnerSidInfo(Result result) throws IOException {
-        SidInfo owner = sidSerializer.deserialize(result.getValue(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(LegacyAclService.ACL_INFO_FAMILY_OWNER_COLUMN)));
+        SidInfo owner = sidSerializer.deserialize(result.getValue(Bytes.toBytes(AclConstant.ACL_INFO_FAMILY), Bytes.toBytes(AclConstant.ACL_INFO_FAMILY_OWNER_COLUMN)));
         return owner;
     }
 
     private Map<String, AceInfo> getAllAceInfo(Result result) throws IOException {
         Map<String, AceInfo> allAceInfoMap = new HashMap<>();
-        NavigableMap<byte[], byte[]> familyMap = result.getFamilyMap(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY));
+        NavigableMap<byte[], byte[]> familyMap = result.getFamilyMap(Bytes.toBytes(AclConstant.ACL_ACES_FAMILY));
         for (Map.Entry<byte[], byte[]> entry : familyMap.entrySet()) {
             String sid = String.valueOf(entry.getKey());
             AceInfo aceInfo = aceSerializer.deserialize(familyMap.get(entry.getValue()));
@@ -245,7 +245,7 @@ public class AclTableMigrationTool {
 
         String username = Bytes.toString(result.getRow());
 
-        byte[] valueBytes = result.getValue(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN));
+        byte[] valueBytes = result.getValue(Bytes.toBytes(AclConstant.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclConstant.USER_AUTHORITY_COLUMN));
         UserGrantedAuthority[] deserialized = ugaSerializer.deserialize(valueBytes);
 
         String password = "";
@@ -253,8 +253,8 @@ public class AclTableMigrationTool {
 
         // password is stored at [0] of authorities for backward compatibility
         if (deserialized != null) {
-            if (deserialized.length > 0 && deserialized[0].getAuthority().startsWith(LegacyUserService.PWD_PREFIX)) {
-                password = deserialized[0].getAuthority().substring(LegacyUserService.PWD_PREFIX.length());
+            if (deserialized.length > 0 && deserialized[0].getAuthority().startsWith(AclConstant.PWD_PREFIX)) {
+                password = deserialized[0].getAuthority().substring(AclConstant.PWD_PREFIX.length());
                 authorities = Arrays.asList(deserialized).subList(1, deserialized.length);
             } else {
                 authorities = Arrays.asList(deserialized);

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/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 22baabe..1eaa31c 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
@@ -26,12 +26,10 @@ import java.util.Collections;
 import java.util.Date;
 import java.util.EnumSet;
 import java.util.List;
-import java.util.Map;
 import java.util.UUID;
 import java.util.WeakHashMap;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeInstance;
@@ -61,8 +59,6 @@ import org.apache.kylin.rest.request.MetricsRequest;
 import org.apache.kylin.rest.response.HBaseResponse;
 import org.apache.kylin.rest.response.MetricsResponse;
 import org.apache.kylin.rest.security.AclPermission;
-import org.apache.kylin.storage.hbase.HBaseConnection;
-import org.apache.kylin.storage.hbase.util.HBaseRegionSizeCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -405,33 +401,27 @@ public class CubeService extends BasicService {
      *
      * @param tableName The table name.
      * @return The HBaseResponse object contains table size, region count. null
-     * if error happens.
+     * if error happens
      * @throws IOException Exception when HTable resource is not closed correctly.
      */
     public HBaseResponse getHTableInfo(String tableName) throws IOException {
         if (htableInfoCache.containsKey(tableName)) {
             return htableInfoCache.get(tableName);
         }
-        Connection conn = HBaseConnection.get(this.getConfig().getStorageUrl());
-        HBaseResponse hr = null;
-        long tableSize = 0;
-        int regionCount = 0;
 
-        HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
-        Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
-
-        for (long s : sizeMap.values()) {
-            tableSize += s;
+        HBaseResponse hr = new HBaseResponse();
+        if ("hbase".equals(getConfig().getMetadataUrl().getScheme())) {
+            try {
+                // use reflection to isolate NoClassDef errors when HBase is not available
+                hr = (HBaseResponse) Class.forName("org.apache.kylin.rest.service.HBaseInfoUtil")//
+                        .getMethod("getHBaseInfo", new Class[] { String.class, String.class })//
+                        .invoke(null, new Object[] { tableName, this.getConfig().getStorageUrl() });
+            } catch (Throwable e) {
+                throw new IOException(e);
+            }
         }
 
-        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/afaa95a0/server-base/src/main/java/org/apache/kylin/rest/service/HBaseInfoUtil.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/HBaseInfoUtil.java b/server-base/src/main/java/org/apache/kylin/rest/service/HBaseInfoUtil.java
new file mode 100644
index 0000000..3f0b2b5
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/HBaseInfoUtil.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.rest.service;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.rest.response.HBaseResponse;
+import org.apache.kylin.storage.hbase.HBaseConnection;
+import org.apache.kylin.storage.hbase.util.HBaseRegionSizeCalculator;
+
+/**
+ * Created by xiefan on 17-5-5.
+ */
+public class HBaseInfoUtil {
+    public static HBaseResponse getHBaseInfo(String tableName, KylinConfig config) throws IOException {
+        if (!config.getStorageUrl().getScheme().equals("hbase"))
+            return null;
+        
+        Connection conn = HBaseConnection.get(config.getStorageUrl());
+        HBaseResponse hr = null;
+        long tableSize = 0;
+        int regionCount = 0;
+
+        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);
+        return hr;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/server-base/src/main/java/org/apache/kylin/rest/service/LegacyAclService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/LegacyAclService.java b/server-base/src/main/java/org/apache/kylin/rest/service/LegacyAclService.java
deleted file mode 100644
index 8ab6ebe..0000000
--- a/server-base/src/main/java/org/apache/kylin/rest/service/LegacyAclService.java
+++ /dev/null
@@ -1,368 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.rest.service;
-
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-
-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.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.CompareOp;
-import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.rest.security.AclHBaseStorage;
-import org.apache.kylin.rest.util.Serializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.security.acls.domain.AccessControlEntryImpl;
-import org.springframework.security.acls.domain.AclAuthorizationStrategy;
-import org.springframework.security.acls.domain.AclImpl;
-import org.springframework.security.acls.domain.AuditLogger;
-import org.springframework.security.acls.domain.GrantedAuthoritySid;
-import org.springframework.security.acls.domain.ObjectIdentityImpl;
-import org.springframework.security.acls.domain.PermissionFactory;
-import org.springframework.security.acls.domain.PrincipalSid;
-import org.springframework.security.acls.model.AccessControlEntry;
-import org.springframework.security.acls.model.Acl;
-import org.springframework.security.acls.model.AlreadyExistsException;
-import org.springframework.security.acls.model.ChildrenExistException;
-import org.springframework.security.acls.model.MutableAcl;
-import org.springframework.security.acls.model.MutableAclService;
-import org.springframework.security.acls.model.NotFoundException;
-import org.springframework.security.acls.model.ObjectIdentity;
-import org.springframework.security.acls.model.PermissionGrantingStrategy;
-import org.springframework.security.acls.model.Sid;
-import org.springframework.security.core.Authentication;
-import org.springframework.security.core.context.SecurityContextHolder;
-import org.springframework.security.util.FieldUtils;
-import org.springframework.util.Assert;
-
-import com.fasterxml.jackson.core.JsonParseException;
-import com.fasterxml.jackson.databind.JsonMappingException;
-
-/**
- * @author xduo
- */
-//@Component("aclService")
-@Deprecated
-public class LegacyAclService implements MutableAclService {
-
-    private static final Logger logger = LoggerFactory.getLogger(LegacyAclService.class);
-
-    public static String ACL_INFO_FAMILY_TYPE_COLUMN = "t";
-    public static String ACL_INFO_FAMILY_OWNER_COLUMN = "o";
-    public static String ACL_INFO_FAMILY_PARENT_COLUMN = "p";
-    public static String ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN = "i";
-
-    private Serializer<SidInfo> sidSerializer = new Serializer<SidInfo>(SidInfo.class);
-    private Serializer<DomainObjectInfo> domainObjSerializer = new Serializer<DomainObjectInfo>(DomainObjectInfo.class);
-    private Serializer<AceInfo> aceSerializer = new Serializer<AceInfo>(AceInfo.class);
-
-    private String aclTableName = null;
-
-    private final Field fieldAces = FieldUtils.getField(AclImpl.class, "aces");
-
-    private final Field fieldAcl = FieldUtils.getField(AccessControlEntryImpl.class, "acl");
-
-    @Autowired
-    protected PermissionGrantingStrategy permissionGrantingStrategy;
-
-    @Autowired
-    protected PermissionFactory aclPermissionFactory;
-
-    @Autowired
-    protected AclAuthorizationStrategy aclAuthorizationStrategy;
-
-    @Autowired
-    protected AuditLogger auditLogger;
-
-    @Autowired
-    protected AclHBaseStorage aclHBaseStorage;
-
-    public LegacyAclService() throws IOException {
-        fieldAces.setAccessible(true);
-        fieldAcl.setAccessible(true);
-    }
-
-    @PostConstruct
-    public void init() throws IOException {
-        aclTableName = aclHBaseStorage.prepareHBaseTable(LegacyAclService.class);
-    }
-
-    @Override
-    public List<ObjectIdentity> findChildren(ObjectIdentity parentIdentity) {
-        List<ObjectIdentity> oids = new ArrayList<ObjectIdentity>();
-        Table htable = null;
-        try {
-            htable = aclHBaseStorage.getTable(aclTableName);
-
-            Scan scan = new Scan();
-            SingleColumnValueFilter parentFilter = new SingleColumnValueFilter(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), CompareOp.EQUAL, domainObjSerializer.serialize(new DomainObjectInfo(parentIdentity)));
-            parentFilter.setFilterIfMissing(true);
-            scan.setFilter(parentFilter);
-
-            ResultScanner scanner = htable.getScanner(scan);
-            for (Result result = scanner.next(); result != null; result = scanner.next()) {
-                String id = Bytes.toString(result.getRow());
-                String type = Bytes.toString(result.getValue(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN)));
-
-                oids.add(new ObjectIdentityImpl(type, id));
-            }
-        } catch (IOException e) {
-            throw new RuntimeException(e.getMessage(), e);
-        } finally {
-            IOUtils.closeQuietly(htable);
-        }
-
-        return oids;
-    }
-
-    @Override
-    public Acl readAclById(ObjectIdentity object) throws NotFoundException {
-        Map<ObjectIdentity, Acl> aclsMap = readAclsById(Arrays.asList(object), null);
-        //        Assert.isTrue(aclsMap.containsKey(object), "There should have been an Acl entry for ObjectIdentity " + object);
-
-        return aclsMap.get(object);
-    }
-
-    @Override
-    public Acl readAclById(ObjectIdentity object, List<Sid> sids) throws NotFoundException {
-        Map<ObjectIdentity, Acl> aclsMap = readAclsById(Arrays.asList(object), sids);
-        Assert.isTrue(aclsMap.containsKey(object), "There should have been an Acl entry for ObjectIdentity " + object);
-
-        return aclsMap.get(object);
-    }
-
-    @Override
-    public Map<ObjectIdentity, Acl> readAclsById(List<ObjectIdentity> objects) throws NotFoundException {
-        return readAclsById(objects, null);
-    }
-
-    @Override
-    public Map<ObjectIdentity, Acl> readAclsById(List<ObjectIdentity> oids, List<Sid> sids) throws NotFoundException {
-        Map<ObjectIdentity, Acl> aclMaps = new HashMap<ObjectIdentity, Acl>();
-        Table htable = null;
-        Result result = null;
-        try {
-            htable = aclHBaseStorage.getTable(aclTableName);
-
-            for (ObjectIdentity oid : oids) {
-                result = htable.get(new Get(Bytes.toBytes(String.valueOf(oid.getIdentifier()))));
-
-                if (null != result && !result.isEmpty()) {
-                    SidInfo owner = sidSerializer.deserialize(result.getValue(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN)));
-                    Sid ownerSid = (null == owner) ? null : (owner.isPrincipal() ? new PrincipalSid(owner.getSid()) : new GrantedAuthoritySid(owner.getSid()));
-                    boolean entriesInheriting = Bytes.toBoolean(result.getValue(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN)));
-
-                    Acl parentAcl = null;
-                    DomainObjectInfo parentInfo = domainObjSerializer.deserialize(result.getValue(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN)));
-                    if (null != parentInfo) {
-                        ObjectIdentity parentObj = new ObjectIdentityImpl(parentInfo.getType(), parentInfo.getId());
-                        parentAcl = readAclById(parentObj, null);
-                    }
-
-                    AclImpl acl = new AclImpl(oid, oid.getIdentifier(), aclAuthorizationStrategy, permissionGrantingStrategy, parentAcl, null, entriesInheriting, ownerSid);
-                    genAces(sids, result, acl);
-
-                    aclMaps.put(oid, acl);
-                } else {
-                    throw new NotFoundException("Unable to find ACL information for object identity '" + oid + "'");
-                }
-            }
-        } catch (IOException e) {
-            throw new RuntimeException(e.getMessage(), e);
-        } finally {
-            IOUtils.closeQuietly(htable);
-        }
-
-        return aclMaps;
-    }
-
-    @Override
-    public MutableAcl createAcl(ObjectIdentity objectIdentity) throws AlreadyExistsException {
-        Acl acl = null;
-
-        try {
-            acl = readAclById(objectIdentity);
-        } catch (NotFoundException e) {
-            //do nothing?
-        }
-        if (null != acl) {
-            throw new AlreadyExistsException("ACL of " + objectIdentity + " exists!");
-        }
-
-        Authentication auth = SecurityContextHolder.getContext().getAuthentication();
-        PrincipalSid sid = new PrincipalSid(auth);
-
-        Table htable = null;
-        try {
-            htable = aclHBaseStorage.getTable(aclTableName);
-
-            Put put = new Put(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
-            put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
-            put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
-            put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
-
-            htable.put(put);
-
-            logger.debug("ACL of " + objectIdentity + " created successfully.");
-        } catch (IOException e) {
-            throw new RuntimeException(e.getMessage(), e);
-        } finally {
-            IOUtils.closeQuietly(htable);
-        }
-
-        return (MutableAcl) readAclById(objectIdentity);
-    }
-
-    @Override
-    public void deleteAcl(ObjectIdentity objectIdentity, boolean deleteChildren) throws ChildrenExistException {
-        Table htable = null;
-        try {
-            htable = aclHBaseStorage.getTable(aclTableName);
-
-            Delete delete = new Delete(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
-
-            List<ObjectIdentity> children = findChildren(objectIdentity);
-            if (!deleteChildren && children.size() > 0) {
-                throw new ChildrenExistException("Children exists for " + objectIdentity);
-            }
-
-            for (ObjectIdentity oid : children) {
-                deleteAcl(oid, deleteChildren);
-            }
-
-            htable.delete(delete);
-
-            logger.debug("ACL of " + objectIdentity + " deleted successfully.");
-        } catch (IOException e) {
-            throw new RuntimeException(e.getMessage(), e);
-        } finally {
-            IOUtils.closeQuietly(htable);
-        }
-    }
-
-    @Override
-    public MutableAcl updateAcl(MutableAcl acl) throws NotFoundException {
-        try {
-            readAclById(acl.getObjectIdentity());
-        } catch (NotFoundException e) {
-            throw e;
-        }
-
-        Table htable = null;
-        try {
-            htable = aclHBaseStorage.getTable(aclTableName);
-
-            Delete delete = new Delete(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
-            delete.deleteFamily(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY));
-            htable.delete(delete);
-
-            Put put = new Put(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
-
-            if (null != acl.getParentAcl()) {
-                put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
-            }
-
-            for (AccessControlEntry ace : acl.getEntries()) {
-                AceInfo aceInfo = new AceInfo(ace);
-                put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
-            }
-
-            if (!put.isEmpty()) {
-                htable.put(put);
-
-                logger.debug("ACL of " + acl.getObjectIdentity() + " updated successfully.");
-            }
-        } catch (IOException e) {
-            throw new RuntimeException(e.getMessage(), e);
-        } finally {
-            IOUtils.closeQuietly(htable);
-        }
-
-        return (MutableAcl) readAclById(acl.getObjectIdentity());
-    }
-
-    private void genAces(List<Sid> sids, Result result, AclImpl acl) throws JsonParseException, JsonMappingException, IOException {
-        List<AceInfo> aceInfos = new ArrayList<AceInfo>();
-        if (null != sids) {
-            // Just return aces in sids
-            for (Sid sid : sids) {
-                String sidName = null;
-                if (sid instanceof PrincipalSid) {
-                    sidName = ((PrincipalSid) sid).getPrincipal();
-                } else if (sid instanceof GrantedAuthoritySid) {
-                    sidName = ((GrantedAuthoritySid) sid).getGrantedAuthority();
-                }
-
-                AceInfo aceInfo = aceSerializer.deserialize(result.getValue(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY), Bytes.toBytes(sidName)));
-                if (null != aceInfo) {
-                    aceInfos.add(aceInfo);
-                }
-            }
-        } else {
-            NavigableMap<byte[], byte[]> familyMap = result.getFamilyMap(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY));
-            for (byte[] qualifier : familyMap.keySet()) {
-                AceInfo aceInfo = aceSerializer.deserialize(familyMap.get(qualifier));
-
-                if (null != aceInfo) {
-                    aceInfos.add(aceInfo);
-                }
-            }
-        }
-
-        List<AccessControlEntry> newAces = new ArrayList<AccessControlEntry>();
-        for (int i = 0; i < aceInfos.size(); i++) {
-            AceInfo aceInfo = aceInfos.get(i);
-
-            if (null != aceInfo) {
-                Sid sid = aceInfo.getSidInfo().isPrincipal() ? new PrincipalSid(aceInfo.getSidInfo().getSid()) : new GrantedAuthoritySid(aceInfo.getSidInfo().getSid());
-                AccessControlEntry ace = new AccessControlEntryImpl(Long.valueOf(i), acl, sid, aclPermissionFactory.buildFromMask(aceInfo.getPermissionMask()), true, false, false);
-                newAces.add(ace);
-            }
-        }
-
-        this.setAces(acl, newAces);
-    }
-
-    private void setAces(AclImpl acl, List<AccessControlEntry> aces) {
-        try {
-            fieldAces.set(acl, aces);
-        } catch (IllegalAccessException e) {
-            throw new IllegalStateException("Could not set AclImpl entries", e);
-        }
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/afaa95a0/server-base/src/main/java/org/apache/kylin/rest/service/LegacyUserService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/LegacyUserService.java b/server-base/src/main/java/org/apache/kylin/rest/service/LegacyUserService.java
deleted file mode 100644
index b8f3700..0000000
--- a/server-base/src/main/java/org/apache/kylin/rest/service/LegacyUserService.java
+++ /dev/null
@@ -1,237 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.rest.service;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-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.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;
-import org.apache.kylin.rest.util.Serializer;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.security.core.GrantedAuthority;
-import org.springframework.security.core.userdetails.User;
-import org.springframework.security.core.userdetails.UserDetails;
-import org.springframework.security.core.userdetails.UsernameNotFoundException;
-import org.springframework.security.provisioning.UserDetailsManager;
-
-import com.fasterxml.jackson.core.JsonParseException;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.JsonMappingException;
-
-/**
- */
-//@Component("userService")
-@Deprecated
-public class LegacyUserService implements UserDetailsManager {
-
-    public static final String PWD_PREFIX = "PWD:";
-
-    private Serializer<UserGrantedAuthority[]> ugaSerializer = new Serializer<UserGrantedAuthority[]>(UserGrantedAuthority[].class);
-
-    private String userTableName = null;
-
-    @Autowired
-    protected AclHBaseStorage aclHBaseStorage;
-
-    @PostConstruct
-    public void init() throws IOException {
-        userTableName = aclHBaseStorage.prepareHBaseTable(LegacyUserService.class);
-    }
-
-    @Override
-    public UserDetails loadUserByUsername(String username) throws UsernameNotFoundException {
-        Table htable = null;
-        try {
-            htable = aclHBaseStorage.getTable(userTableName);
-
-            Get get = new Get(Bytes.toBytes(username));
-            get.addFamily(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY));
-            Result result = htable.get(get);
-
-            User user = hbaseRowToUser(result);
-            if (user == null)
-                throw new UsernameNotFoundException("User " + username + " not found.");
-
-            return user;
-        } catch (IOException e) {
-            throw new RuntimeException(e.getMessage(), e);
-        } finally {
-            IOUtils.closeQuietly(htable);
-        }
-    }
-
-    private User hbaseRowToUser(Result result) throws JsonParseException, JsonMappingException, IOException {
-        if (null == result || result.isEmpty())
-            return null;
-
-        String username = Bytes.toString(result.getRow());
-
-        byte[] valueBytes = result.getValue(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN));
-        UserGrantedAuthority[] deserialized = ugaSerializer.deserialize(valueBytes);
-
-        String password = "";
-        List<UserGrantedAuthority> authorities = Collections.emptyList();
-
-        // password is stored at [0] of authorities for backward compatibility
-        if (deserialized != null) {
-            if (deserialized.length > 0 && deserialized[0].getAuthority().startsWith(PWD_PREFIX)) {
-                password = deserialized[0].getAuthority().substring(PWD_PREFIX.length());
-                authorities = Arrays.asList(deserialized).subList(1, deserialized.length);
-            } else {
-                authorities = Arrays.asList(deserialized);
-            }
-        }
-
-        return new User(username, password, authorities);
-    }
-
-    private Pair<byte[], byte[]> userToHBaseRow(UserDetails user) throws JsonProcessingException {
-        byte[] key = Bytes.toBytes(user.getUsername());
-
-        Collection<? extends GrantedAuthority> authorities = user.getAuthorities();
-        if (authorities == null)
-            authorities = Collections.emptyList();
-
-        UserGrantedAuthority[] serializing = new UserGrantedAuthority[authorities.size() + 1];
-
-        // password is stored as the [0] authority
-        serializing[0] = new UserGrantedAuthority(PWD_PREFIX + user.getPassword());
-        int i = 1;
-        for (GrantedAuthority a : authorities) {
-            serializing[i++] = new UserGrantedAuthority(a.getAuthority());
-        }
-
-        byte[] value = ugaSerializer.serialize(serializing);
-        return Pair.newPair(key, value);
-    }
-
-    @Override
-    public void createUser(UserDetails user) {
-        updateUser(user);
-    }
-
-    @Override
-    public void updateUser(UserDetails user) {
-        Table htable = null;
-        try {
-            htable = aclHBaseStorage.getTable(userTableName);
-
-            Pair<byte[], byte[]> pair = userToHBaseRow(user);
-            Put put = new Put(pair.getKey());
-
-            put.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
-
-            htable.put(put);
-        } catch (IOException e) {
-            throw new RuntimeException(e.getMessage(), e);
-        } finally {
-            IOUtils.closeQuietly(htable);
-        }
-    }
-
-    @Override
-    public void deleteUser(String username) {
-        Table htable = null;
-        try {
-            htable = aclHBaseStorage.getTable(userTableName);
-
-            Delete delete = new Delete(Bytes.toBytes(username));
-
-            htable.delete(delete);
-        } catch (IOException e) {
-            throw new RuntimeException(e.getMessage(), e);
-        } finally {
-            IOUtils.closeQuietly(htable);
-        }
-    }
-
-    @Override
-    public void changePassword(String oldPassword, String newPassword) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean userExists(String username) {
-        Table htable = null;
-        try {
-            htable = aclHBaseStorage.getTable(userTableName);
-
-            Result result = htable.get(new Get(Bytes.toBytes(username)));
-
-            return null != result && !result.isEmpty();
-        } catch (IOException e) {
-            throw new RuntimeException(e.getMessage(), e);
-        } finally {
-            IOUtils.closeQuietly(htable);
-        }
-    }
-
-    public List<String> listUserAuthorities() {
-        List<String> all = new ArrayList<String>();
-        for (UserDetails user : listUsers()) {
-            for (GrantedAuthority auth : user.getAuthorities()) {
-                if (!all.contains(auth.getAuthority())) {
-                    all.add(auth.getAuthority());
-                }
-            }
-        }
-        return all;
-    }
-
-    public List<UserDetails> listUsers() {
-        Scan s = new Scan();
-        s.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN));
-
-        List<UserDetails> all = new ArrayList<UserDetails>();
-        Table htable = null;
-        ResultScanner scanner = null;
-        try {
-            htable = aclHBaseStorage.getTable(userTableName);
-            scanner = htable.getScanner(s);
-
-            for (Result result = scanner.next(); result != null; result = scanner.next()) {
-                User user = hbaseRowToUser(result);
-                all.add(user);
-            }
-        } catch (IOException e) {
-            throw new RuntimeException("Failed to scan users", e);
-        } finally {
-            IOUtils.closeQuietly(scanner);
-            IOUtils.closeQuietly(htable);
-        }
-        return all;
-    }
-
-}