You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vg...@apache.org on 2015/07/09 19:23:54 UTC

[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/08595ffa/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 8f52f83..39ab9e7 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -217,6 +217,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() {
   }
 
@@ -551,10 +574,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);
@@ -564,6 +588,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);
@@ -666,6 +693,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();
 
@@ -673,7 +701,7 @@ public class ObjectStore implements RawStore, Configurable {
       MDatabase db = getMDatabase(dbname);
       pm.retrieve(db);
       if (db != null) {
-        List<MDBPrivilege> dbGrants = this.listDatabaseGrants(dbname);
+        List<MDBPrivilege> dbGrants = this.listDatabaseGrants(dbname, queryWrapper);
         if (dbGrants != null && dbGrants.size() > 0) {
           pm.deletePersistentAll(dbGrants);
         }
@@ -684,36 +712,36 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      queryWrapper.close();
     }
     return success;
   }
 
-
   @Override
   public List<String> getDatabases(String pattern) throws MetaException {
     boolean commited = false;
     List<String> 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<String>();
       for (Iterator i = names.iterator(); i.hasNext();) {
         databases.add((String) i.next());
@@ -723,6 +751,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!commited) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
     return databases;
   }
@@ -781,9 +812,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());
@@ -796,6 +828,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!commited) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
     return type;
   }
@@ -803,9 +838,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());
@@ -821,6 +857,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
     return success;
   }
@@ -956,9 +995,9 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<String> getTables(String dbName, String pattern)
-      throws MetaException {
+  public List<String> getTables(String dbName, String pattern) throws MetaException {
     boolean commited = false;
+    Query query = null;
     List<String> tbls = null;
     try {
       openTransaction();
@@ -966,25 +1005,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<String>();
       for (Iterator i = names.iterator(); i.hasNext();) {
         tbls.add((String) i.next());
@@ -994,6 +1032,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!commited) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
     return tbls;
   }
@@ -1006,11 +1047,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);
@@ -1020,20 +1062,24 @@ public class ObjectStore implements RawStore, Configurable {
       if (!commited) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
     return mtbl;
   }
 
   @Override
-  public List<Table> getTableObjectsByName(String db, List<String> tbl_names)
-      throws MetaException, UnknownDBException {
+  public List<Table> getTableObjectsByName(String db, List<String> tbl_names) throws MetaException,
+      UnknownDBException {
     List<Table> tables = new ArrayList<Table>();
     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");
@@ -1046,7 +1092,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);
@@ -1058,6 +1104,12 @@ public class ObjectStore implements RawStore, Configurable {
       if (!committed) {
         rollbackTransaction();
       }
+      if (dbExistsQuery != null) {
+        dbExistsQuery.closeAll();
+      }
+      if (query != null) {
+        query.closeAll();
+      }
     }
     return tables;
   }
@@ -1208,9 +1260,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;
     }
@@ -1296,8 +1348,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.
@@ -1404,7 +1454,6 @@ public class ObjectStore implements RawStore, Configurable {
     return !doesExist;
   }
 
-
   @Override
   public boolean addPartitions(String dbName, String tblName,
                                PartitionSpecProxy partitionSpec, boolean ifNotExists)
@@ -1531,10 +1580,11 @@ public class ObjectStore implements RawStore, Configurable {
     return part;
   }
 
-  private MPartition getMPartition(String dbName, String tableName,
-      List<String> part_vals) throws MetaException {
+  private MPartition getMPartition(String dbName, String tableName, List<String> part_vals)
+      throws MetaException {
     MPartition mpart = null;
     boolean commited = false;
+    Query query = null;
     try {
       openTransaction();
       dbName = HiveStringUtils.normalizeIdentifier(dbName);
@@ -1546,10 +1596,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);
@@ -1559,6 +1610,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!commited) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
     return mpart;
   }
@@ -1750,8 +1804,13 @@ public class ObjectStore implements RawStore, Configurable {
       }
       @Override
       protected List<Partition> getJdoResult(
-          GetHelper<List<Partition>> ctx) throws MetaException, NoSuchObjectException {
-        return convertToParts(listMPartitions(dbName, tblName, maxParts));
+          GetHelper<List<Partition>> ctx) throws MetaException {
+        QueryWrapper queryWrapper = new QueryWrapper();
+        try {
+          return convertToParts(listMPartitions(dbName, tblName, maxParts, queryWrapper));
+        } finally {
+          queryWrapper.close();
+        }
       }
     }.run(false);
   }
@@ -1759,11 +1818,13 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public List<Partition> getPartitionsWithAuth(String dbName, String tblName,
       short max, String userName, List<String> groupNames)
-      throws MetaException, NoSuchObjectException, InvalidObjectException {
+          throws MetaException, InvalidObjectException {
     boolean success = false;
+    QueryWrapper queryWrapper = new QueryWrapper();
+
     try {
       openTransaction();
-      List<MPartition> mparts = listMPartitions(dbName, tblName, max);
+      List<MPartition> mparts = listMPartitions(dbName, tblName, max, queryWrapper);
       List<Partition> parts = new ArrayList<Partition>(mparts.size());
       if (mparts != null && mparts.size()>0) {
         for (MPartition mpart : mparts) {
@@ -1786,6 +1847,7 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      queryWrapper.close();
     }
   }
 
@@ -1822,7 +1884,6 @@ public class ObjectStore implements RawStore, Configurable {
     }
   }
 
-
   private List<Partition> convertToParts(List<MPartition> mparts) throws MetaException {
     return convertToParts(mparts, null);
   }
@@ -1875,20 +1936,22 @@ public class ObjectStore implements RawStore, Configurable {
     List<String> pns = new ArrayList<String>();
     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;
   }
 
@@ -1908,51 +1971,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<String> part_vals, short max_parts, String resultsCol)
+      List<String> 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<FieldSchema> 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
@@ -1961,11 +2019,13 @@ public class ObjectStore implements RawStore, Configurable {
       throws MetaException, InvalidObjectException, NoSuchObjectException {
     List<Partition> partitions = new ArrayList<Partition>();
     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);
@@ -1985,6 +2045,7 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      queryWrapper.close();
     }
     return partitions;
   }
@@ -1994,11 +2055,13 @@ public class ObjectStore implements RawStore, Configurable {
       List<String> part_vals, short max_parts) throws MetaException, NoSuchObjectException {
     List<String> partitionNames = new ArrayList<String>();
     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);
       }
@@ -2007,14 +2070,13 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      queryWrapper.close();
     }
     return partitionNames;
   }
 
   // TODO:pc implement max
-  private List<MPartition> listMPartitions(String dbName, String tableName,
-      int max) {
-
+  private List<MPartition> listMPartitions(String dbName, String tableName, int max, QueryWrapper queryWrapper) {
     boolean success = false;
     List<MPartition> mparts = null;
     try {
@@ -2022,11 +2084,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<MPartition>) query.execute(tableName, dbName);
@@ -2216,8 +2277,8 @@ public class ObjectStore implements RawStore, Configurable {
   private List<Partition> getPartitionsViaOrmFilter(Table table, ExpressionTree tree,
       short maxParts, boolean isValidatedFilter) throws MetaException {
     Map<String, Object> params = new HashMap<String, Object>();
-    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;
@@ -2227,14 +2288,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<MPartition> mparts = (List<MPartition>) 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");
@@ -2243,10 +2301,6 @@ public class ObjectStore implements RawStore, Configurable {
     return results;
   }
 
-  private static class Out<T> {
-    public T val;
-  }
-
   /**
    * Gets partition names from the table via ORM (JDOQL) name filter.
    * @param dbName Database name.
@@ -2259,16 +2313,19 @@ public class ObjectStore implements RawStore, Configurable {
     if (partNames.isEmpty()) {
       return new ArrayList<Partition>();
     }
-    Out<Query> query = new Out<Query>();
-    List<MPartition> mparts = null;
-    try {
-      mparts = getMPartitionsViaOrmFilter(dbName, tblName, partNames, query);
-      return convertToParts(dbName, tblName, mparts);
-    } finally {
-      if (query.val != null) {
-        query.val.closeAll();
-      }
+    ObjectPair<Query, Map<String, String>> queryWithParams =
+        getPartQueryWithParams(dbName, tblName, partNames);
+    Query query = queryWithParams.getFirst();
+    query.setResultClass(MPartition.class);
+    query.setClass(MPartition.class);
+    query.setOrdering("partitionName ascending");
+    @SuppressWarnings("unchecked")
+    List<MPartition> mparts = (List<MPartition>)query.executeWithMap(queryWithParams.getSecond());
+    List<Partition> partitions = convertToParts(dbName, tblName, mparts);
+    if (query != null) {
+      query.closeAll();
     }
+    return partitions;
   }
 
   private void dropPartitionsNoTxn(String dbName, String tblName, List<String> partNames) {
@@ -2304,27 +2361,15 @@ public class ObjectStore implements RawStore, Configurable {
         sd.setCD(null);
       }
     }
+    if (query != null) {
+      query.closeAll();
+    }
     return candidateCds;
   }
 
-  private List<MPartition> getMPartitionsViaOrmFilter(String dbName,
-      String tblName, List<String> partNames, Out<Query> out) {
-    ObjectPair<Query, Map<String, String>> 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<MPartition> result = (List<MPartition>)query.executeWithMap(queryWithParams.getSecond());
-    return result;
-  }
-
-  private ObjectPair<Query, Map<String, String>> getPartQueryWithParams(
-      String dbName, String tblName, List<String> partNames) {
-    StringBuilder sb = new StringBuilder(
-        "table.tableName == t1 && table.database.name == t2 && (");
+  private ObjectPair<Query, Map<String, String>> getPartQueryWithParams(String dbName,
+      String tblName, List<String> partNames) {
+    StringBuilder sb = new StringBuilder("table.tableName == t1 && table.database.name == t2 && (");
     int n = 0;
     Map<String, String> params = new HashMap<String, String>();
     for (Iterator<String> itr = partNames.iterator(); itr.hasNext();) {
@@ -2337,16 +2382,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, Map<String,String>>(query, params);
+    return new ObjectPair<Query, Map<String, String>>(query, params);
   }
 
   @Override
@@ -2668,6 +2710,7 @@ public class ObjectStore implements RawStore, Configurable {
   public List<String> listTableNamesByFilter(String dbName, String filter, short maxTables)
       throws MetaException {
     boolean success = false;
+    Query query = null;
     List<String> tableNames = new ArrayList<String>();
     try {
       openTransaction();
@@ -2675,7 +2718,7 @@ public class ObjectStore implements RawStore, Configurable {
       dbName = HiveStringUtils.normalizeIdentifier(dbName);
       Map<String, Object> params = new HashMap<String, Object>();
       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);
@@ -2684,14 +2727,14 @@ public class ObjectStore implements RawStore, Configurable {
       }
       LOG.debug("filter specified is " + filter + "," + " JDOQL filter is " + queryFilterString);
       for (Entry<String, Object> 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<String> tableNamesSet = new HashSet<String>();
       for (Iterator i = names.iterator(); i.hasNext();) {
         tableNamesSet.add((String) i.next());
@@ -2700,58 +2743,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<String> listPartitionNamesByFilter(String dbName, String tableName,
-      String filter, short maxParts) throws MetaException {
+  public List<String> listPartitionNamesByFilter(String dbName, String tableName, String filter,
+      short maxParts) throws MetaException {
     boolean success = false;
+    Query query = null;
     List<String> partNames = new ArrayList<String>();
     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<String, Object> params = new HashMap<String, Object>();
       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<String>();
       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");
@@ -2759,6 +2798,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
     return partNames;
   }
@@ -2962,10 +3004,12 @@ public class ObjectStore implements RawStore, Configurable {
     }
 
     boolean success = false;
+    QueryWrapper queryWrapper = new QueryWrapper();
+
     try {
       openTransaction();
       LOG.debug("execute removeUnusedColumnDescriptor");
-      List<MStorageDescriptor> referencedSDs = listStorageDescriptorsWithCD(oldCD, 1);
+      List<MStorageDescriptor> 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);
@@ -2977,6 +3021,7 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      queryWrapper.close();
     }
   }
 
@@ -3005,21 +3050,22 @@ public class ObjectStore implements RawStore, Configurable {
    * @param maxSDs the maximum number of SDs to return
    * @return a list of storage descriptors
    */
-  private List<MStorageDescriptor> listStorageDescriptorsWithCD(MColumnDescriptor oldCD,
-      long maxSDs) {
+  private List<MStorageDescriptor> listStorageDescriptorsWithCD(
+      MColumnDescriptor oldCD,
+      long maxSDs,
+      QueryWrapper queryWrapper) {
     boolean success = false;
     List<MStorageDescriptor> 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<MStorageDescriptor>) query.execute(oldCD);
+      sds = (List<MStorageDescriptor>)query.execute(oldCD);
       LOG.debug("Done executing query for listStorageDescriptorsWithCD");
       pm.retrieveAll(sds);
       success = commitTransaction();
@@ -3096,9 +3142,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);
@@ -3108,19 +3156,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;
   }
@@ -3161,64 +3213,55 @@ public class ObjectStore implements RawStore, Configurable {
   public List<Index> getIndexes(String dbName, String origTableName, int max)
       throws MetaException {
     boolean success = false;
+    Query query = null;
     try {
+      LOG.debug("Executing getIndexes");
       openTransaction();
-      List<MIndex> mIndexList = listMIndexes(dbName, origTableName, max);
-      List<Index> indexes = new ArrayList<Index>(mIndexList.size());
-      for (MIndex midx : mIndexList) {
-        indexes.add(this.convertToIndex(midx));
-      }
-      success = commitTransaction();
-      return indexes;
-    } finally {
-      if (!success) {
-        rollbackTransaction();
-      }
-    }
-  }
 
-  private List<MIndex> listMIndexes(String dbName, String origTableName,
-      int max) {
-    boolean success = false;
-    List<MIndex> 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<MIndex>) query.execute(origTableName, dbName);
-      LOG.debug("Done executing query for listMIndexes");
-      pm.retrieveAll(mindexes);
+      List<MIndex> mIndexes = (List<MIndex>) query.execute(origTableName, dbName);
+      pm.retrieveAll(mIndexes);
+
+      List<Index> indexes = new ArrayList<Index>(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<String> listIndexNames(String dbName, String origTableName,
-      short max) throws MetaException {
+  public List<String> listIndexNames(String dbName, String origTableName, short max)
+      throws MetaException {
     List<String> pns = new ArrayList<String>();
     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());
       }
@@ -3227,6 +3270,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
     return pns;
   }
@@ -3243,8 +3289,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;
@@ -3334,13 +3379,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);
@@ -3350,6 +3398,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!commited) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
     return mRoleMember;
   }
@@ -3358,6 +3409,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);
@@ -3370,10 +3422,11 @@ public class ObjectStore implements RawStore, Configurable {
           pm.deletePersistentAll(roleMap);
         }
         List<MRoleMap> 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<MGlobalPrivilege> userGrants = listPrincipalGlobalGrants(
             mRol.getRoleName(), PrincipalType.ROLE);
@@ -3381,30 +3434,36 @@ public class ObjectStore implements RawStore, Configurable {
           pm.deletePersistentAll(userGrants);
         }
         List<MDBPrivilege> dbGrants = listPrincipalAllDBGrant(mRol
-            .getRoleName(), PrincipalType.ROLE);
+            .getRoleName(), PrincipalType.ROLE, queryWrapper);
         if (dbGrants.size() > 0) {
           pm.deletePersistentAll(dbGrants);
         }
+        queryWrapper.close();
         List<MTablePrivilege> tabPartGrants = listPrincipalAllTableGrants(
-            mRol.getRoleName(), PrincipalType.ROLE);
+            mRol.getRoleName(), PrincipalType.ROLE, queryWrapper);
         if (tabPartGrants.size() > 0) {
           pm.deletePersistentAll(tabPartGrants);
         }
+        queryWrapper.close();
         List<MPartitionPrivilege> partGrants = listPrincipalAllPartitionGrants(
-            mRol.getRoleName(), PrincipalType.ROLE);
+            mRol.getRoleName(), PrincipalType.ROLE, queryWrapper);
         if (partGrants.size() > 0) {
           pm.deletePersistentAll(partGrants);
         }
+        queryWrapper.close();
         List<MTableColumnPrivilege> tblColumnGrants = listPrincipalAllTableColumnGrants(
-            mRol.getRoleName(), PrincipalType.ROLE);
+            mRol.getRoleName(), PrincipalType.ROLE, queryWrapper);
         if (tblColumnGrants.size() > 0) {
           pm.deletePersistentAll(tblColumnGrants);
         }
+        queryWrapper.close();
         List<MPartitionColumnPrivilege> 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);
       }
@@ -3413,6 +3472,8 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+
+      queryWrapper.close();
     }
     return success;
   }
@@ -3461,66 +3522,62 @@ public class ObjectStore implements RawStore, Configurable {
 
   @SuppressWarnings("unchecked")
   @Override
-  public List<MRoleMap> listRoles(String principalName,
-      PrincipalType principalType) {
+  public List<MRoleMap> listRoles(String principalName, PrincipalType principalType) {
     boolean success = false;
-    List<MRoleMap> mRoleMember = null;
+    Query query = null;
+    List<MRoleMap> mRoleMember = new ArrayList<MRoleMap>();
+
     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<MRoleMap>) query.executeWithArray(
-          principalName, principalType.toString());
-      LOG.debug("Done executing query for listMSecurityUserRoleMap");
-      pm.retrieveAll(mRoleMember);
+      List<MRoleMap> mRoles =
+          (List<MRoleMap>) 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<MRoleMap>();
-      } else {
-        mRoleMember = new ArrayList<MRoleMap>(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<MRoleMap> listMSecurityPrincipalMembershipRole(final String roleName,
-      final PrincipalType principalType) {
+      final PrincipalType principalType,
+      QueryWrapper queryWrapper) {
     boolean success = false;
     List<MRoleMap> 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<MRoleMap>) 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();
@@ -3543,9 +3600,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);
@@ -3555,6 +3613,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!commited) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
     return mrole;
   }
@@ -3562,13 +3623,14 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public List<String> 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<String> roleNames  = new ArrayList<String>();
+      List<String> roleNames = new ArrayList<String>();
       for (Iterator i = names.iterator(); i.hasNext();) {
         roleNames.add((String) i.next());
       }
@@ -3578,6 +3640,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
   }
 
@@ -4386,49 +4451,61 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public List<MRoleMap> listRoleMembers(String roleName) {
     boolean success = false;
-    List<MRoleMap> mRoleMemeberList = null;
+    Query query = null;
+    List<MRoleMap> mRoleMemeberList = new ArrayList<MRoleMap>();
     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<MRoleMap>) query.execute(
-          roleName);
-      LOG.debug("Done executing query for listMSecurityUserRoleMember");
-      pm.retrieveAll(mRoleMemeberList);
+      List<MRoleMap> mRoles = (List<MRoleMap>) 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<MGlobalPrivilege> listPrincipalGlobalGrants(String principalName, PrincipalType principalType) {
+  public List<MGlobalPrivilege> listPrincipalGlobalGrants(String principalName,
+      PrincipalType principalType) {
     boolean commited = false;
-    List<MGlobalPrivilege> userNameDbPriv = null;
+    Query query = null;
+    List<MGlobalPrivilege> userNameDbPriv = new ArrayList<MGlobalPrivilege>();
     try {
+      List<MGlobalPrivilege> 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<MGlobalPrivilege>) 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<MGlobalPrivilege>) 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;
   }
@@ -4436,9 +4513,10 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public List<HiveObjectPrivilege> listGlobalGrantsAll() {
     boolean commited = false;
+    Query query = null;
     try {
       openTransaction();
-      Query query = pm.newQuery(MGlobalPrivilege.class);
+      query = pm.newQuery(MGlobalPrivilege.class);
       List<MGlobalPrivilege> userNameDbPriv = (List<MGlobalPrivilege>) query.execute();
       pm.retrieveAll(userNameDbPriv);
       commited = commitTransaction();
@@ -4447,6 +4525,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!commited) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
   }
 
@@ -4470,25 +4551,32 @@ public class ObjectStore implements RawStore, Configurable {
   public List<MDBPrivilege> listPrincipalDBGrants(String principalName,
       PrincipalType principalType, String dbName) {
     boolean success = false;
-    List<MDBPrivilege> mSecurityDBList = null;
+    Query query = null;
+    List<MDBPrivilege> mSecurityDBList = new ArrayList<MDBPrivilege>();
     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<MDBPrivilege>) 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<MDBPrivilege> mPrivs =
+          (List<MDBPrivilege>) 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;
   }
@@ -4496,12 +4584,22 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public List<HiveObjectPrivilege> 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<HiveObjectPrivilege> listDBGrantsAll(String dbName) {
-    return convertDB(listDatabaseGrants(dbName));
+    QueryWrapper queryWrapper = new QueryWrapper();
+    try {
+      return convertDB(listDatabaseGrants(dbName, queryWrapper));
+      } finally {
+        queryWrapper.close();
+      }
   }
 
   private List<HiveObjectPrivilege> convertDB(List<MDBPrivilege> privs) {
@@ -4522,26 +4620,28 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @SuppressWarnings("unchecked")
-  private List<MDBPrivilege> listPrincipalAllDBGrant(
-      String principalName, PrincipalType principalType) {
+  private List<MDBPrivilege> listPrincipalAllDBGrant(String principalName,
+      PrincipalType principalType,
+      QueryWrapper queryWrapper) {
     boolean success = false;
+    Query query = null;
     List<MDBPrivilege> 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<MDBPrivilege>) 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<MDBPrivilege>) query.execute(principalName, principalType.toString());
       } else {
-        Query query = pm.newQuery(MDBPrivilege.class);
+        query = queryWrapper.query = pm.newQuery(MDBPrivilege.class);
         mSecurityDBList = (List<MDBPrivilege>) 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) {
@@ -4552,91 +4652,101 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @SuppressWarnings("unchecked")
-  public List<MTablePrivilege> listAllTableGrants(String dbName,
-      String tableName) {
+  public List<MTablePrivilege> listAllTableGrants(String dbName, String tableName) {
     boolean success = false;
+    Query query = null;
     tableName = HiveStringUtils.normalizeIdentifier(tableName);
     dbName = HiveStringUtils.normalizeIdentifier(dbName);
-    List<MTablePrivilege> mSecurityTabList = null;
+    List<MTablePrivilege> mSecurityTabList = new ArrayList<MTablePrivilege>();
     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<MTablePrivilege>) query
-          .executeWithArray(tableName, dbName);
+      query = pm.newQuery(MTablePrivilege.class, queryStr);
+      query.declareParameters("java.lang.String t1, java.lang.String t2");
+      List<MTablePrivilege> mPrivs  = (List<MTablePrivilege>) 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<MPartitionPrivilege> listTableAllPartitionGrants(String dbName,
-      String tableName) {
+  public List<MPartitionPrivilege> listTableAllPartitionGrants(String dbName, String tableName) {
     tableName = HiveStringUtils.normalizeIdentifier(tableName);
     dbName = HiveStringUtils.normalizeIdentifier(dbName);
     boolean success = false;
-    List<MPartitionPrivilege> mSecurityTabPartList = null;
+    Query query = null;
+    List<MPartitionPrivilege> mSecurityTabPartList = new ArrayList<MPartitionPrivilege>();
     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<MPartitionPrivilege>) 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<MPartitionPrivilege> mPrivs = (List<MPartitionPrivilege>) 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<MTableColumnPrivilege> listTableAllColumnGrants(String dbName,
-      String tableName) {
+  public List<MTableColumnPrivilege> listTableAllColumnGrants(String dbName, String tableName) {
     boolean success = false;
-    List<MTableColumnPrivilege> mTblColPrivilegeList = null;
+    Query query = null;
+    List<MTableColumnPrivilege> mTblColPrivilegeList = new ArrayList<MTableColumnPrivilege>();
     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<MTableColumnPrivilege>) query
-          .executeWithArray(tableName, dbName);
-      LOG.debug("Done executing query for listTableAllColumnGrants");
-      pm.retrieveAll(mTblColPrivilegeList);
+      List<MTableColumnPrivilege> mPrivs =
+          (List<MTableColumnPrivilege>) 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;
   }
@@ -4645,26 +4755,32 @@ public class ObjectStore implements RawStore, Configurable {
   public List<MPartitionColumnPrivilege> listTableAllPartitionColumnGrants(String dbName,
       String tableName) {
     boolean success = false;
+    Query query = null;
     tableName = HiveStringUtils.normalizeIdentifier(tableName);
     dbName = HiveStringUtils.normalizeIdentifier(dbName);
-
-    List<MPartitionColumnPrivilege> mSecurityColList = null;
+    List<MPartitionColumnPrivilege> mSecurityColList = new ArrayList<MPartitionColumnPrivilege>();
     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<MPartitionColumnPrivilege>) query
-          .executeWithArray(tableName, dbName);
-      LOG.debug("Done executing query for listTableAllPartitionColumnGrants");
-      pm.retrieveAll(mSecurityColList);
+      List<MPartitionColumnPrivilege> mPrivs =
+          (List<MPartitionColumnPrivilege>) 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;
   }
@@ -4704,19 +4820,17 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @SuppressWarnings("unchecked")
-  private List<MDBPrivilege> listDatabaseGrants(String dbName) {
+  private List<MDBPrivilege> 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<MDBPrivilege> mSecurityDBList = (List<MDBPrivilege>) query
-          .executeWithArray(dbName);
-      LOG.debug("Done executing query for listDatabaseGrants");
+      List<MDBPrivilege> mSecurityDBList = (List<MDBPrivilege>) query.executeWithArray(dbName);
       pm.retrieveAll(mSecurityDBList);
       success = commitTransaction();
       LOG.debug("Done retrieving all objects for listDatabaseGrants");
@@ -4792,162 +4906,181 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   @SuppressWarnings("unchecked")
-  public List<MTablePrivilege> listAllTableGrants(
-      String principalName, PrincipalType principalType, String dbName,
-      String tableName) {
+  public List<MTablePrivilege> listAllTableGrants(String principalName,
+      PrincipalType principalType, String dbName, String tableName) {
     tableName = HiveStringUtils.normalizeIdentifier(tableName);
     dbName = HiveStringUtils.normalizeIdentifier(dbName);
-
     boolean success = false;
-    List<MTablePrivilege> mSecurityTabPartList = null;
+    Query query = null;
+    List<MTablePrivilege> mSecurityTabPartList = new ArrayList<MTablePrivilege>();
     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<MTablePrivilege>) 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<MTablePrivilege> mPrivs =
+          (List<MTablePrivilege>) 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<MPartitionPrivilege> listPrincipalPartitionGrants(
-      String principalName, PrincipalType principalType, String dbName,
-      String tableName, String partName) {
+  public List<MPartitionPrivilege> 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<MPartitionPrivilege> mSecurityTabPartList = null;
+    List<MPartitionPrivilege> mSecurityTabPartList = new ArrayList<MPartitionPrivilege>();
     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<MPartitionPrivilege>) 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<MPartitionPrivilege> mPrivs =
+          (List<MPartitionPrivilege>) 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<MTableColumnPrivilege> listPrincipalTableColumnGrants(
-      String principalName, PrincipalType principalType, String dbName,
-      String tableName, String columnName) {
+  public List<MTableColumnPrivilege> 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<MTableColumnPrivilege> mSecurityColList = null;
+    List<MTableColumnPrivilege> mSecurityColList = new ArrayList<MTableColumnPrivilege>();
     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<MTableColumnPrivilege>) 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<MTableColumnPrivilege> mPrivs =
+          (List<MTableColumnPrivilege>) 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<MPartitionColumnPrivilege> listPrincipalPartitionColumnGrants(
-      String principalName, PrincipalType principalType, String dbName,
-      String tableName, String partitionName, String columnName) {
+  public List<MPartitionColumnPrivilege> 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<MPartitionColumnPrivilege> mSecurityColList = null;
+    List<MPartitionColumnPrivilege> mSecurityColList = new ArrayList<MPartitionColumnPrivilege>();
     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<MPartitionColumnPrivilege> mPrivs =
+          (List<MPartitionColumnPrivilege>) query.executeWithArray(principalName,
+              principalType.toString(), tableName, dbName, partitionName, columnName);
+      pm.retrieveAll(mPrivs);
+      success = commitTransaction();
 
-      mSecurityColList = (List<MPartitionColumnPrivilege>) 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<HiveObjectPrivilege> listPrincipalPartitionColumnGrantsAll(
-      String principalName, PrincipalType principalType) {
+  public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrantsAll(String principalName,
+      PrincipalType principalType) {
     boolean success = false;
+    Query query = null;
     try {
       openTransaction();
       LOG.debug("Executing listPrincipalPartitionColumnGrantsAll");
       List<MPartitionColumnPrivilege> 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<MPartitionColumnPrivilege>)
-            query.executeWithArray(principalName, principalType.toString());
+        mSecurityTabPartList =
+            (List<MPartitionColumnPrivilege>) query.executeWithArray(principalName,
+                principalType.toString());
       } else {
-        Query query = pm.newQuery(MPartitionColumnPrivilege.class);
+        query = pm.newQuery(MPartitionColumnPrivilege.class);
         mSecurityTabPartList = (List<MPartitionColumnPrivilege>) query.execute();
       }
       LOG.debug("Done executing query for listPrincipalPartitionColumnGrantsAll");
@@ -4960,23 +5093,29 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPartitionColumnGrantsAll(
-      String dbName, String tableName, String partitionName, String columnName) {
+  public List<HiveObjectPrivilege> 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<MPartitionColumnPrivilege> mSecurityTabPartList = (List<MPartitionColumnPrivilege>)
-          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<MPartitionColumnPrivilege> mSecurityTabPartList =
+          (List<MPartitionColumnPrivilege>) query.executeWithArray(tableName, dbName,
+              partitionName, columnName);
       LOG.debug("Done executing query for listPartitionColumnGrantsAll");
       pm.retrieveAll(mSecurityTabPartList);
       List<HiveObjectPrivilege> result = convertPartCols(mSecurityTabPartList);
@@ -4987,6 +5126,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
   }
 
@@ -5012,23 +5154,22 @@ public class ObjectStore implements RawStore, Configurable {
 
   @SuppressWarnings("unchecked")
   private List<MTablePrivilege> listPrincipalAllTableGrants(
-      String principalName, PrincipalType principalType) {
+      String principalName, PrincipalType principalType, QueryWrapper queryWrapper) {
     boolean success = false;
     List<MTablePrivilege> 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<MTablePrivilege>) 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();
@@ -5038,21 +5179,21 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPrincipalTableGrantsAll(
-      String principalName, PrincipalType principalType) {
+  public List<HiveObjectPrivilege> listPrincipalTableGrantsAll(String principalName,
+      PrincipalType principalType) {
     boolean success = false;
+    Query query = null;
     try {
       openTransaction();
       LOG.debug("Executing listPrincipalAllTableGrants");
       List<MTablePrivilege> 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<MTablePrivilege>) query.execute(
-            principalName, principalType.toString());
+        mSecurityTabPartList =
+            (List<MTablePrivilege>) query.execute(principalName, principalType.toString());
       } else {
-        Query query = pm.newQuery(MTablePrivilege.class);
+        query = pm.newQuery(MTablePrivilege.class);
         mSecurityTabPartList = (List<MTablePrivilege>) query.execute();
       }
       LOG.debug("Done executing query for listPrincipalAllTableGrants");
@@ -5065,20 +5206,24 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
   }
 
   @Override
   public List<HiveObjectPrivilege> 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<MTablePrivilege> mSecurityTabPartList = (List<MTablePrivilege>)
-          query.executeWithArray(tableName, dbName);
+      List<MTablePrivilege> mSecurityTabPartList =
+          (List<MTablePrivilege>) query.executeWithArray(tableName, dbName);
       LOG.debug("Done executing query for listTableGrantsAll");
       pm.retrieveAll(mSecurityTabPartList);
       List<HiveObjectPrivilege> result = convertTable(mSecurityTabPartList);
@@ -5089,6 +5234,9 @@ public class ObjectStore implements RawStore, Configurable {
       if (!success) {
         rollbackTransaction();
       }
+      if (query != null) {
+        query.closeAll();
+      }
     }
   }
 
@@ -5112,24 +5260,20 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @SuppressWarnings("unchecked")
-  private List<MPartitionPrivilege> listPrincipalAllPartitionGrants(
-      String principalName, PrincipalType principalType) {
+  private List<MPartitionPrivilege> listPrincipalAllPartitionGrants(String principalName,
+      PrincipalType principalType, QueryWrapper queryWrapper) {
     boolean success = false;
     List<MPartitionPrivilege> 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<MPartitionPrivilege>) query.execute(
-          principalName, principalType.toString());
-      LOG
-          .debug("Done executing query for listPrincipalAllPartitionGrants");
+      mSecurityTabPartList =
+          (List<MPartitionPrivilege>) 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();
@@ -5139,21 +5283,22 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPrincipalPartitionGrantsAll(
-      String principalName, PrincipalType principalType) {
+  public List<HiveObjectPrivilege> listPrincipalPartitionGrantsAll(String principalName,
+      PrincipalType principalType) {
     boolean success = false;
+    Query query = null;
     try {
       openTransaction();
       LOG.deb

<TRUNCATED>