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 <liy...@apache.org> Authored: Tue May 30 00:40:50 2017 +0800 Committer: nichunen <zjsy...@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; - } - -}