Return-Path: X-Original-To: apmail-hive-commits-archive@www.apache.org Delivered-To: apmail-hive-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E2B1E18477 for ; Thu, 9 Jul 2015 17:24:44 +0000 (UTC) Received: (qmail 15793 invoked by uid 500); 9 Jul 2015 17:24:44 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 15706 invoked by uid 500); 9 Jul 2015 17:24:44 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 15408 invoked by uid 99); 9 Jul 2015 17:24:44 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 09 Jul 2015 17:24:44 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 798B7E6839; Thu, 9 Jul 2015 17:24:44 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: vgumashta@apache.org To: commits@hive.apache.org Date: Thu, 09 Jul 2015 17:24:45 -0000 Message-Id: <8bd28cfe9d0e42be9ece5ead8319b455@git.apache.org> In-Reply-To: <0dd895daa9d542af8e6186f057e88738@git.apache.org> References: <0dd895daa9d542af8e6186f057e88738@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [2/3] hive git commit: HIVE-10895: ObjectStore does not close Query objects in some calls, causing a potential leak in some metastore db resources (Aihua Xu reviewed by Chaoyu Tang, Sergey Shelukhin, Vaibhav Gumashta) http://git-wip-us.apache.org/repos/asf/hive/blob/9da7b3e9/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java ---------------------------------------------------------------------- diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java index 4273c0b..bce4511 100644 --- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -213,6 +213,29 @@ public class ObjectStore implements RawStore, Configurable { private Pattern partitionValidationPattern; + /** + * A class to pass the Query object to the caller to let the caller release + * resources by calling QueryWrapper.query.closeAll() after consuming all the query results. + */ + public static class QueryWrapper { + public Query query; + + /** + * Explicitly closes the query object to release the resources + */ + public void close() { + if (query != null) { + query.closeAll(); + query = null; + } + } + + @Override + protected void finalize() { + this.close(); + } + } + public ObjectStore() { } @@ -533,10 +556,11 @@ public class ObjectStore implements RawStore, Configurable { private MDatabase getMDatabase(String name) throws NoSuchObjectException { MDatabase mdb = null; boolean commited = false; + Query query = null; try { openTransaction(); name = HiveStringUtils.normalizeIdentifier(name); - Query query = pm.newQuery(MDatabase.class, "name == dbname"); + query = pm.newQuery(MDatabase.class, "name == dbname"); query.declareParameters("java.lang.String dbname"); query.setUnique(true); mdb = (MDatabase) query.execute(name); @@ -546,6 +570,9 @@ public class ObjectStore implements RawStore, Configurable { if (!commited) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } if (mdb == null) { throw new NoSuchObjectException("There is no database named " + name); @@ -648,6 +675,7 @@ public class ObjectStore implements RawStore, Configurable { boolean success = false; LOG.info("Dropping database " + dbname + " along with all tables"); dbname = HiveStringUtils.normalizeIdentifier(dbname); + QueryWrapper queryWrapper = new QueryWrapper(); try { openTransaction(); @@ -655,7 +683,7 @@ public class ObjectStore implements RawStore, Configurable { MDatabase db = getMDatabase(dbname); pm.retrieve(db); if (db != null) { - List dbGrants = this.listDatabaseGrants(dbname); + List dbGrants = this.listDatabaseGrants(dbname, queryWrapper); if (dbGrants != null && dbGrants.size() > 0) { pm.deletePersistentAll(dbGrants); } @@ -666,36 +694,36 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + queryWrapper.close(); } return success; } - @Override public List getDatabases(String pattern) throws MetaException { boolean commited = false; List databases = null; + Query query = null; try { openTransaction(); // Take the pattern and split it on the | to get all the composing // patterns String[] subpatterns = pattern.trim().split("\\|"); - String query = "select name from org.apache.hadoop.hive.metastore.model.MDatabase where ("; + String queryStr = "select name from org.apache.hadoop.hive.metastore.model.MDatabase where ("; boolean first = true; for (String subpattern : subpatterns) { subpattern = "(?i)" + subpattern.replaceAll("\\*", ".*"); if (!first) { - query = query + " || "; + queryStr = queryStr + " || "; } - query = query + " name.matches(\"" + subpattern + "\")"; + queryStr = queryStr + " name.matches(\"" + subpattern + "\")"; first = false; } - query = query + ")"; - - Query q = pm.newQuery(query); - q.setResult("name"); - q.setOrdering("name ascending"); - Collection names = (Collection) q.execute(); + queryStr = queryStr + ")"; + query = pm.newQuery(queryStr); + query.setResult("name"); + query.setOrdering("name ascending"); + Collection names = (Collection) query.execute(); databases = new ArrayList(); for (Iterator i = names.iterator(); i.hasNext();) { databases.add((String) i.next()); @@ -705,6 +733,9 @@ public class ObjectStore implements RawStore, Configurable { if (!commited) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return databases; } @@ -763,9 +794,10 @@ public class ObjectStore implements RawStore, Configurable { public Type getType(String typeName) { Type type = null; boolean commited = false; + Query query = null; try { openTransaction(); - Query query = pm.newQuery(MType.class, "name == typeName"); + query = pm.newQuery(MType.class, "name == typeName"); query.declareParameters("java.lang.String typeName"); query.setUnique(true); MType mtype = (MType) query.execute(typeName.trim()); @@ -778,6 +810,9 @@ public class ObjectStore implements RawStore, Configurable { if (!commited) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return type; } @@ -785,9 +820,10 @@ public class ObjectStore implements RawStore, Configurable { @Override public boolean dropType(String typeName) { boolean success = false; + Query query = null; try { openTransaction(); - Query query = pm.newQuery(MType.class, "name == typeName"); + query = pm.newQuery(MType.class, "name == typeName"); query.declareParameters("java.lang.String typeName"); query.setUnique(true); MType type = (MType) query.execute(typeName.trim()); @@ -803,6 +839,9 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return success; } @@ -938,9 +977,9 @@ public class ObjectStore implements RawStore, Configurable { } @Override - public List getTables(String dbName, String pattern) - throws MetaException { + public List getTables(String dbName, String pattern) throws MetaException { boolean commited = false; + Query query = null; List tbls = null; try { openTransaction(); @@ -948,25 +987,24 @@ public class ObjectStore implements RawStore, Configurable { // Take the pattern and split it on the | to get all the composing // patterns String[] subpatterns = pattern.trim().split("\\|"); - String query = - "select tableName from org.apache.hadoop.hive.metastore.model.MTable " - + "where database.name == dbName && ("; + String queryStr = + "select tableName from org.apache.hadoop.hive.metastore.model.MTable " + + "where database.name == dbName && ("; boolean first = true; for (String subpattern : subpatterns) { subpattern = "(?i)" + subpattern.replaceAll("\\*", ".*"); if (!first) { - query = query + " || "; + queryStr = queryStr + " || "; } - query = query + " tableName.matches(\"" + subpattern + "\")"; + queryStr = queryStr + " tableName.matches(\"" + subpattern + "\")"; first = false; } - query = query + ")"; - - Query q = pm.newQuery(query); - q.declareParameters("java.lang.String dbName"); - q.setResult("tableName"); - q.setOrdering("tableName ascending"); - Collection names = (Collection) q.execute(dbName); + queryStr = queryStr + ")"; + query = pm.newQuery(queryStr); + query.declareParameters("java.lang.String dbName"); + query.setResult("tableName"); + query.setOrdering("tableName ascending"); + Collection names = (Collection) query.execute(dbName); tbls = new ArrayList(); for (Iterator i = names.iterator(); i.hasNext();) { tbls.add((String) i.next()); @@ -976,6 +1014,9 @@ public class ObjectStore implements RawStore, Configurable { if (!commited) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return tbls; } @@ -988,11 +1029,12 @@ public class ObjectStore implements RawStore, Configurable { private MTable getMTable(String db, String table) { MTable mtbl = null; boolean commited = false; + Query query = null; try { openTransaction(); db = HiveStringUtils.normalizeIdentifier(db); table = HiveStringUtils.normalizeIdentifier(table); - Query query = pm.newQuery(MTable.class, "tableName == table && database.name == db"); + query = pm.newQuery(MTable.class, "tableName == table && database.name == db"); query.declareParameters("java.lang.String table, java.lang.String db"); query.setUnique(true); mtbl = (MTable) query.execute(table, db); @@ -1002,20 +1044,24 @@ public class ObjectStore implements RawStore, Configurable { if (!commited) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mtbl; } @Override - public List getTableObjectsByName(String db, List tbl_names) - throws MetaException, UnknownDBException { + public List
getTableObjectsByName(String db, List tbl_names) throws MetaException, + UnknownDBException { List
tables = new ArrayList
(); boolean committed = false; + Query dbExistsQuery = null; + Query query = null; try { openTransaction(); - db = HiveStringUtils.normalizeIdentifier(db); - Query dbExistsQuery = pm.newQuery(MDatabase.class, "name == db"); + dbExistsQuery = pm.newQuery(MDatabase.class, "name == db"); dbExistsQuery.declareParameters("java.lang.String db"); dbExistsQuery.setUnique(true); dbExistsQuery.setResult("name"); @@ -1028,7 +1074,7 @@ public class ObjectStore implements RawStore, Configurable { for (String t : tbl_names) { lowered_tbl_names.add(HiveStringUtils.normalizeIdentifier(t)); } - Query query = pm.newQuery(MTable.class); + query = pm.newQuery(MTable.class); query.setFilter("database.name == db && tbl_names.contains(tableName)"); query.declareParameters("java.lang.String db, java.util.Collection tbl_names"); Collection mtables = (Collection) query.execute(db, lowered_tbl_names); @@ -1040,6 +1086,12 @@ public class ObjectStore implements RawStore, Configurable { if (!committed) { rollbackTransaction(); } + if (dbExistsQuery != null) { + dbExistsQuery.closeAll(); + } + if (query != null) { + query.closeAll(); + } } return tables; } @@ -1190,9 +1242,9 @@ public class ObjectStore implements RawStore, Configurable { // MSD and SD should be same objects. Not sure how to make then same right now // MSerdeInfo *& SerdeInfo should be same as well - private StorageDescriptor convertToStorageDescriptor(MStorageDescriptor msd, - boolean noFS) - throws MetaException { + private StorageDescriptor convertToStorageDescriptor( + MStorageDescriptor msd, + boolean noFS) throws MetaException { if (msd == null) { return null; } @@ -1278,8 +1330,6 @@ public class ObjectStore implements RawStore, Configurable { return map; } - - /** * Converts a storage descriptor to a db-backed storage descriptor. Creates a * new db-backed column descriptor object for this SD. @@ -1386,7 +1436,6 @@ public class ObjectStore implements RawStore, Configurable { return !doesExist; } - @Override public boolean addPartitions(String dbName, String tblName, PartitionSpecProxy partitionSpec, boolean ifNotExists) @@ -1513,10 +1562,11 @@ public class ObjectStore implements RawStore, Configurable { return part; } - private MPartition getMPartition(String dbName, String tableName, - List part_vals) throws MetaException { + private MPartition getMPartition(String dbName, String tableName, List part_vals) + throws MetaException { MPartition mpart = null; boolean commited = false; + Query query = null; try { openTransaction(); dbName = HiveStringUtils.normalizeIdentifier(dbName); @@ -1528,10 +1578,11 @@ public class ObjectStore implements RawStore, Configurable { } // Change the query to use part_vals instead of the name which is // redundant TODO: callers of this often get part_vals out of name for no reason... - String name = Warehouse.makePartName(convertToFieldSchemas(mtbl - .getPartitionKeys()), part_vals); - Query query = pm.newQuery(MPartition.class, - "table.tableName == t1 && table.database.name == t2 && partitionName == t3"); + String name = + Warehouse.makePartName(convertToFieldSchemas(mtbl.getPartitionKeys()), part_vals); + query = + pm.newQuery(MPartition.class, + "table.tableName == t1 && table.database.name == t2 && partitionName == t3"); query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3"); query.setUnique(true); mpart = (MPartition) query.execute(tableName, dbName, name); @@ -1541,6 +1592,9 @@ public class ObjectStore implements RawStore, Configurable { if (!commited) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mpart; } @@ -1732,8 +1786,13 @@ public class ObjectStore implements RawStore, Configurable { } @Override protected List getJdoResult( - GetHelper> ctx) throws MetaException, NoSuchObjectException { - return convertToParts(listMPartitions(dbName, tblName, maxParts)); + GetHelper> ctx) throws MetaException { + QueryWrapper queryWrapper = new QueryWrapper(); + try { + return convertToParts(listMPartitions(dbName, tblName, maxParts, queryWrapper)); + } finally { + queryWrapper.close(); + } } }.run(false); } @@ -1741,11 +1800,13 @@ public class ObjectStore implements RawStore, Configurable { @Override public List getPartitionsWithAuth(String dbName, String tblName, short max, String userName, List groupNames) - throws MetaException, NoSuchObjectException, InvalidObjectException { + throws MetaException, InvalidObjectException { boolean success = false; + QueryWrapper queryWrapper = new QueryWrapper(); + try { openTransaction(); - List mparts = listMPartitions(dbName, tblName, max); + List mparts = listMPartitions(dbName, tblName, max, queryWrapper); List parts = new ArrayList(mparts.size()); if (mparts != null && mparts.size()>0) { for (MPartition mpart : mparts) { @@ -1768,6 +1829,7 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + queryWrapper.close(); } } @@ -1804,7 +1866,6 @@ public class ObjectStore implements RawStore, Configurable { } } - private List convertToParts(List mparts) throws MetaException { return convertToParts(mparts, null); } @@ -1857,20 +1918,22 @@ public class ObjectStore implements RawStore, Configurable { List pns = new ArrayList(); dbName = HiveStringUtils.normalizeIdentifier(dbName); tableName = HiveStringUtils.normalizeIdentifier(tableName); - Query q = pm.newQuery( - "select partitionName from org.apache.hadoop.hive.metastore.model.MPartition " - + "where table.database.name == t1 && table.tableName == t2 " - + "order by partitionName asc"); - q.declareParameters("java.lang.String t1, java.lang.String t2"); - q.setResult("partitionName"); - - if(max > 0) { - q.setRange(0, max); - } - Collection names = (Collection) q.execute(dbName, tableName); + Query query = + pm.newQuery("select partitionName from org.apache.hadoop.hive.metastore.model.MPartition " + + "where table.database.name == t1 && table.tableName == t2 " + + "order by partitionName asc"); + query.declareParameters("java.lang.String t1, java.lang.String t2"); + query.setResult("partitionName"); + if (max > 0) { + query.setRange(0, max); + } + Collection names = (Collection) query.execute(dbName, tableName); for (Iterator i = names.iterator(); i.hasNext();) { pns.add((String) i.next()); } + if (query != null) { + query.closeAll(); + } return pns; } @@ -1890,51 +1953,46 @@ public class ObjectStore implements RawStore, Configurable { * has types of String, and if resultsCol is null, the types are MPartition. */ private Collection getPartitionPsQueryResults(String dbName, String tableName, - List part_vals, short max_parts, String resultsCol) + List part_vals, short max_parts, String resultsCol, QueryWrapper queryWrapper) throws MetaException, NoSuchObjectException { dbName = HiveStringUtils.normalizeIdentifier(dbName); tableName = HiveStringUtils.normalizeIdentifier(tableName); Table table = getTable(dbName, tableName); - if (table == null) { throw new NoSuchObjectException(dbName + "." + tableName + " table not found"); } - List partCols = table.getPartitionKeys(); int numPartKeys = partCols.size(); if (part_vals.size() > numPartKeys) { throw new MetaException("Incorrect number of partition values"); } - partCols = partCols.subList(0, part_vals.size()); - //Construct a pattern of the form: partKey=partVal/partKey2=partVal2/... + // Construct a pattern of the form: partKey=partVal/partKey2=partVal2/... // where partVal is either the escaped partition value given as input, // or a regex of the form ".*" - //This works because the "=" and "/" separating key names and partition key/values + // This works because the "=" and "/" separating key names and partition key/values // are not escaped. String partNameMatcher = Warehouse.makePartName(partCols, part_vals, ".*"); - //add ".*" to the regex to match anything else afterwards the partial spec. + // add ".*" to the regex to match anything else afterwards the partial spec. if (part_vals.size() < numPartKeys) { partNameMatcher += ".*"; } - - Query q = pm.newQuery(MPartition.class); + Query query = queryWrapper.query = pm.newQuery(MPartition.class); StringBuilder queryFilter = new StringBuilder("table.database.name == dbName"); queryFilter.append(" && table.tableName == tableName"); queryFilter.append(" && partitionName.matches(partialRegex)"); - q.setFilter(queryFilter.toString()); - q.declareParameters("java.lang.String dbName, " + - "java.lang.String tableName, java.lang.String partialRegex"); - - if( max_parts >= 0 ) { - //User specified a row limit, set it on the Query - q.setRange(0, max_parts); + query.setFilter(queryFilter.toString()); + query.declareParameters("java.lang.String dbName, " + + "java.lang.String tableName, java.lang.String partialRegex"); + if (max_parts >= 0) { + // User specified a row limit, set it on the Query + query.setRange(0, max_parts); } if (resultsCol != null && !resultsCol.isEmpty()) { - q.setResult(resultsCol); + query.setResult(resultsCol); } - return (Collection) q.execute(dbName, tableName, partNameMatcher); + return (Collection) query.execute(dbName, tableName, partNameMatcher); } @Override @@ -1943,11 +2001,13 @@ public class ObjectStore implements RawStore, Configurable { throws MetaException, InvalidObjectException, NoSuchObjectException { List partitions = new ArrayList(); boolean success = false; + QueryWrapper queryWrapper = new QueryWrapper(); + try { openTransaction(); LOG.debug("executing listPartitionNamesPsWithAuth"); Collection parts = getPartitionPsQueryResults(db_name, tbl_name, - part_vals, max_parts, null); + part_vals, max_parts, null, queryWrapper); MTable mtbl = getMTable(db_name, tbl_name); for (Object o : parts) { Partition part = convertToPart((MPartition) o); @@ -1967,6 +2027,7 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + queryWrapper.close(); } return partitions; } @@ -1976,11 +2037,13 @@ public class ObjectStore implements RawStore, Configurable { List part_vals, short max_parts) throws MetaException, NoSuchObjectException { List partitionNames = new ArrayList(); boolean success = false; + QueryWrapper queryWrapper = new QueryWrapper(); + try { openTransaction(); LOG.debug("Executing listPartitionNamesPs"); Collection names = getPartitionPsQueryResults(dbName, tableName, - part_vals, max_parts, "partitionName"); + part_vals, max_parts, "partitionName", queryWrapper); for (Object o : names) { partitionNames.add((String) o); } @@ -1989,14 +2052,13 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + queryWrapper.close(); } return partitionNames; } // TODO:pc implement max - private List listMPartitions(String dbName, String tableName, - int max) { - + private List listMPartitions(String dbName, String tableName, int max, QueryWrapper queryWrapper) { boolean success = false; List mparts = null; try { @@ -2004,11 +2066,10 @@ public class ObjectStore implements RawStore, Configurable { LOG.debug("Executing listMPartitions"); dbName = HiveStringUtils.normalizeIdentifier(dbName); tableName = HiveStringUtils.normalizeIdentifier(tableName); - Query query = pm.newQuery(MPartition.class, - "table.tableName == t1 && table.database.name == t2"); + Query query = queryWrapper.query = pm.newQuery(MPartition.class, "table.tableName == t1 && table.database.name == t2"); query.declareParameters("java.lang.String t1, java.lang.String t2"); query.setOrdering("partitionName ascending"); - if(max > 0) { + if (max > 0) { query.setRange(0, max); } mparts = (List) query.execute(tableName, dbName); @@ -2198,8 +2259,8 @@ public class ObjectStore implements RawStore, Configurable { private List getPartitionsViaOrmFilter(Table table, ExpressionTree tree, short maxParts, boolean isValidatedFilter) throws MetaException { Map params = new HashMap(); - String jdoFilter = makeQueryFilterString( - table.getDbName(), table, tree, params, isValidatedFilter); + String jdoFilter = + makeQueryFilterString(table.getDbName(), table, tree, params, isValidatedFilter); if (jdoFilter == null) { assert !isValidatedFilter; return null; @@ -2209,14 +2270,11 @@ public class ObjectStore implements RawStore, Configurable { // User specified a row limit, set it on the Query query.setRange(0, maxParts); } - String parameterDeclaration = makeParameterDeclarationStringObj(params); query.declareParameters(parameterDeclaration); query.setOrdering("partitionName ascending"); - @SuppressWarnings("unchecked") List mparts = (List) query.executeWithMap(params); - LOG.debug("Done executing query for getPartitionsViaOrmFilter"); pm.retrieveAll(mparts); // TODO: why is this inconsistent with what we get by names? LOG.debug("Done retrieving all objects for getPartitionsViaOrmFilter"); @@ -2225,10 +2283,6 @@ public class ObjectStore implements RawStore, Configurable { return results; } - private static class Out { - public T val; - } - /** * Gets partition names from the table via ORM (JDOQL) name filter. * @param dbName Database name. @@ -2241,16 +2295,19 @@ public class ObjectStore implements RawStore, Configurable { if (partNames.isEmpty()) { return new ArrayList(); } - Out query = new Out(); - List mparts = null; - try { - mparts = getMPartitionsViaOrmFilter(dbName, tblName, partNames, query); - return convertToParts(dbName, tblName, mparts); - } finally { - if (query.val != null) { - query.val.closeAll(); - } + ObjectPair> queryWithParams = + getPartQueryWithParams(dbName, tblName, partNames); + Query query = queryWithParams.getFirst(); + query.setResultClass(MPartition.class); + query.setClass(MPartition.class); + query.setOrdering("partitionName ascending"); + @SuppressWarnings("unchecked") + List mparts = (List)query.executeWithMap(queryWithParams.getSecond()); + List partitions = convertToParts(dbName, tblName, mparts); + if (query != null) { + query.closeAll(); } + return partitions; } private void dropPartitionsNoTxn(String dbName, String tblName, List partNames) { @@ -2286,27 +2343,15 @@ public class ObjectStore implements RawStore, Configurable { sd.setCD(null); } } + if (query != null) { + query.closeAll(); + } return candidateCds; } - private List getMPartitionsViaOrmFilter(String dbName, - String tblName, List partNames, Out out) { - ObjectPair> queryWithParams = - getPartQueryWithParams(dbName, tblName, partNames); - Query query = out.val = queryWithParams.getFirst(); - query.setResultClass(MPartition.class); - query.setClass(MPartition.class); - query.setOrdering("partitionName ascending"); - - @SuppressWarnings("unchecked") - List result = (List)query.executeWithMap(queryWithParams.getSecond()); - return result; - } - - private ObjectPair> getPartQueryWithParams( - String dbName, String tblName, List partNames) { - StringBuilder sb = new StringBuilder( - "table.tableName == t1 && table.database.name == t2 && ("); + private ObjectPair> getPartQueryWithParams(String dbName, + String tblName, List partNames) { + StringBuilder sb = new StringBuilder("table.tableName == t1 && table.database.name == t2 && ("); int n = 0; Map params = new HashMap(); for (Iterator itr = partNames.iterator(); itr.hasNext();) { @@ -2319,16 +2364,13 @@ public class ObjectStore implements RawStore, Configurable { } sb.setLength(sb.length() - 4); // remove the last " || " sb.append(')'); - Query query = pm.newQuery(); query.setFilter(sb.toString()); - LOG.debug(" JDOQL filter is " + sb.toString()); params.put("t1", HiveStringUtils.normalizeIdentifier(tblName)); params.put("t2", HiveStringUtils.normalizeIdentifier(dbName)); - query.declareParameters(makeParameterDeclarationString(params)); - return new ObjectPair>(query, params); + return new ObjectPair>(query, params); } @Override @@ -2650,6 +2692,7 @@ public class ObjectStore implements RawStore, Configurable { public List listTableNamesByFilter(String dbName, String filter, short maxTables) throws MetaException { boolean success = false; + Query query = null; List tableNames = new ArrayList(); try { openTransaction(); @@ -2657,7 +2700,7 @@ public class ObjectStore implements RawStore, Configurable { dbName = HiveStringUtils.normalizeIdentifier(dbName); Map params = new HashMap(); String queryFilterString = makeQueryFilterString(dbName, null, filter, params); - Query query = pm.newQuery(MTable.class); + query = pm.newQuery(MTable.class); query.declareImports("import java.lang.String"); query.setResult("tableName"); query.setResultClass(java.lang.String.class); @@ -2666,14 +2709,14 @@ public class ObjectStore implements RawStore, Configurable { } LOG.debug("filter specified is " + filter + "," + " JDOQL filter is " + queryFilterString); for (Entry entry : params.entrySet()) { - LOG.debug("key: " + entry.getKey() + " value: " + entry.getValue() + - " class: " + entry.getValue().getClass().getName()); + LOG.debug("key: " + entry.getKey() + " value: " + entry.getValue() + " class: " + + entry.getValue().getClass().getName()); } String parameterDeclaration = makeParameterDeclarationStringObj(params); query.declareParameters(parameterDeclaration); query.setFilter(queryFilterString); - Collection names = (Collection) query.executeWithMap(params); - //have to emulate "distinct", otherwise tables with the same name may be returned + Collection names = (Collection)query.executeWithMap(params); + // have to emulate "distinct", otherwise tables with the same name may be returned Set tableNamesSet = new HashSet(); for (Iterator i = names.iterator(); i.hasNext();) { tableNamesSet.add((String) i.next()); @@ -2682,58 +2725,54 @@ public class ObjectStore implements RawStore, Configurable { LOG.debug("Done executing query for listTableNamesByFilter"); success = commitTransaction(); LOG.debug("Done retrieving all objects for listTableNamesByFilter"); - } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return tableNames; } @Override - public List listPartitionNamesByFilter(String dbName, String tableName, - String filter, short maxParts) throws MetaException { + public List listPartitionNamesByFilter(String dbName, String tableName, String filter, + short maxParts) throws MetaException { boolean success = false; + Query query = null; List partNames = new ArrayList(); try { openTransaction(); LOG.debug("Executing listMPartitionNamesByFilter"); dbName = HiveStringUtils.normalizeIdentifier(dbName); tableName = HiveStringUtils.normalizeIdentifier(tableName); - MTable mtable = getMTable(dbName, tableName); - if( mtable == null ) { + if (mtable == null) { // To be consistent with the behavior of listPartitionNames, if the // table or db does not exist, we return an empty list return partNames; } Map params = new HashMap(); String queryFilterString = makeQueryFilterString(dbName, mtable, filter, params); - Query query = pm.newQuery( - "select partitionName from org.apache.hadoop.hive.metastore.model.MPartition " - + "where " + queryFilterString); - - if( maxParts >= 0 ) { - //User specified a row limit, set it on the Query + query = + pm.newQuery("select partitionName from org.apache.hadoop.hive.metastore.model.MPartition " + + "where " + queryFilterString); + if (maxParts >= 0) { + // User specified a row limit, set it on the Query query.setRange(0, maxParts); } - - LOG.debug("Filter specified is " + filter + "," + - " JDOQL filter is " + queryFilterString); + LOG.debug("Filter specified is " + filter + "," + " JDOQL filter is " + queryFilterString); LOG.debug("Parms is " + params); - String parameterDeclaration = makeParameterDeclarationStringObj(params); query.declareParameters(parameterDeclaration); query.setOrdering("partitionName ascending"); query.setResult("partitionName"); - Collection names = (Collection) query.executeWithMap(params); partNames = new ArrayList(); for (Iterator i = names.iterator(); i.hasNext();) { partNames.add((String) i.next()); } - LOG.debug("Done executing query for listMPartitionNamesByFilter"); success = commitTransaction(); LOG.debug("Done retrieving all objects for listMPartitionNamesByFilter"); @@ -2741,6 +2780,9 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return partNames; } @@ -2944,10 +2986,12 @@ public class ObjectStore implements RawStore, Configurable { } boolean success = false; + QueryWrapper queryWrapper = new QueryWrapper(); + try { openTransaction(); LOG.debug("execute removeUnusedColumnDescriptor"); - List referencedSDs = listStorageDescriptorsWithCD(oldCD, 1); + List referencedSDs = listStorageDescriptorsWithCD(oldCD, 1, queryWrapper); //if no other SD references this CD, we can throw it out. if (referencedSDs != null && referencedSDs.isEmpty()) { pm.retrieve(oldCD); @@ -2959,6 +3003,7 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + queryWrapper.close(); } } @@ -2987,21 +3032,22 @@ public class ObjectStore implements RawStore, Configurable { * @param maxSDs the maximum number of SDs to return * @return a list of storage descriptors */ - private List listStorageDescriptorsWithCD(MColumnDescriptor oldCD, - long maxSDs) { + private List listStorageDescriptorsWithCD( + MColumnDescriptor oldCD, + long maxSDs, + QueryWrapper queryWrapper) { boolean success = false; List sds = null; try { openTransaction(); LOG.debug("Executing listStorageDescriptorsWithCD"); - Query query = pm.newQuery(MStorageDescriptor.class, - "this.cd == inCD"); + Query query = queryWrapper.query = pm.newQuery(MStorageDescriptor.class, "this.cd == inCD"); query.declareParameters("MColumnDescriptor inCD"); - if(maxSDs >= 0) { - //User specified a row limit, set it on the Query + if (maxSDs >= 0) { + // User specified a row limit, set it on the Query query.setRange(0, maxSDs); } - sds = (List) query.execute(oldCD); + sds = (List)query.execute(oldCD); LOG.debug("Done executing query for listStorageDescriptorsWithCD"); pm.retrieveAll(sds); success = commitTransaction(); @@ -3078,9 +3124,11 @@ public class ObjectStore implements RawStore, Configurable { return success; } - private MIndex getMIndex(String dbName, String originalTblName, String indexName) throws MetaException { + private MIndex getMIndex(String dbName, String originalTblName, String indexName) + throws MetaException { MIndex midx = null; boolean commited = false; + Query query = null; try { openTransaction(); dbName = HiveStringUtils.normalizeIdentifier(dbName); @@ -3090,19 +3138,23 @@ public class ObjectStore implements RawStore, Configurable { commited = commitTransaction(); return null; } - - Query query = pm.newQuery(MIndex.class, - "origTable.tableName == t1 && origTable.database.name == t2 && indexName == t3"); + query = + pm.newQuery(MIndex.class, + "origTable.tableName == t1 && origTable.database.name == t2 && indexName == t3"); query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3"); query.setUnique(true); - midx = (MIndex) query.execute(originalTblName, dbName, - HiveStringUtils.normalizeIdentifier(indexName)); + midx = + (MIndex) query.execute(originalTblName, dbName, + HiveStringUtils.normalizeIdentifier(indexName)); pm.retrieve(midx); commited = commitTransaction(); } finally { if (!commited) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return midx; } @@ -3143,64 +3195,55 @@ public class ObjectStore implements RawStore, Configurable { public List getIndexes(String dbName, String origTableName, int max) throws MetaException { boolean success = false; + Query query = null; try { + LOG.debug("Executing getIndexes"); openTransaction(); - List mIndexList = listMIndexes(dbName, origTableName, max); - List indexes = new ArrayList(mIndexList.size()); - for (MIndex midx : mIndexList) { - indexes.add(this.convertToIndex(midx)); - } - success = commitTransaction(); - return indexes; - } finally { - if (!success) { - rollbackTransaction(); - } - } - } - private List listMIndexes(String dbName, String origTableName, - int max) { - boolean success = false; - List mindexes = null; - try { - openTransaction(); - LOG.debug("Executing listMIndexes"); dbName = HiveStringUtils.normalizeIdentifier(dbName); origTableName = HiveStringUtils.normalizeIdentifier(origTableName); - Query query = pm.newQuery(MIndex.class, - "origTable.tableName == t1 && origTable.database.name == t2"); + query = + pm.newQuery(MIndex.class, "origTable.tableName == t1 && origTable.database.name == t2"); query.declareParameters("java.lang.String t1, java.lang.String t2"); - mindexes = (List) query.execute(origTableName, dbName); - LOG.debug("Done executing query for listMIndexes"); - pm.retrieveAll(mindexes); + List mIndexes = (List) query.execute(origTableName, dbName); + pm.retrieveAll(mIndexes); + + List indexes = new ArrayList(mIndexes.size()); + for (MIndex mIdx : mIndexes) { + indexes.add(this.convertToIndex(mIdx)); + } success = commitTransaction(); - LOG.debug("Done retrieving all objects for listMIndexes"); + LOG.debug("Done retrieving all objects for getIndexes"); + + return indexes; } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } - return mindexes; } @Override - public List listIndexNames(String dbName, String origTableName, - short max) throws MetaException { + public List listIndexNames(String dbName, String origTableName, short max) + throws MetaException { List pns = new ArrayList(); boolean success = false; + Query query = null; try { openTransaction(); LOG.debug("Executing listIndexNames"); dbName = HiveStringUtils.normalizeIdentifier(dbName); origTableName = HiveStringUtils.normalizeIdentifier(origTableName); - Query q = pm.newQuery( - "select indexName from org.apache.hadoop.hive.metastore.model.MIndex " - + "where origTable.database.name == t1 && origTable.tableName == t2 " - + "order by indexName asc"); - q.declareParameters("java.lang.String t1, java.lang.String t2"); - q.setResult("indexName"); - Collection names = (Collection) q.execute(dbName, origTableName); + query = + pm.newQuery("select indexName from org.apache.hadoop.hive.metastore.model.MIndex " + + "where origTable.database.name == t1 && origTable.tableName == t2 " + + "order by indexName asc"); + query.declareParameters("java.lang.String t1, java.lang.String t2"); + query.setResult("indexName"); + Collection names = (Collection) query.execute(dbName, origTableName); for (Iterator i = names.iterator(); i.hasNext();) { pns.add((String) i.next()); } @@ -3209,6 +3252,9 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return pns; } @@ -3225,8 +3271,7 @@ public class ObjectStore implements RawStore, Configurable { throw new InvalidObjectException("Role " + roleName + " already exists."); } int now = (int)(System.currentTimeMillis()/1000); - MRole mRole = new MRole(roleName, now, - ownerName); + MRole mRole = new MRole(roleName, now, ownerName); pm.makePersistent(mRole); commited = commitTransaction(); success = true; @@ -3316,13 +3361,16 @@ public class ObjectStore implements RawStore, Configurable { return success; } - private MRoleMap getMSecurityUserRoleMap(String userName, - PrincipalType principalType, String roleName) { + private MRoleMap getMSecurityUserRoleMap(String userName, PrincipalType principalType, + String roleName) { MRoleMap mRoleMember = null; boolean commited = false; + Query query = null; try { openTransaction(); - Query query = pm.newQuery(MRoleMap.class, "principalName == t1 && principalType == t2 && role.roleName == t3"); + query = + pm.newQuery(MRoleMap.class, + "principalName == t1 && principalType == t2 && role.roleName == t3"); query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3"); query.setUnique(true); mRoleMember = (MRoleMap) query.executeWithArray(userName, principalType.toString(), roleName); @@ -3332,6 +3380,9 @@ public class ObjectStore implements RawStore, Configurable { if (!commited) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mRoleMember; } @@ -3340,6 +3391,7 @@ public class ObjectStore implements RawStore, Configurable { public boolean removeRole(String roleName) throws MetaException, NoSuchObjectException { boolean success = false; + QueryWrapper queryWrapper = new QueryWrapper(); try { openTransaction(); MRole mRol = getMRole(roleName); @@ -3352,10 +3404,11 @@ public class ObjectStore implements RawStore, Configurable { pm.deletePersistentAll(roleMap); } List roleMember = listMSecurityPrincipalMembershipRole(mRol - .getRoleName(), PrincipalType.ROLE); + .getRoleName(), PrincipalType.ROLE, queryWrapper); if (roleMember.size() > 0) { pm.deletePersistentAll(roleMember); } + queryWrapper.close(); // then remove all the grants List userGrants = listPrincipalGlobalGrants( mRol.getRoleName(), PrincipalType.ROLE); @@ -3363,30 +3416,36 @@ public class ObjectStore implements RawStore, Configurable { pm.deletePersistentAll(userGrants); } List dbGrants = listPrincipalAllDBGrant(mRol - .getRoleName(), PrincipalType.ROLE); + .getRoleName(), PrincipalType.ROLE, queryWrapper); if (dbGrants.size() > 0) { pm.deletePersistentAll(dbGrants); } + queryWrapper.close(); List tabPartGrants = listPrincipalAllTableGrants( - mRol.getRoleName(), PrincipalType.ROLE); + mRol.getRoleName(), PrincipalType.ROLE, queryWrapper); if (tabPartGrants.size() > 0) { pm.deletePersistentAll(tabPartGrants); } + queryWrapper.close(); List partGrants = listPrincipalAllPartitionGrants( - mRol.getRoleName(), PrincipalType.ROLE); + mRol.getRoleName(), PrincipalType.ROLE, queryWrapper); if (partGrants.size() > 0) { pm.deletePersistentAll(partGrants); } + queryWrapper.close(); List tblColumnGrants = listPrincipalAllTableColumnGrants( - mRol.getRoleName(), PrincipalType.ROLE); + mRol.getRoleName(), PrincipalType.ROLE, queryWrapper); if (tblColumnGrants.size() > 0) { pm.deletePersistentAll(tblColumnGrants); } + queryWrapper.close(); List partColumnGrants = listPrincipalAllPartitionColumnGrants( - mRol.getRoleName(), PrincipalType.ROLE); + mRol.getRoleName(), PrincipalType.ROLE, queryWrapper); if (partColumnGrants.size() > 0) { pm.deletePersistentAll(partColumnGrants); } + queryWrapper.close(); + // finally remove the role pm.deletePersistent(mRol); } @@ -3395,6 +3454,8 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + + queryWrapper.close(); } return success; } @@ -3443,66 +3504,62 @@ public class ObjectStore implements RawStore, Configurable { @SuppressWarnings("unchecked") @Override - public List listRoles(String principalName, - PrincipalType principalType) { + public List listRoles(String principalName, PrincipalType principalType) { boolean success = false; - List mRoleMember = null; + Query query = null; + List mRoleMember = new ArrayList(); + try { - openTransaction(); LOG.debug("Executing listRoles"); - Query query = pm - .newQuery( - MRoleMap.class, - "principalName == t1 && principalType == t2"); - query - .declareParameters("java.lang.String t1, java.lang.String t2"); + + openTransaction(); + query = pm.newQuery(MRoleMap.class, "principalName == t1 && principalType == t2"); + query.declareParameters("java.lang.String t1, java.lang.String t2"); query.setUnique(false); - mRoleMember = (List) query.executeWithArray( - principalName, principalType.toString()); - LOG.debug("Done executing query for listMSecurityUserRoleMap"); - pm.retrieveAll(mRoleMember); + List mRoles = + (List) query.executeWithArray(principalName, principalType.toString()); + pm.retrieveAll(mRoles); success = commitTransaction(); - LOG.debug("Done retrieving all objects for listMSecurityUserRoleMap"); + + mRoleMember.addAll(mRoles); + + LOG.debug("Done retrieving all objects for listRoles"); } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } if (principalType == PrincipalType.USER) { // All users belong to public role implicitly, add that role - if (mRoleMember == null) { - mRoleMember = new ArrayList(); - } else { - mRoleMember = new ArrayList(mRoleMember); - } MRole publicRole = new MRole(HiveMetaStore.PUBLIC, 0, HiveMetaStore.PUBLIC); - mRoleMember.add(new MRoleMap(principalName, principalType.toString(), publicRole, 0, - null, null, false)); + mRoleMember.add(new MRoleMap(principalName, principalType.toString(), publicRole, 0, null, + null, false)); } - return mRoleMember; + return mRoleMember; } @SuppressWarnings("unchecked") private List listMSecurityPrincipalMembershipRole(final String roleName, - final PrincipalType principalType) { + final PrincipalType principalType, + QueryWrapper queryWrapper) { boolean success = false; List mRoleMemebership = null; try { - openTransaction(); LOG.debug("Executing listMSecurityPrincipalMembershipRole"); - Query query = pm.newQuery(MRoleMap.class, - "principalName == t1 && principalType == t2"); - query - .declareParameters("java.lang.String t1, java.lang.String t2"); + + openTransaction(); + Query query = queryWrapper.query = pm.newQuery(MRoleMap.class, "principalName == t1 && principalType == t2"); + query.declareParameters("java.lang.String t1, java.lang.String t2"); mRoleMemebership = (List) query.execute(roleName, principalType.toString()); - LOG - .debug("Done executing query for listMSecurityPrincipalMembershipRole"); pm.retrieveAll(mRoleMemebership); success = commitTransaction(); - LOG - .debug("Done retrieving all objects for listMSecurityPrincipalMembershipRole"); + + LOG.debug("Done retrieving all objects for listMSecurityPrincipalMembershipRole"); } finally { if (!success) { rollbackTransaction(); @@ -3525,9 +3582,10 @@ public class ObjectStore implements RawStore, Configurable { private MRole getMRole(String roleName) { MRole mrole = null; boolean commited = false; + Query query = null; try { openTransaction(); - Query query = pm.newQuery(MRole.class, "roleName == t1"); + query = pm.newQuery(MRole.class, "roleName == t1"); query.declareParameters("java.lang.String t1"); query.setUnique(true); mrole = (MRole) query.execute(roleName); @@ -3537,6 +3595,9 @@ public class ObjectStore implements RawStore, Configurable { if (!commited) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mrole; } @@ -3544,13 +3605,14 @@ public class ObjectStore implements RawStore, Configurable { @Override public List listRoleNames() { boolean success = false; + Query query = null; try { openTransaction(); LOG.debug("Executing listAllRoleNames"); - Query query = pm.newQuery("select roleName from org.apache.hadoop.hive.metastore.model.MRole"); + query = pm.newQuery("select roleName from org.apache.hadoop.hive.metastore.model.MRole"); query.setResult("roleName"); Collection names = (Collection) query.execute(); - List roleNames = new ArrayList(); + List roleNames = new ArrayList(); for (Iterator i = names.iterator(); i.hasNext();) { roleNames.add((String) i.next()); } @@ -3560,6 +3622,9 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } } @@ -4368,49 +4433,61 @@ public class ObjectStore implements RawStore, Configurable { @Override public List listRoleMembers(String roleName) { boolean success = false; - List mRoleMemeberList = null; + Query query = null; + List mRoleMemeberList = new ArrayList(); try { + LOG.debug("Executing listRoleMembers"); + openTransaction(); - LOG.debug("Executing listMSecurityUserRoleMember"); - Query query = pm.newQuery(MRoleMap.class, - "role.roleName == t1"); + query = pm.newQuery(MRoleMap.class, "role.roleName == t1"); query.declareParameters("java.lang.String t1"); query.setUnique(false); - mRoleMemeberList = (List) query.execute( - roleName); - LOG.debug("Done executing query for listMSecurityUserRoleMember"); - pm.retrieveAll(mRoleMemeberList); + List mRoles = (List) query.execute(roleName); + pm.retrieveAll(mRoles); success = commitTransaction(); - LOG.debug("Done retrieving all objects for listMSecurityUserRoleMember"); + + mRoleMemeberList.addAll(mRoles); + + LOG.debug("Done retrieving all objects for listRoleMembers"); } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mRoleMemeberList; } @SuppressWarnings("unchecked") @Override - public List listPrincipalGlobalGrants(String principalName, PrincipalType principalType) { + public List listPrincipalGlobalGrants(String principalName, + PrincipalType principalType) { boolean commited = false; - List userNameDbPriv = null; + Query query = null; + List userNameDbPriv = new ArrayList(); try { + List mPrivs = null; openTransaction(); if (principalName != null) { - Query query = pm.newQuery(MGlobalPrivilege.class, - "principalName == t1 && principalType == t2 "); - query.declareParameters( - "java.lang.String t1, java.lang.String t2"); - userNameDbPriv = (List) query - .executeWithArray(principalName, principalType.toString()); - pm.retrieveAll(userNameDbPriv); + query = pm.newQuery(MGlobalPrivilege.class, "principalName == t1 && principalType == t2 "); + query.declareParameters("java.lang.String t1, java.lang.String t2"); + mPrivs = (List) query + .executeWithArray(principalName, principalType.toString()); + pm.retrieveAll(mPrivs); } commited = commitTransaction(); + if (mPrivs != null) { + userNameDbPriv.addAll(mPrivs); + } } finally { if (!commited) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return userNameDbPriv; } @@ -4418,9 +4495,10 @@ public class ObjectStore implements RawStore, Configurable { @Override public List listGlobalGrantsAll() { boolean commited = false; + Query query = null; try { openTransaction(); - Query query = pm.newQuery(MGlobalPrivilege.class); + query = pm.newQuery(MGlobalPrivilege.class); List userNameDbPriv = (List) query.execute(); pm.retrieveAll(userNameDbPriv); commited = commitTransaction(); @@ -4429,6 +4507,9 @@ public class ObjectStore implements RawStore, Configurable { if (!commited) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } } @@ -4452,25 +4533,32 @@ public class ObjectStore implements RawStore, Configurable { public List listPrincipalDBGrants(String principalName, PrincipalType principalType, String dbName) { boolean success = false; - List mSecurityDBList = null; + Query query = null; + List mSecurityDBList = new ArrayList(); dbName = HiveStringUtils.normalizeIdentifier(dbName); - try { - openTransaction(); LOG.debug("Executing listPrincipalDBGrants"); - Query query = pm.newQuery(MDBPrivilege.class, - "principalName == t1 && principalType == t2 && database.name == t3"); - query - .declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3"); - mSecurityDBList = (List) query.executeWithArray(principalName, principalType.toString(), dbName); - LOG.debug("Done executing query for listPrincipalDBGrants"); - pm.retrieveAll(mSecurityDBList); + + openTransaction(); + query = + pm.newQuery(MDBPrivilege.class, + "principalName == t1 && principalType == t2 && database.name == t3"); + query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3"); + List mPrivs = + (List) query.executeWithArray(principalName, principalType.toString(), + dbName); + pm.retrieveAll(mPrivs); success = commitTransaction(); + + mSecurityDBList.addAll(mPrivs); LOG.debug("Done retrieving all objects for listPrincipalDBGrants"); } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mSecurityDBList; } @@ -4478,12 +4566,22 @@ public class ObjectStore implements RawStore, Configurable { @Override public List listPrincipalDBGrantsAll( String principalName, PrincipalType principalType) { - return convertDB(listPrincipalAllDBGrant(principalName, principalType)); + QueryWrapper queryWrapper = new QueryWrapper(); + try { + return convertDB(listPrincipalAllDBGrant(principalName, principalType, queryWrapper)); + } finally { + queryWrapper.close(); + } } @Override public List listDBGrantsAll(String dbName) { - return convertDB(listDatabaseGrants(dbName)); + QueryWrapper queryWrapper = new QueryWrapper(); + try { + return convertDB(listDatabaseGrants(dbName, queryWrapper)); + } finally { + queryWrapper.close(); + } } private List convertDB(List privs) { @@ -4504,26 +4602,28 @@ public class ObjectStore implements RawStore, Configurable { } @SuppressWarnings("unchecked") - private List listPrincipalAllDBGrant( - String principalName, PrincipalType principalType) { + private List listPrincipalAllDBGrant(String principalName, + PrincipalType principalType, + QueryWrapper queryWrapper) { boolean success = false; + Query query = null; List mSecurityDBList = null; try { - openTransaction(); LOG.debug("Executing listPrincipalAllDBGrant"); + + openTransaction(); if (principalName != null && principalType != null) { - Query query = pm.newQuery(MDBPrivilege.class, - "principalName == t1 && principalType == t2"); - query - .declareParameters("java.lang.String t1, java.lang.String t2"); - mSecurityDBList = (List) query.execute(principalName, principalType.toString()); + query = queryWrapper.query = pm.newQuery(MDBPrivilege.class, "principalName == t1 && principalType == t2"); + query.declareParameters("java.lang.String t1, java.lang.String t2"); + mSecurityDBList = + (List) query.execute(principalName, principalType.toString()); } else { - Query query = pm.newQuery(MDBPrivilege.class); + query = queryWrapper.query = pm.newQuery(MDBPrivilege.class); mSecurityDBList = (List) query.execute(); } - LOG.debug("Done executing query for listPrincipalAllDBGrant"); pm.retrieveAll(mSecurityDBList); success = commitTransaction(); + LOG.debug("Done retrieving all objects for listPrincipalAllDBGrant"); } finally { if (!success) { @@ -4534,91 +4634,101 @@ public class ObjectStore implements RawStore, Configurable { } @SuppressWarnings("unchecked") - public List listAllTableGrants(String dbName, - String tableName) { + public List listAllTableGrants(String dbName, String tableName) { boolean success = false; + Query query = null; tableName = HiveStringUtils.normalizeIdentifier(tableName); dbName = HiveStringUtils.normalizeIdentifier(dbName); - List mSecurityTabList = null; + List mSecurityTabList = new ArrayList(); tableName = HiveStringUtils.normalizeIdentifier(tableName); dbName = HiveStringUtils.normalizeIdentifier(dbName); try { - openTransaction(); LOG.debug("Executing listAllTableGrants"); + + openTransaction(); String queryStr = "table.tableName == t1 && table.database.name == t2"; - Query query = pm.newQuery( - MTablePrivilege.class, queryStr); - query.declareParameters( - "java.lang.String t1, java.lang.String t2"); - mSecurityTabList = (List) query - .executeWithArray(tableName, dbName); + query = pm.newQuery(MTablePrivilege.class, queryStr); + query.declareParameters("java.lang.String t1, java.lang.String t2"); + List mPrivs = (List) query.executeWithArray(tableName, dbName); LOG.debug("Done executing query for listAllTableGrants"); - pm.retrieveAll(mSecurityTabList); + pm.retrieveAll(mPrivs); success = commitTransaction(); - LOG - .debug("Done retrieving all objects for listAllTableGrants"); + + mSecurityTabList.addAll(mPrivs); + + LOG.debug("Done retrieving all objects for listAllTableGrants"); } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mSecurityTabList; } @SuppressWarnings("unchecked") - public List listTableAllPartitionGrants(String dbName, - String tableName) { + public List listTableAllPartitionGrants(String dbName, String tableName) { tableName = HiveStringUtils.normalizeIdentifier(tableName); dbName = HiveStringUtils.normalizeIdentifier(dbName); boolean success = false; - List mSecurityTabPartList = null; + Query query = null; + List mSecurityTabPartList = new ArrayList(); try { - openTransaction(); LOG.debug("Executing listTableAllPartitionGrants"); + + openTransaction(); String queryStr = "partition.table.tableName == t1 && partition.table.database.name == t2"; - Query query = pm.newQuery( - MPartitionPrivilege.class, queryStr); - query.declareParameters( - "java.lang.String t1, java.lang.String t2"); - mSecurityTabPartList = (List) query - .executeWithArray(tableName, dbName); - LOG.debug("Done executing query for listTableAllPartitionGrants"); - pm.retrieveAll(mSecurityTabPartList); + query = pm.newQuery(MPartitionPrivilege.class, queryStr); + query.declareParameters("java.lang.String t1, java.lang.String t2"); + List mPrivs = (List) query.executeWithArray(tableName, dbName); + pm.retrieveAll(mPrivs); success = commitTransaction(); - LOG - .debug("Done retrieving all objects for listTableAllPartitionGrants"); + + mSecurityTabPartList.addAll(mPrivs); + + LOG.debug("Done retrieving all objects for listTableAllPartitionGrants"); } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mSecurityTabPartList; } @SuppressWarnings("unchecked") - public List listTableAllColumnGrants(String dbName, - String tableName) { + public List listTableAllColumnGrants(String dbName, String tableName) { boolean success = false; - List mTblColPrivilegeList = null; + Query query = null; + List mTblColPrivilegeList = new ArrayList(); tableName = HiveStringUtils.normalizeIdentifier(tableName); dbName = HiveStringUtils.normalizeIdentifier(dbName); - try { - openTransaction(); LOG.debug("Executing listTableAllColumnGrants"); + + openTransaction(); String queryStr = "table.tableName == t1 && table.database.name == t2"; - Query query = pm.newQuery(MTableColumnPrivilege.class, queryStr); + query = pm.newQuery(MTableColumnPrivilege.class, queryStr); query.declareParameters("java.lang.String t1, java.lang.String t2"); - mTblColPrivilegeList = (List) query - .executeWithArray(tableName, dbName); - LOG.debug("Done executing query for listTableAllColumnGrants"); - pm.retrieveAll(mTblColPrivilegeList); + List mPrivs = + (List) query.executeWithArray(tableName, dbName); + pm.retrieveAll(mPrivs); success = commitTransaction(); + + mTblColPrivilegeList.addAll(mPrivs); + LOG.debug("Done retrieving all objects for listTableAllColumnGrants"); } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mTblColPrivilegeList; } @@ -4627,26 +4737,32 @@ public class ObjectStore implements RawStore, Configurable { public List listTableAllPartitionColumnGrants(String dbName, String tableName) { boolean success = false; + Query query = null; tableName = HiveStringUtils.normalizeIdentifier(tableName); dbName = HiveStringUtils.normalizeIdentifier(dbName); - - List mSecurityColList = null; + List mSecurityColList = new ArrayList(); try { - openTransaction(); LOG.debug("Executing listTableAllPartitionColumnGrants"); + + openTransaction(); String queryStr = "partition.table.tableName == t1 && partition.table.database.name == t2"; - Query query = pm.newQuery(MPartitionColumnPrivilege.class, queryStr); + query = pm.newQuery(MPartitionColumnPrivilege.class, queryStr); query.declareParameters("java.lang.String t1, java.lang.String t2"); - mSecurityColList = (List) query - .executeWithArray(tableName, dbName); - LOG.debug("Done executing query for listTableAllPartitionColumnGrants"); - pm.retrieveAll(mSecurityColList); + List mPrivs = + (List) query.executeWithArray(tableName, dbName); + pm.retrieveAll(mPrivs); success = commitTransaction(); + + mSecurityColList.addAll(mPrivs); + LOG.debug("Done retrieving all objects for listTableAllPartitionColumnGrants"); } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mSecurityColList; } @@ -4686,19 +4802,17 @@ public class ObjectStore implements RawStore, Configurable { } @SuppressWarnings("unchecked") - private List listDatabaseGrants(String dbName) { + private List listDatabaseGrants(String dbName, QueryWrapper queryWrapper) { dbName = HiveStringUtils.normalizeIdentifier(dbName); - boolean success = false; + try { - openTransaction(); LOG.debug("Executing listDatabaseGrants"); - Query query = pm.newQuery(MDBPrivilege.class, - "database.name == t1"); + + openTransaction(); + Query query = queryWrapper.query = pm.newQuery(MDBPrivilege.class, "database.name == t1"); query.declareParameters("java.lang.String t1"); - List mSecurityDBList = (List) query - .executeWithArray(dbName); - LOG.debug("Done executing query for listDatabaseGrants"); + List mSecurityDBList = (List) query.executeWithArray(dbName); pm.retrieveAll(mSecurityDBList); success = commitTransaction(); LOG.debug("Done retrieving all objects for listDatabaseGrants"); @@ -4774,162 +4888,181 @@ public class ObjectStore implements RawStore, Configurable { @Override @SuppressWarnings("unchecked") - public List listAllTableGrants( - String principalName, PrincipalType principalType, String dbName, - String tableName) { + public List listAllTableGrants(String principalName, + PrincipalType principalType, String dbName, String tableName) { tableName = HiveStringUtils.normalizeIdentifier(tableName); dbName = HiveStringUtils.normalizeIdentifier(dbName); - boolean success = false; - List mSecurityTabPartList = null; + Query query = null; + List mSecurityTabPartList = new ArrayList(); try { openTransaction(); LOG.debug("Executing listAllTableGrants"); - Query query = pm.newQuery( - MTablePrivilege.class, + query = + pm.newQuery(MTablePrivilege.class, "principalName == t1 && principalType == t2 && table.tableName == t3 && table.database.name == t4"); - query.declareParameters( - "java.lang.String t1, java.lang.String t2, java.lang.String t3, java.lang.String t4"); - mSecurityTabPartList = (List) query - .executeWithArray(principalName, principalType.toString(), tableName, dbName); - LOG.debug("Done executing query for listAllTableGrants"); - pm.retrieveAll(mSecurityTabPartList); + query + .declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3, java.lang.String t4"); + List mPrivs = + (List) query.executeWithArray(principalName, principalType.toString(), + tableName, dbName); + pm.retrieveAll(mPrivs); success = commitTransaction(); - LOG - .debug("Done retrieving all objects for listAllTableGrants"); + + mSecurityTabPartList.addAll(mPrivs); + + LOG.debug("Done retrieving all objects for listAllTableGrants"); } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mSecurityTabPartList; } @SuppressWarnings("unchecked") @Override - public List listPrincipalPartitionGrants( - String principalName, PrincipalType principalType, String dbName, - String tableName, String partName) { + public List listPrincipalPartitionGrants(String principalName, + PrincipalType principalType, String dbName, String tableName, String partName) { boolean success = false; + Query query = null; tableName = HiveStringUtils.normalizeIdentifier(tableName); dbName = HiveStringUtils.normalizeIdentifier(dbName); - - List mSecurityTabPartList = null; + List mSecurityTabPartList = new ArrayList(); try { - openTransaction(); - LOG.debug("Executing listMSecurityPrincipalPartitionGrant"); - Query query = pm.newQuery( - MPartitionPrivilege.class, - "principalName == t1 && principalType == t2 && partition.table.tableName == t3 " + - "&& partition.table.database.name == t4 && partition.partitionName == t5"); - query.declareParameters( - "java.lang.String t1, java.lang.String t2, java.lang.String t3, java.lang.String t4, " + - "java.lang.String t5"); - mSecurityTabPartList = (List) query - .executeWithArray(principalName, principalType.toString(), tableName, dbName, partName); - LOG.debug("Done executing query for listMSecurityPrincipalPartitionGrant"); + LOG.debug("Executing listPrincipalPartitionGrants"); - pm.retrieveAll(mSecurityTabPartList); + openTransaction(); + query = + pm.newQuery(MPartitionPrivilege.class, + "principalName == t1 && principalType == t2 && partition.table.tableName == t3 " + + "&& partition.table.database.name == t4 && partition.partitionName == t5"); + query + .declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3, java.lang.String t4, " + + "java.lang.String t5"); + List mPrivs = + (List) query.executeWithArray(principalName, + principalType.toString(), tableName, dbName, partName); + pm.retrieveAll(mPrivs); success = commitTransaction(); - LOG.debug("Done retrieving all objects for listMSecurityPrincipalPartitionGrant"); + + mSecurityTabPartList.addAll(mPrivs); + + LOG.debug("Done retrieving all objects for listPrincipalPartitionGrants"); } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mSecurityTabPartList; } @SuppressWarnings("unchecked") @Override - public List listPrincipalTableColumnGrants( - String principalName, PrincipalType principalType, String dbName, - String tableName, String columnName) { + public List listPrincipalTableColumnGrants(String principalName, + PrincipalType principalType, String dbName, String tableName, String columnName) { boolean success = false; + Query query = null; tableName = HiveStringUtils.normalizeIdentifier(tableName); dbName = HiveStringUtils.normalizeIdentifier(dbName); columnName = HiveStringUtils.normalizeIdentifier(columnName); - List mSecurityColList = null; + List mSecurityColList = new ArrayList(); try { - openTransaction(); LOG.debug("Executing listPrincipalTableColumnGrants"); - String queryStr = "principalName == t1 && principalType == t2 && " + - "table.tableName == t3 && table.database.name == t4 && columnName == t5 "; - Query query = pm.newQuery(MTableColumnPrivilege.class, queryStr); - query - .declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3, " + - "java.lang.String t4, java.lang.String t5"); - mSecurityColList = (List) query.executeWithArray( - principalName, principalType.toString(), tableName, dbName, columnName); - LOG.debug("Done executing query for listPrincipalTableColumnGrants"); - pm.retrieveAll(mSecurityColList); + + openTransaction(); + String queryStr = + "principalName == t1 && principalType == t2 && " + + "table.tableName == t3 && table.database.name == t4 && columnName == t5 "; + query = pm.newQuery(MTableColumnPrivilege.class, queryStr); + query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3, " + + "java.lang.String t4, java.lang.String t5"); + List mPrivs = + (List) query.executeWithArray(principalName, + principalType.toString(), tableName, dbName, columnName); + pm.retrieveAll(mPrivs); success = commitTransaction(); - LOG - .debug("Done retrieving all objects for listPrincipalTableColumnGrants"); + + mSecurityColList.addAll(mPrivs); + + LOG.debug("Done retrieving all objects for listPrincipalTableColumnGrants"); } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mSecurityColList; } @Override @SuppressWarnings("unchecked") - public List listPrincipalPartitionColumnGrants( - String principalName, PrincipalType principalType, String dbName, - String tableName, String partitionName, String columnName) { + public List listPrincipalPartitionColumnGrants(String principalName, + PrincipalType principalType, String dbName, String tableName, String partitionName, + String columnName) { boolean success = false; + Query query = null; tableName = HiveStringUtils.normalizeIdentifier(tableName); dbName = HiveStringUtils.normalizeIdentifier(dbName); columnName = HiveStringUtils.normalizeIdentifier(columnName); - - List mSecurityColList = null; + List mSecurityColList = new ArrayList(); try { - openTransaction(); LOG.debug("Executing listPrincipalPartitionColumnGrants"); - Query query = pm - .newQuery( + + openTransaction(); + query = pm.newQuery( MPartitionColumnPrivilege.class, - "principalName == t1 && principalType == t2 && partition.table.tableName == t3 " + - "&& partition.table.database.name == t4 && partition.partitionName == t5 && columnName == t6"); - query - .declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3, " + - "java.lang.String t4, java.lang.String t5, java.lang.String t6"); + "principalName == t1 && principalType == t2 && partition.table.tableName == t3 " + + "&& partition.table.database.name == t4 && partition.partitionName == t5 && columnName == t6"); + query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3, " + + "java.lang.String t4, java.lang.String t5, java.lang.String t6"); + List mPrivs = + (List) query.executeWithArray(principalName, + principalType.toString(), tableName, dbName, partitionName, columnName); + pm.retrieveAll(mPrivs); + success = commitTransaction(); - mSecurityColList = (List) query - .executeWithArray(principalName, principalType.toString(), tableName, - dbName, partitionName, columnName); - LOG.debug("Done executing query for listPrincipalPartitionColumnGrants"); - pm.retrieveAll(mSecurityColList); + mSecurityColList.addAll(mPrivs); - success = commitTransaction(); - LOG - .debug("Done retrieving all objects for listPrincipalPartitionColumnGrants"); + LOG.debug("Done retrieving all objects for listPrincipalPartitionColumnGrants"); } finally { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } return mSecurityColList; } @Override - public List listPrincipalPartitionColumnGrantsAll( - String principalName, PrincipalType principalType) { + public List listPrincipalPartitionColumnGrantsAll(String principalName, + PrincipalType principalType) { boolean success = false; + Query query = null; try { openTransaction(); LOG.debug("Executing listPrincipalPartitionColumnGrantsAll"); List mSecurityTabPartList; if (principalName != null && principalType != null) { - Query query = pm.newQuery(MPartitionColumnPrivilege.class, - "principalName == t1 && principalType == t2"); + query = + pm.newQuery(MPartitionColumnPrivilege.class, + "principalName == t1 && principalType == t2"); query.declareParameters("java.lang.String t1, java.lang.String t2"); - mSecurityTabPartList = (List) - query.executeWithArray(principalName, principalType.toString()); + mSecurityTabPartList = + (List) query.executeWithArray(principalName, + principalType.toString()); } else { - Query query = pm.newQuery(MPartitionColumnPrivilege.class); + query = pm.newQuery(MPartitionColumnPrivilege.class); mSecurityTabPartList = (List) query.execute(); } LOG.debug("Done executing query for listPrincipalPartitionColumnGrantsAll"); @@ -4942,23 +5075,29 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } } @Override - public List listPartitionColumnGrantsAll( - String dbName, String tableName, String partitionName, String columnName) { + public List listPartitionColumnGrantsAll(String dbName, String tableName, + String partitionName, String columnName) { boolean success = false; + Query query = null; try { openTransaction(); LOG.debug("Executing listPartitionColumnGrantsAll"); - Query query = pm.newQuery(MPartitionColumnPrivilege.class, - "partition.table.tableName == t3 && partition.table.database.name == t4 && " + - "partition.partitionName == t5 && columnName == t6"); - query.declareParameters( - "java.lang.String t3, java.lang.String t4, java.lang.String t5, java.lang.String t6"); - List mSecurityTabPartList = (List) - query.executeWithArray(tableName, dbName, partitionName, columnName); + query = + pm.newQuery(MPartitionColumnPrivilege.class, + "partition.table.tableName == t3 && partition.table.database.name == t4 && " + + "partition.partitionName == t5 && columnName == t6"); + query + .declareParameters("java.lang.String t3, java.lang.String t4, java.lang.String t5, java.lang.String t6"); + List mSecurityTabPartList = + (List) query.executeWithArray(tableName, dbName, + partitionName, columnName); LOG.debug("Done executing query for listPartitionColumnGrantsAll"); pm.retrieveAll(mSecurityTabPartList); List result = convertPartCols(mSecurityTabPartList); @@ -4969,6 +5108,9 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } } @@ -4994,23 +5136,22 @@ public class ObjectStore implements RawStore, Configurable { @SuppressWarnings("unchecked") private List listPrincipalAllTableGrants( - String principalName, PrincipalType principalType) { + String principalName, PrincipalType principalType, QueryWrapper queryWrapper) { boolean success = false; List mSecurityTabPartList = null; try { - openTransaction(); LOG.debug("Executing listPrincipalAllTableGrants"); - Query query = pm.newQuery(MTablePrivilege.class, + + openTransaction(); + Query query = queryWrapper.query = pm.newQuery(MTablePrivilege.class, "principalName == t1 && principalType == t2"); query.declareParameters("java.lang.String t1, java.lang.String t2"); mSecurityTabPartList = (List) query.execute( principalName, principalType.toString()); - LOG - .debug("Done executing query for listPrincipalAllTableGrants"); pm.retrieveAll(mSecurityTabPartList); success = commitTransaction(); - LOG - .debug("Done retrieving all objects for listPrincipalAllTableGrants"); + + LOG.debug("Done retrieving all objects for listPrincipalAllTableGrants"); } finally { if (!success) { rollbackTransaction(); @@ -5020,21 +5161,21 @@ public class ObjectStore implements RawStore, Configurable { } @Override - public List listPrincipalTableGrantsAll( - String principalName, PrincipalType principalType) { + public List listPrincipalTableGrantsAll(String principalName, + PrincipalType principalType) { boolean success = false; + Query query = null; try { openTransaction(); LOG.debug("Executing listPrincipalAllTableGrants"); List mSecurityTabPartList; if (principalName != null && principalType != null) { - Query query = pm.newQuery(MTablePrivilege.class, - "principalName == t1 && principalType == t2"); + query = pm.newQuery(MTablePrivilege.class, "principalName == t1 && principalType == t2"); query.declareParameters("java.lang.String t1, java.lang.String t2"); - mSecurityTabPartList = (List) query.execute( - principalName, principalType.toString()); + mSecurityTabPartList = + (List) query.execute(principalName, principalType.toString()); } else { - Query query = pm.newQuery(MTablePrivilege.class); + query = pm.newQuery(MTablePrivilege.class); mSecurityTabPartList = (List) query.execute(); } LOG.debug("Done executing query for listPrincipalAllTableGrants"); @@ -5047,20 +5188,24 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } } @Override public List listTableGrantsAll(String dbName, String tableName) { boolean success = false; + Query query = null; try { openTransaction(); LOG.debug("Executing listTableGrantsAll"); - Query query = pm.newQuery(MTablePrivilege.class, - "table.tableName == t1 && table.database.name == t2"); + query = + pm.newQuery(MTablePrivilege.class, "table.tableName == t1 && table.database.name == t2"); query.declareParameters("java.lang.String t1, java.lang.String t2"); - List mSecurityTabPartList = (List) - query.executeWithArray(tableName, dbName); + List mSecurityTabPartList = + (List) query.executeWithArray(tableName, dbName); LOG.debug("Done executing query for listTableGrantsAll"); pm.retrieveAll(mSecurityTabPartList); List result = convertTable(mSecurityTabPartList); @@ -5071,6 +5216,9 @@ public class ObjectStore implements RawStore, Configurable { if (!success) { rollbackTransaction(); } + if (query != null) { + query.closeAll(); + } } } @@ -5094,24 +5242,20 @@ public class ObjectStore implements RawStore, Configurable { } @SuppressWarnings("unchecked") - private List listPrincipalAllPartitionGrants( - String principalName, PrincipalType principalType) { + private List listPrincipalAllPartitionGrants(String principalName, + PrincipalType principalType, QueryWrapper queryWrapper) { boolean success = false; List mSecurityTabPartList = null; try { openTransaction(); LOG.debug("Executing listPrincipalAllPartitionGrants"); - Query query = pm.newQuery(MPartitionPrivilege.class, - "principalName == t1 && principalType == t2"); + Query query = queryWrapper.query = pm.newQuery(MPartitionPrivilege.class, "principalName == t1 && principalType == t2"); query.declareParameters("java.lang.String t1, java.lang.String t2"); - mSecurityTabPartList = (List) query.execute( - principalName, principalType.toString()); - LOG - .debug("Done executing query for listPrincipalAllPartitionGrants"); + mSecurityTabPartList = + (List) query.execute(principalName, principalType.toString()); pm.retrieveAll(mSecurityTabPartList); success = commitTransaction(); - LOG - .debug("Done retrieving all objects for listPrincipalAllPartitionGrants"); + LOG.debug("Done retrieving all objects for listPrincipalAllPartitionGrants"); } finally { if (!success) { rollbackTransaction(); @@ -5121,21 +5265,22 @@ public class ObjectStore implements RawStore, Configurable { } @Override - public List listPrincipalPartitionGrantsAll( - String principalName, PrincipalType principalType) { + public List listPrincipalPartitionGrantsAll(String principalName, + PrincipalType principalType) { boolean success = false; + Query query = null; try { openTransaction(); LOG.debug