You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2017/11/09 17:46:32 UTC

hive git commit: HIVE-17911 : org.apache.hadoop.hive.metastore.ObjectStore - Tune Up (Beluga Behr via Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master 0e54f51a3 -> 82cb3d57a


HIVE-17911 : org.apache.hadoop.hive.metastore.ObjectStore - Tune Up (Beluga Behr via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master
Commit: 82cb3d57abf2705069d82807e52177fdb41ff5ca
Parents: 0e54f51
Author: BELUGA BEHR <da...@gmail.com>
Authored: Thu Nov 9 09:45:57 2017 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Thu Nov 9 09:45:57 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hive/metastore/ObjectStore.java      | 369 +++++++++----------
 1 file changed, 172 insertions(+), 197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/82cb3d57/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 188135c..9e9239b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -74,6 +74,8 @@ import javax.jdo.datastore.JDOConnection;
 import javax.jdo.identity.IntIdentity;
 import javax.sql.DataSource;
 
+
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -224,18 +226,18 @@ public class ObjectStore implements RawStore, Configurable {
   * Verify the schema only once per JVM since the db connection info is static
   */
   private final static AtomicBoolean isSchemaVerified = new AtomicBoolean(false);
-  private static final Logger LOG = LoggerFactory.getLogger(ObjectStore.class.getName());
+  private static final Logger LOG = LoggerFactory.getLogger(ObjectStore.class);
 
   private enum TXN_STATUS {
     NO_STATE, OPEN, COMMITED, ROLLBACK
   }
 
-  private static final Map<String, Class> PINCLASSMAP;
+  private static final Map<String, Class<?>> PINCLASSMAP;
   private static final String HOSTNAME;
   private static final String USER;
   private static final String JDO_PARAM = ":param";
   static {
-    Map<String, Class> map = new HashMap<>();
+    Map<String, Class<?>> map = new HashMap<>();
     map.put("table", MTable.class);
     map.put("storagedescriptor", MStorageDescriptor.class);
     map.put("serdeinfo", MSerDeInfo.class);
@@ -253,11 +255,7 @@ public class ObjectStore implements RawStore, Configurable {
     }
     HOSTNAME = hostname;
     String user = System.getenv("USER");
-    if (user == null) {
-      USER = "UNKNOWN";
-    } else {
-      USER = user;
-    }
+    USER = org.apache.commons.lang.StringUtils.defaultString(user, "UNKNOWN");
   }
 
 
@@ -394,13 +392,13 @@ public class ObjectStore implements RawStore, Configurable {
         numTries--;
         boolean retriable = isRetriableException(e);
         if ((numTries > 0) && retriable){
-          LOG.info("Retriable exception while instantiating ObjectStore, retrying. "
-              + numTries + " tries left", e);
+          LOG.info("Retriable exception while instantiating ObjectStore, retrying. " +
+              "{} tries left", numTries, e);
           try {
             Thread.sleep(retryInterval);
           } catch (InterruptedException ie) {
             // Restore the interrupted status, since we do not want to catch it.
-            LOG.debug("Interrupted while sleeping before retrying.",ie);
+            LOG.debug("Interrupted while sleeping before retrying.", ie);
             Thread.currentThread().interrupt();
           }
           // If we're here, we'll proceed down the next while loop iteration.
@@ -465,14 +463,12 @@ public class ObjectStore implements RawStore, Configurable {
       expressionProxy = createExpressionProxy(conf);
       if (MetastoreConf.getBoolVar(getConf(), ConfVars.TRY_DIRECT_SQL)) {
         String schema = prop.getProperty("javax.jdo.mapping.Schema");
-        if (schema != null && schema.isEmpty()) {
-          schema = null;
-        }
+        schema = org.apache.commons.lang.StringUtils.defaultIfBlank(schema, null);
         directSql = new MetaStoreDirectSql(pm, conf, schema);
       }
     }
-    LOG.debug("RawStore: " + this + ", with PersistenceManager: " + pm +
-        " created in the thread with id: " + Thread.currentThread().getId());
+    LOG.debug("RawStore: {}, with PersistenceManager: {}" +
+        " created in the thread with id: {}", this, pm, Thread.currentThread().getId());
   }
 
   private DatabaseProduct determineDatabaseProduct() {
@@ -506,7 +502,6 @@ public class ObjectStore implements RawStore, Configurable {
   private static PartitionExpressionProxy createExpressionProxy(Configuration conf) {
     String className = MetastoreConf.getVar(conf, ConfVars.EXPRESSION_PROXY_CLASS);
     try {
-      @SuppressWarnings("unchecked")
       Class<? extends PartitionExpressionProxy> clazz =
            JavaUtils.getClass(className, PartitionExpressionProxy.class);
       return JavaUtils.newInstance(clazz, new Class<?>[0], new Object[0]);
@@ -530,7 +525,7 @@ public class ObjectStore implements RawStore, Configurable {
         if (pair != null && pair.length == 2) {
           System.setProperty(pair[0].trim(), pair[1].trim());
         } else {
-          LOG.warn("Invalid metastore property value for " + ConfVars.DBACCESS_SSL_PROPS);
+          LOG.warn("Invalid metastore property value for {}", ConfVars.DBACCESS_SSL_PROPS);
         }
       }
     }
@@ -549,10 +544,11 @@ public class ObjectStore implements RawStore, Configurable {
     // has to be a separate first step because we don't set the default values in the config object.
     for (ConfVars var : MetastoreConf.dataNucleusAndJdoConfs) {
       String confVal = MetastoreConf.getAsString(conf, var);
-      Object prevVal = prop.setProperty(var.getVarname(), confVal);
-      if (LOG.isDebugEnabled() && MetastoreConf.isPrintable(var.getVarname())) {
-        LOG.debug("Overriding " + var.getVarname() + " value " + prevVal
-            + " from  jpox.properties with " + confVal);
+      String varName = var.getVarname();
+      Object prevVal = prop.setProperty(varName, confVal);
+      if (MetastoreConf.isPrintable(varName)) {
+        LOG.debug("Overriding {} value {} from jpox.properties with {}",
+          varName, prevVal, confVal);
       }
     }
 
@@ -577,7 +573,7 @@ public class ObjectStore implements RawStore, Configurable {
     // Password may no longer be in the conf, use getPassword()
     try {
       String passwd = MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.PWD);
-      if (passwd != null && !passwd.isEmpty()) {
+      if (org.apache.commons.lang.StringUtils.isNotEmpty(passwd)) {
         // We can get away with the use of varname here because varname == hiveName for PWD
         prop.setProperty(ConfVars.PWD.getVarname(), passwd);
       }
@@ -588,7 +584,7 @@ public class ObjectStore implements RawStore, Configurable {
     if (LOG.isDebugEnabled()) {
       for (Entry<Object, Object> e : prop.entrySet()) {
         if (MetastoreConf.isPrintable(e.getKey().toString())) {
-          LOG.debug(e.getKey() + " = " + e.getValue());
+          LOG.debug("{} = {}", e.getKey(), e.getValue());
         }
       }
     }
@@ -607,9 +603,9 @@ public class ObjectStore implements RawStore, Configurable {
     final String autoStartKey = "datanucleus.autoStartMechanismMode";
     final String autoStartIgnore = "ignored";
     String currentAutoStartVal = conf.get(autoStartKey);
-    if(currentAutoStartVal != null && !currentAutoStartVal.equalsIgnoreCase(autoStartIgnore)) {
-      LOG.warn(autoStartKey + " is set to unsupported value " + conf.get(autoStartKey) +
-          " . Setting it to value " + autoStartIgnore);
+    if (!autoStartIgnore.equalsIgnoreCase(currentAutoStartVal)) {
+      LOG.warn("{} is set to unsupported value {} . Setting it to value: {}", autoStartKey,
+        conf.get(autoStartKey), autoStartIgnore);
     }
     conf.set(autoStartKey, autoStartIgnore);
   }
@@ -640,17 +636,16 @@ public class ObjectStore implements RawStore, Configurable {
       DataStoreCache dsc = pmf.getDataStoreCache();
       if (dsc != null) {
         String objTypes = MetastoreConf.getVar(conf, ConfVars.CACHE_PINOBJTYPES);
-        LOG.info("Setting MetaStore object pin classes with hive.metastore.cache.pinobjtypes=\"" + objTypes + "\"");
-        if (objTypes != null && objTypes.length() > 0) {
-          objTypes = objTypes.toLowerCase();
-          String[] typeTokens = objTypes.split(",");
+        LOG.info("Setting MetaStore object pin classes with hive.metastore.cache.pinobjtypes=\"{}\"", objTypes);
+        if (org.apache.commons.lang.StringUtils.isNotEmpty(objTypes)) {
+          String[] typeTokens = objTypes.toLowerCase().split(",");
           for (String type : typeTokens) {
             type = type.trim();
             if (PINCLASSMAP.containsKey(type)) {
               dsc.pinAll(true, PINCLASSMAP.get(type));
-            }
-            else {
-              LOG.warn(type + " is not one of the pinnable object types: " + org.apache.commons.lang.StringUtils.join(PINCLASSMAP.keySet(), " "));
+            } else {
+              LOG.warn("{} is not one of the pinnable object types: {}", type,
+                org.apache.commons.lang.StringUtils.join(PINCLASSMAP.keySet(), " "));
             }
           }
         }
@@ -669,9 +664,8 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   public void shutdown() {
+    LOG.debug("RawStore: {}, with PersistenceManager: {} will be shutdown", this, pm);
     if (pm != null) {
-      LOG.debug("RawStore: " + this + ", with PersistenceManager: " + pm +
-          " will be shutdown");
       pm.close();
       pm = null;
     }
@@ -838,7 +832,7 @@ public class ObjectStore implements RawStore, Configurable {
       ex = e;
     }
     if (db == null) {
-      LOG.warn("Failed to get database " + name +", returning NoSuchObjectException", ex);
+      LOG.warn("Failed to get database {}, returning NoSuchObjectException", name, ex);
       throw new NoSuchObjectException(name + (ex == null ? "" : (": " + ex.getMessage())));
     }
     return db;
@@ -876,8 +870,9 @@ public class ObjectStore implements RawStore, Configurable {
     db.setLocationUri(mdb.getLocationUri());
     db.setParameters(convertMap(mdb.getParameters()));
     db.setOwnerName(mdb.getOwnerName());
-    String type = mdb.getOwnerType();
-    db.setOwnerType((null == type || type.trim().isEmpty()) ? null : PrincipalType.valueOf(type));
+    String type = org.apache.commons.lang.StringUtils.defaultIfBlank(mdb.getOwnerType(), null);
+    PrincipalType principalType = (type == null) ? null : PrincipalType.valueOf(type);
+    db.setOwnerType(principalType);
     return db;
   }
 
@@ -923,7 +918,7 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public boolean dropDatabase(String dbname) throws NoSuchObjectException, MetaException {
     boolean success = false;
-    LOG.info("Dropping database " + dbname + " along with all tables");
+    LOG.info("Dropping database {} along with all tables", dbname);
     dbname = normalizeIdentifier(dbname);
     QueryWrapper queryWrapper = new QueryWrapper();
     try {
@@ -934,7 +929,7 @@ public class ObjectStore implements RawStore, Configurable {
       pm.retrieve(db);
       if (db != null) {
         List<MDBPrivilege> dbGrants = this.listDatabaseGrants(dbname, queryWrapper);
-        if (dbGrants != null && dbGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(dbGrants)) {
           pm.deletePersistentAll(dbGrants);
         }
         pm.deletePersistent(db);
@@ -965,11 +960,8 @@ public class ObjectStore implements RawStore, Configurable {
       query = pm.newQuery(MDatabase.class, filterBuilder.toString());
       query.setResult("name");
       query.setOrdering("name ascending");
-      Collection names = (Collection) query.executeWithArray(parameterVals.toArray(new String[parameterVals.size()]));
-      databases = new ArrayList<>();
-      for (Iterator i = names.iterator(); i.hasNext();) {
-        databases.add((String) i.next());
-      }
+      Collection<String> names = (Collection<String>) query.executeWithArray(parameterVals.toArray(new String[0]));
+      databases = new ArrayList<>(names);
       commited = commitTransaction();
     } finally {
       rollbackAndCleanup(commited, query);
@@ -1082,7 +1074,7 @@ public class ObjectStore implements RawStore, Configurable {
       success = commitTransaction();
     } catch (JDOObjectNotFoundException e) {
       success = commitTransaction();
-      LOG.debug("type not found " + typeName, e);
+      LOG.debug("type not found {}", typeName, e);
     } finally {
       rollbackAndCleanup(success, query);
     }
@@ -1190,36 +1182,36 @@ public class ObjectStore implements RawStore, Configurable {
       if (tbl != null) {
         // first remove all the grants
         List<MTablePrivilege> tabGrants = listAllTableGrants(dbName, tableName);
-        if (tabGrants != null && tabGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(tabGrants)) {
           pm.deletePersistentAll(tabGrants);
         }
         List<MTableColumnPrivilege> tblColGrants = listTableAllColumnGrants(dbName,
             tableName);
-        if (tblColGrants != null && tblColGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(tblColGrants)) {
           pm.deletePersistentAll(tblColGrants);
         }
 
         List<MPartitionPrivilege> partGrants = this.listTableAllPartitionGrants(dbName, tableName);
-        if (partGrants != null && partGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(partGrants)) {
           pm.deletePersistentAll(partGrants);
         }
 
         List<MPartitionColumnPrivilege> partColGrants = listTableAllPartitionColumnGrants(dbName,
             tableName);
-        if (partColGrants != null && partColGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(partColGrants)) {
           pm.deletePersistentAll(partColGrants);
         }
         // delete column statistics if present
         try {
           deleteTableColumnStatistics(dbName, tableName, null);
         } catch (NoSuchObjectException e) {
-          LOG.info("Found no table level column statistics associated with db " + dbName +
-          " table " + tableName + " record to delete");
+          LOG.info("Found no table level column statistics associated with db {}" +
+          " table {} record to delete", dbName, tableName);
         }
 
         List<MConstraint> tabConstraints = listAllTableConstraintsWithOptionalConstraintName(
                                            dbName, tableName, null);
-        if (tabConstraints != null && tabConstraints.size() > 0) {
+        if (CollectionUtils.isNotEmpty(tabConstraints)) {
           pm.deletePersistentAll(tabConstraints);
         }
 
@@ -1326,11 +1318,8 @@ public class ObjectStore implements RawStore, Configurable {
       query = pm.newQuery(MTable.class, filterBuilder.toString());
       query.setResult("tableName");
       query.setOrdering("tableName ascending");
-      Collection names = (Collection) query.executeWithArray(parameterVals.toArray(new String[parameterVals.size()]));
-      tbls = new ArrayList<>();
-      for (Iterator i = names.iterator(); i.hasNext();) {
-        tbls.add((String) i.next());
-      }
+      Collection<String> names = (Collection<String>) query.executeWithArray(parameterVals.toArray(new String[0]));
+      tbls = new ArrayList<>(names);
       commited = commitTransaction();
     } finally {
       rollbackAndCleanup(commited, query);
@@ -1515,11 +1504,11 @@ public class ObjectStore implements RawStore, Configurable {
       dbExistsQuery.setUnique(true);
       dbExistsQuery.setResult("name");
       String dbNameIfExists = (String) dbExistsQuery.execute(db);
-      if (dbNameIfExists == null || dbNameIfExists.isEmpty()) {
+      if (org.apache.commons.lang.StringUtils.isEmpty(dbNameIfExists)) {
         throw new UnknownDBException("Could not find database " + db);
       }
 
-      List<String> lowered_tbl_names = new ArrayList<>();
+      List<String> lowered_tbl_names = new ArrayList<>(tbl_names.size());
       for (String t : tbl_names) {
         lowered_tbl_names.add(normalizeIdentifier(t));
       }
@@ -1584,7 +1573,7 @@ public class ObjectStore implements RawStore, Configurable {
     try {
       mdb = getMDatabase(tbl.getDbName());
     } catch (NoSuchObjectException e) {
-      LOG.error(StringUtils.stringifyException(e));
+      LOG.error("Could not convert to MTable", e);
       throw new InvalidObjectException("Database " + tbl.getDbName()
           + " doesn't exist.");
     }
@@ -1857,7 +1846,7 @@ public class ObjectStore implements RawStore, Configurable {
           }
         }
       }
-      if (toPersist.size() > 0) {
+      if (CollectionUtils.isNotEmpty(toPersist)) {
         pm.makePersistentAll(toPersist);
         pm.flush();
       }
@@ -1980,7 +1969,7 @@ public class ObjectStore implements RawStore, Configurable {
           toPersist.add(partColumn);
         }
 
-        if (toPersist.size() > 0) {
+        if (CollectionUtils.isNotEmpty(toPersist)) {
           pm.makePersistentAll(toPersist);
         }
       }
@@ -2038,7 +2027,7 @@ public class ObjectStore implements RawStore, Configurable {
       // We need to compare partition name with requested name since some DBs
       // (like MySQL, Derby) considers 'a' = 'a ' whereas others like (Postgres,
       // Oracle) doesn't exhibit this problem.
-      if (mparts != null && mparts.size() > 0) {
+      if (CollectionUtils.isNotEmpty(mparts)) {
         if (mparts.size() > 1) {
           throw new MetaException(
               "Expecting only one partition but more than one partitions are found.");
@@ -2143,7 +2132,9 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public void dropPartitions(String dbName, String tblName, List<String> partNames)
       throws MetaException, NoSuchObjectException {
-    if (partNames.isEmpty()) return;
+    if (CollectionUtils.isEmpty(partNames)) {
+      return;
+    }
     boolean success = false;
     openTransaction();
     try {
@@ -2195,7 +2186,7 @@ public class ObjectStore implements RawStore, Configurable {
             part.getTable().getTableName(),
             Lists.newArrayList(partName));
 
-        if (partGrants != null && partGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(partGrants)) {
           pm.deletePersistentAll(partGrants);
         }
 
@@ -2203,7 +2194,7 @@ public class ObjectStore implements RawStore, Configurable {
             part.getTable().getDatabase().getName(),
             part.getTable().getTableName(),
             Lists.newArrayList(partName));
-        if (partColumnGrants != null && partColumnGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(partColumnGrants)) {
           pm.deletePersistentAll(partColumnGrants);
         }
 
@@ -2268,7 +2259,7 @@ public class ObjectStore implements RawStore, Configurable {
       openTransaction();
       List<MPartition> mparts = listMPartitions(dbName, tblName, max, queryWrapper);
       List<Partition> parts = new ArrayList<>(mparts.size());
-      if (mparts != null && mparts.size()>0) {
+      if (CollectionUtils.isNotEmpty(mparts)) {
         for (MPartition mpart : mparts) {
           MTable mtbl = mpart.getTable();
           Partition part = convertToPart(mpart);
@@ -2421,13 +2412,13 @@ public class ObjectStore implements RawStore, Configurable {
       if (filter == null || filter.isEmpty()) {
         PartitionValuesResponse response =
             getDistinctValuesForPartitionsNoTxn(dbName, tableName, cols, applyDistinct, ascending, maxParts);
-        LOG.info("Number of records fetched: " + response.getPartitionValues().size());
+        LOG.info("Number of records fetched: {}", response.getPartitionValues().size());
         return response;
       } else {
         PartitionValuesResponse response =
             extractPartitionNamesByFilter(dbName, tableName, filter, cols, ascending, applyDistinct, maxParts);
         if (response != null && response.getPartitionValues() != null) {
-          LOG.info("Number of records fetched with filter: " + response.getPartitionValues().size());
+          LOG.info("Number of records fetched with filter: {}", response.getPartitionValues().size());
         }
         return response;
       }
@@ -2442,8 +2433,7 @@ public class ObjectStore implements RawStore, Configurable {
                                                                 List<FieldSchema> cols, boolean ascending, boolean applyDistinct, long maxParts)
       throws MetaException, NoSuchObjectException {
 
-    LOG.info("Database: " + dbName + " Table:" + tableName + " filter\"" + filter + "\" cols:" + cols);
-    List<String> partitionResults = new ArrayList<String>();
+    LOG.info("Database: {} Table: {} filter: \"{}\" cols: {}", dbName, tableName, filter, cols);
     List<String> partitionNames = null;
     List<Partition> partitions = null;
     Table tbl = getTable(dbName, tableName);
@@ -2451,7 +2441,7 @@ public class ObjectStore implements RawStore, Configurable {
       // Get partitions by name - ascending or descending
       partitionNames = getPartitionNamesByFilter(dbName, tableName, filter, ascending, maxParts);
     } catch (MetaException e) {
-      LOG.warn("Querying by partition names failed, trying out with partition objects, filter:" + filter);
+      LOG.warn("Querying by partition names failed, trying out with partition objects, filter: {}", filter);
     }
 
     if (partitionNames == null) {
@@ -2480,12 +2470,9 @@ public class ObjectStore implements RawStore, Configurable {
     // Return proper response
     PartitionValuesResponse response = new PartitionValuesResponse();
     response.setPartitionValues(new ArrayList<PartitionValuesRow>(partitionNames.size()));
-    LOG.info("Converting responses to Partition values for items:" + partitionNames.size());
+    LOG.info("Converting responses to Partition values for items: {}", partitionNames.size());
     for (String partName : partitionNames) {
-      ArrayList<String> vals = new ArrayList<String>(tbl.getPartitionKeys().size());
-      for (FieldSchema key : tbl.getPartitionKeys()) {
-        vals.add(null);
-      }
+      ArrayList<String> vals = new ArrayList<String>(Collections.nCopies(tbl.getPartitionKeys().size(), null));
       PartitionValuesRow row = new PartitionValuesRow();
       Warehouse.makeValsFromName(partName, vals);
       for (String value : vals) {
@@ -2525,9 +2512,10 @@ public class ObjectStore implements RawStore, Configurable {
         query.setRange(0, maxParts);
       }
 
-      LOG.debug("Filter specified is " + filter + "," +
-          " JDOQL filter is " + queryFilterString);
-      LOG.debug("Parms is " + params);
+      LOG.debug("Filter specified is {}, JDOQL filter is {}", filter,
+        queryFilterString);
+
+      LOG.debug("Parms is {}", params);
 
       String parameterDeclaration = makeParameterDeclarationStringObj(params);
       query.declareParameters(parameterDeclaration);
@@ -2538,15 +2526,12 @@ public class ObjectStore implements RawStore, Configurable {
       }
       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());
-      }
+      Collection<String> names = (Collection<String>) query.executeWithMap(params);
+      partNames = new ArrayList<String>(names);
 
       LOG.debug("Done executing query for getPartitionNamesByFilter");
       success = commitTransaction();
-      LOG.debug("Done retrieving all objects for getPartitionNamesByFilter, size:" + partNames.size());
+      LOG.debug("Done retrieving all objects for getPartitionNamesByFilter, size: {}", partNames.size());
       query.closeAll();
     } finally {
       if (!success) {
@@ -2586,7 +2571,7 @@ public class ObjectStore implements RawStore, Configurable {
         partValuesSelect.append(extractPartitionKey(key, partitionKeys)).append(", ");
       }
       partValuesSelect.setLength(partValuesSelect.length() - 2);
-      LOG.info("Columns to be selected from Partitions: " + partValuesSelect);
+      LOG.info("Columns to be selected from Partitions: {}", partValuesSelect);
       q.setResult(partValuesSelect.toString());
 
       PartitionValuesResponse response = new PartitionValuesResponse();
@@ -2628,10 +2613,8 @@ public class ObjectStore implements RawStore, Configurable {
     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());
-    }
+    Collection<String> names = (Collection<String>) query.execute(dbName, tableName);
+    pns.addAll(names);
 
     if (query != null) {
       query.closeAll();
@@ -2742,11 +2725,9 @@ public class ObjectStore implements RawStore, Configurable {
     try {
       openTransaction();
       LOG.debug("Executing listPartitionNamesPs");
-      Collection names = getPartitionPsQueryResults(dbName, tableName,
+      Collection<String> names = getPartitionPsQueryResults(dbName, tableName,
           part_vals, max_parts, "partitionName", queryWrapper);
-      for (Object o : names) {
-        partitionNames.add((String) o);
-      }
+      partitionNames.addAll(names);
       success = commitTransaction();
     } finally {
       rollbackAndCleanup(success, queryWrapper);
@@ -2773,7 +2754,7 @@ public class ObjectStore implements RawStore, Configurable {
       LOG.debug("Done executing query for listMPartitions");
       pm.retrieveAll(mparts);
       success = commitTransaction();
-      LOG.debug("Done retrieving all objects for listMPartitions " + mparts);
+      LOG.debug("Done retrieving all objects for listMPartitions {}", mparts);
     } finally {
       if (!success) {
         rollbackTransaction();
@@ -2969,7 +2950,7 @@ public class ObjectStore implements RawStore, Configurable {
     Query query = queryWithParams.getFirst();
     query.setClass(MPartition.class);
     long deleted = query.deletePersistentAll(queryWithParams.getSecond());
-    LOG.debug("Deleted " + deleted + " partition from store");
+    LOG.debug("Deleted {} partition from store", deleted);
     query.closeAll();
   }
 
@@ -3019,7 +3000,7 @@ public class ObjectStore implements RawStore, Configurable {
     sb.append(')');
     Query query = pm.newQuery();
     query.setFilter(sb.toString());
-    LOG.debug(" JDOQL filter is " + sb.toString());
+    LOG.debug(" JDOQL filter is {}", sb);
     params.put("t1", normalizeIdentifier(tblName));
     params.put("t2", normalizeIdentifier(dbName));
     query.declareParameters(makeParameterDeclarationString(params));
@@ -3205,8 +3186,11 @@ public class ObjectStore implements RawStore, Configurable {
     private T commit() {
       success = commitTransaction();
       if (doTrace) {
-        LOG.debug(describeResult() + " retrieved using " + (doUseDirectSql ? "SQL" : "ORM")
-            + " in " + ((System.nanoTime() - start) / 1000000.0) + "ms");
+        double time = ((System.nanoTime() - start) / 1000000.0);
+        String result = describeResult();
+        String retrieveType = doUseDirectSql ? "SQL" : "ORM";
+
+        LOG.debug("{} retrieved using {} in {}ms", result, retrieveType, time);
       }
       return results;
     }
@@ -3250,7 +3234,7 @@ public class ObjectStore implements RawStore, Configurable {
 
     @Override
     protected String describeResult() {
-      return "db details for db " + dbName;
+      return "db details for db ".concat(dbName);
     }
   }
 
@@ -3269,7 +3253,7 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public int getNumPartitionsByFilter(String dbName, String tblName,
                                       String filter) throws MetaException, NoSuchObjectException {
-    final ExpressionTree exprTree = (filter != null && !filter.isEmpty())
+    final ExpressionTree exprTree = org.apache.commons.lang.StringUtils.isNotEmpty(filter)
         ? PartFilterExprUtil.getFilterParser(filter).tree : ExpressionTree.EMPTY_TREE;
 
     return new GetHelper<Integer>(dbName, tblName, true, true) {
@@ -3437,11 +3421,11 @@ public class ObjectStore implements RawStore, Configurable {
     tree.generateJDOFilterFragment(getConf(), table, params, queryBuilder);
     if (queryBuilder.hasError()) {
       assert !isValidatedFilter;
-      LOG.info("JDO filter pushdown cannot be used: " + queryBuilder.getErrorMessage());
+      LOG.info("JDO filter pushdown cannot be used: {}", queryBuilder.getErrorMessage());
       return null;
     }
     String jdoFilter = queryBuilder.getFilter();
-    LOG.debug("jdoFilter = " + jdoFilter);
+    LOG.debug("jdoFilter = {}", jdoFilter);
     return jdoFilter;
   }
 
@@ -3460,7 +3444,7 @@ public class ObjectStore implements RawStore, Configurable {
     for (Entry<String, Object> entry : params.entrySet()) {
       paramDecl.append(", ");
       paramDecl.append(entry.getValue().getClass().getName());
-      paramDecl.append(" ");
+      paramDecl.append(' ');
       paramDecl.append(entry.getKey());
     }
     return paramDecl.toString();
@@ -3485,21 +3469,19 @@ public class ObjectStore implements RawStore, Configurable {
       if (maxTables >= 0) {
         query.setRange(0, maxTables);
       }
-      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("filter specified is {}, JDOQL filter is {}", filter, queryFilterString);
+      if (LOG.isDebugEnabled()) {
+        for (Entry<String, Object> entry : params.entrySet()) {
+          LOG.debug("key: {} value: {} class: {}", entry.getKey(), entry.getValue(), 
+             entry.getValue().getClass().getName());
+        }
       }
       String parameterDeclaration = makeParameterDeclarationStringObj(params);
       query.declareParameters(parameterDeclaration);
       query.setFilter(queryFilterString);
-      Collection names = (Collection)query.executeWithMap(params);
+      Collection<String> names = (Collection<String>)query.executeWithMap(params);
       // have to emulate "distinct", otherwise tables with the same name may be returned
-      Set<String> tableNamesSet = new HashSet<>();
-      for (Iterator i = names.iterator(); i.hasNext();) {
-        tableNamesSet.add((String) i.next());
-      }
-      tableNames = new ArrayList<>(tableNamesSet);
+      tableNames = new ArrayList<>(new HashSet<>(names));
       LOG.debug("Done executing query for listTableNamesByFilter");
       success = commitTransaction();
       LOG.debug("Done retrieving all objects for listTableNamesByFilter");
@@ -3535,17 +3517,14 @@ public class ObjectStore implements RawStore, Configurable {
         // 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("Parms is " + params);
+      LOG.debug("Filter specified is {}, JDOQL filter is {}", filter, 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());
-      }
+      Collection<String> names = (Collection<String>) query.executeWithMap(params);
+      partNames = new ArrayList<>(names);
       LOG.debug("Done executing query for listMPartitionNamesByFilter");
       success = commitTransaction();
       LOG.debug("Done retrieving all objects for listMPartitionNamesByFilter");
@@ -3801,8 +3780,7 @@ public class ObjectStore implements RawStore, Configurable {
     if (cols == null) {
       return null;
     }
-    for (int i = 0; i < cols.size(); i++) {
-      MFieldSchema mfs = cols.get(i);
+    for (MFieldSchema mfs : cols) {
       if (mfs.getName().equalsIgnoreCase(col)) {
         return mfs;
       }
@@ -3879,18 +3857,18 @@ public class ObjectStore implements RawStore, Configurable {
       MMetastoreDBProperties prop = new MMetastoreDBProperties();
       prop.setPropertykey("guid");
       final String guid = UUID.randomUUID().toString();
-      LOG.debug("Attempting to add a guid " + guid + " for the metastore db");
+      LOG.debug("Attempting to add a guid {} for the metastore db", guid);
       prop.setPropertyValue(guid);
       prop.setDescription("Metastore DB GUID generated on "
           + LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS")));
       pm.makePersistent(prop);
       success = commitTransaction();
       if (success) {
-        LOG.info("Metastore db guid " + guid + " created successfully");
+        LOG.info("Metastore db guid {} created successfully", guid);
         return guid;
       }
     } catch (Exception e) {
-      LOG.warn(e.getMessage(), e);
+      LOG.warn("Metastore db guid creation failed", e);
     } finally {
       rollbackAndCleanup(success, query);
     }
@@ -3915,7 +3893,7 @@ public class ObjectStore implements RawStore, Configurable {
       List<String> uuids = new ArrayList<>();
       for (Iterator<MMetastoreDBProperties> i = names.iterator(); i.hasNext();) {
         String uuid = i.next().getPropertyValue();
-        LOG.debug("Found guid " + uuid);
+        LOG.debug("Found guid {}", uuid);
         uuids.add(uuid);
       }
       success = commitTransaction();
@@ -3923,7 +3901,7 @@ public class ObjectStore implements RawStore, Configurable {
         throw new MetaException("Multiple uuids found");
       }
       if(!uuids.isEmpty()) {
-        LOG.debug("Returning guid of metastore db : " + uuids.get(0));
+        LOG.debug("Returning guid of metastore db : {}", uuids.get(0));
         return uuids.get(0);
       }
     } finally {
@@ -3938,7 +3916,7 @@ public class ObjectStore implements RawStore, Configurable {
           throws InvalidObjectException, MetaException {
     List<String> fkNames = new ArrayList<>();
 
-    if (foreignKeys.size() > 0) {
+    if (CollectionUtils.isNotEmpty(foreignKeys)) {
       List<MConstraint> mpkfks = new ArrayList<>();
       String currentConstraintName = null;
       // We start iterating through the foreign keys. This list might contain more than a single
@@ -4673,48 +4651,48 @@ public class ObjectStore implements RawStore, Configurable {
         // first remove all the membership, the membership that this role has
         // been granted
         List<MRoleMap> roleMap = listMRoleMembers(mRol.getRoleName());
-        if (roleMap.size() > 0) {
+        if (CollectionUtils.isNotEmpty(roleMap)) {
           pm.deletePersistentAll(roleMap);
         }
         List<MRoleMap> roleMember = listMSecurityPrincipalMembershipRole(mRol
             .getRoleName(), PrincipalType.ROLE, queryWrapper);
-        if (roleMember.size() > 0) {
+        if (CollectionUtils.isNotEmpty(roleMember)) {
           pm.deletePersistentAll(roleMember);
         }
         queryWrapper.close();
         // then remove all the grants
         List<MGlobalPrivilege> userGrants = listPrincipalMGlobalGrants(
             mRol.getRoleName(), PrincipalType.ROLE);
-        if (userGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(userGrants)) {
           pm.deletePersistentAll(userGrants);
         }
         List<MDBPrivilege> dbGrants = listPrincipalAllDBGrant(mRol
             .getRoleName(), PrincipalType.ROLE, queryWrapper);
-        if (dbGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(dbGrants)) {
           pm.deletePersistentAll(dbGrants);
         }
         queryWrapper.close();
         List<MTablePrivilege> tabPartGrants = listPrincipalAllTableGrants(
             mRol.getRoleName(), PrincipalType.ROLE, queryWrapper);
-        if (tabPartGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(tabPartGrants)) {
           pm.deletePersistentAll(tabPartGrants);
         }
         queryWrapper.close();
         List<MPartitionPrivilege> partGrants = listPrincipalAllPartitionGrants(
             mRol.getRoleName(), PrincipalType.ROLE, queryWrapper);
-        if (partGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(partGrants)) {
           pm.deletePersistentAll(partGrants);
         }
         queryWrapper.close();
         List<MTableColumnPrivilege> tblColumnGrants = listPrincipalAllTableColumnGrants(
             mRol.getRoleName(), PrincipalType.ROLE, queryWrapper);
-        if (tblColumnGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(tblColumnGrants)) {
           pm.deletePersistentAll(tblColumnGrants);
         }
         queryWrapper.close();
         List<MPartitionColumnPrivilege> partColumnGrants = listPrincipalAllPartitionColumnGrants(
             mRol.getRoleName(), PrincipalType.ROLE, queryWrapper);
-        if (partColumnGrants.size() > 0) {
+        if (CollectionUtils.isNotEmpty(partColumnGrants)) {
           pm.deletePersistentAll(partColumnGrants);
         }
         queryWrapper.close();
@@ -4931,7 +4909,7 @@ public class ObjectStore implements RawStore, Configurable {
       openTransaction();
       if (userName != null) {
         List<MGlobalPrivilege> user = this.listPrincipalMGlobalGrants(userName, PrincipalType.USER);
-        if(user.size()>0) {
+        if(CollectionUtils.isNotEmpty(user)) {
           Map<String, List<PrivilegeGrantInfo>> userPriv = new HashMap<>();
           List<PrivilegeGrantInfo> grantInfos = new ArrayList<>(user.size());
           for (int i = 0; i < user.size(); i++) {
@@ -4944,12 +4922,12 @@ public class ObjectStore implements RawStore, Configurable {
           ret.setUserPrivileges(userPriv);
         }
       }
-      if (groupNames != null && groupNames.size() > 0) {
+      if (CollectionUtils.isNotEmpty(groupNames)) {
         Map<String, List<PrivilegeGrantInfo>> groupPriv = new HashMap<>();
         for(String groupName: groupNames) {
           List<MGlobalPrivilege> group =
               this.listPrincipalMGlobalGrants(groupName, PrincipalType.GROUP);
-          if(group.size()>0) {
+          if(CollectionUtils.isNotEmpty(group)) {
             List<PrivilegeGrantInfo> grantInfos = new ArrayList<>(group.size());
             for (int i = 0; i < group.size(); i++) {
               MGlobalPrivilege item = group.get(i);
@@ -4979,7 +4957,7 @@ public class ObjectStore implements RawStore, Configurable {
     if (principalName != null) {
       List<MDBPrivilege> userNameDbPriv = this.listPrincipalMDBGrants(
           principalName, principalType, dbName);
-      if (userNameDbPriv != null && userNameDbPriv.size() > 0) {
+      if (CollectionUtils.isNotEmpty(userNameDbPriv)) {
         List<PrivilegeGrantInfo> grantInfos = new ArrayList<>(
             userNameDbPriv.size());
         for (int i = 0; i < userNameDbPriv.size(); i++) {
@@ -5011,7 +4989,7 @@ public class ObjectStore implements RawStore, Configurable {
             PrincipalType.USER));
         ret.setUserPrivileges(dbUserPriv);
       }
-      if (groupNames != null && groupNames.size() > 0) {
+      if (CollectionUtils.isNotEmpty(groupNames)) {
         Map<String, List<PrivilegeGrantInfo>> dbGroupPriv = new HashMap<>();
         for (String groupName : groupNames) {
           dbGroupPriv.put(groupName, getDBPrivilege(dbName, groupName,
@@ -5020,7 +4998,7 @@ public class ObjectStore implements RawStore, Configurable {
         ret.setGroupPrivileges(dbGroupPriv);
       }
       Set<String> roleNames = listAllRolesInHierarchy(userName, groupNames);
-      if (roleNames != null && roleNames.size() > 0) {
+      if (CollectionUtils.isNotEmpty(roleNames)) {
         Map<String, List<PrivilegeGrantInfo>> dbRolePriv = new HashMap<>();
         for (String roleName : roleNames) {
           dbRolePriv
@@ -5054,7 +5032,7 @@ public class ObjectStore implements RawStore, Configurable {
             tableName, partition, userName, PrincipalType.USER));
         ret.setUserPrivileges(partUserPriv);
       }
-      if (groupNames != null && groupNames.size() > 0) {
+      if (CollectionUtils.isNotEmpty(groupNames)) {
         Map<String, List<PrivilegeGrantInfo>> partGroupPriv = new HashMap<>();
         for (String groupName : groupNames) {
           partGroupPriv.put(groupName, getPartitionPrivilege(dbName, tableName,
@@ -5063,7 +5041,7 @@ public class ObjectStore implements RawStore, Configurable {
         ret.setGroupPrivileges(partGroupPriv);
       }
       Set<String> roleNames = listAllRolesInHierarchy(userName, groupNames);
-      if (roleNames != null && roleNames.size() > 0) {
+      if (CollectionUtils.isNotEmpty(roleNames)) {
         Map<String, List<PrivilegeGrantInfo>> partRolePriv = new HashMap<>();
         for (String roleName : roleNames) {
           partRolePriv.put(roleName, getPartitionPrivilege(dbName, tableName,
@@ -5097,7 +5075,7 @@ public class ObjectStore implements RawStore, Configurable {
             tableName, userName, PrincipalType.USER));
         ret.setUserPrivileges(tableUserPriv);
       }
-      if (groupNames != null && groupNames.size() > 0) {
+      if (CollectionUtils.isNotEmpty(groupNames)) {
         Map<String, List<PrivilegeGrantInfo>> tableGroupPriv = new HashMap<>();
         for (String groupName : groupNames) {
           tableGroupPriv.put(groupName, getTablePrivilege(dbName, tableName,
@@ -5106,7 +5084,7 @@ public class ObjectStore implements RawStore, Configurable {
         ret.setGroupPrivileges(tableGroupPriv);
       }
       Set<String> roleNames = listAllRolesInHierarchy(userName, groupNames);
-      if (roleNames != null && roleNames.size() > 0) {
+      if (CollectionUtils.isNotEmpty(roleNames)) {
         Map<String, List<PrivilegeGrantInfo>> tableRolePriv = new HashMap<>();
         for (String roleName : roleNames) {
           tableRolePriv.put(roleName, getTablePrivilege(dbName, tableName,
@@ -5142,7 +5120,7 @@ public class ObjectStore implements RawStore, Configurable {
             columnName, partitionName, userName, PrincipalType.USER));
         ret.setUserPrivileges(columnUserPriv);
       }
-      if (groupNames != null && groupNames.size() > 0) {
+      if (CollectionUtils.isNotEmpty(groupNames)) {
         Map<String, List<PrivilegeGrantInfo>> columnGroupPriv = new HashMap<>();
         for (String groupName : groupNames) {
           columnGroupPriv.put(groupName, getColumnPrivilege(dbName, tableName,
@@ -5151,7 +5129,7 @@ public class ObjectStore implements RawStore, Configurable {
         ret.setGroupPrivileges(columnGroupPriv);
       }
       Set<String> roleNames = listAllRolesInHierarchy(userName, groupNames);
-      if (roleNames != null && roleNames.size() > 0) {
+      if (CollectionUtils.isNotEmpty(roleNames)) {
         Map<String, List<PrivilegeGrantInfo>> columnRolePriv = new HashMap<>();
         for (String roleName : roleNames) {
           columnRolePriv.put(roleName, getColumnPrivilege(dbName, tableName,
@@ -5179,7 +5157,7 @@ public class ObjectStore implements RawStore, Configurable {
       List<MPartitionPrivilege> userNameTabPartPriv = this
           .listPrincipalMPartitionGrants(principalName, principalType,
               dbName, tableName, partName);
-      if (userNameTabPartPriv != null && userNameTabPartPriv.size() > 0) {
+      if (CollectionUtils.isNotEmpty(userNameTabPartPriv)) {
         List<PrivilegeGrantInfo> grantInfos = new ArrayList<>(
             userNameTabPartPriv.size());
         for (int i = 0; i < userNameTabPartPriv.size(); i++) {
@@ -5208,7 +5186,7 @@ public class ObjectStore implements RawStore, Configurable {
       List<MTablePrivilege> userNameTabPartPriv = this
           .listAllMTableGrants(principalName, principalType,
               dbName, tableName);
-      if (userNameTabPartPriv != null && userNameTabPartPriv.size() > 0) {
+      if (CollectionUtils.isNotEmpty(userNameTabPartPriv)) {
         List<PrivilegeGrantInfo> grantInfos = new ArrayList<>(
             userNameTabPartPriv.size());
         for (int i = 0; i < userNameTabPartPriv.size(); i++) {
@@ -5235,7 +5213,7 @@ public class ObjectStore implements RawStore, Configurable {
       List<MTableColumnPrivilege> userNameColumnPriv = this
           .listPrincipalMTableColumnGrants(principalName, principalType,
               dbName, tableName, columnName);
-      if (userNameColumnPriv != null && userNameColumnPriv.size() > 0) {
+      if (CollectionUtils.isNotEmpty(userNameColumnPriv)) {
         List<PrivilegeGrantInfo> grantInfos = new ArrayList<>(
             userNameColumnPriv.size());
         for (int i = 0; i < userNameColumnPriv.size(); i++) {
@@ -5250,7 +5228,7 @@ public class ObjectStore implements RawStore, Configurable {
       List<MPartitionColumnPrivilege> userNameColumnPriv = this
           .listPrincipalMPartitionColumnGrants(principalName,
               principalType, dbName, tableName, partitionName, columnName);
-      if (userNameColumnPriv != null && userNameColumnPriv.size() > 0) {
+      if (CollectionUtils.isNotEmpty(userNameColumnPriv)) {
         List<PrivilegeGrantInfo> grantInfos = new ArrayList<>(
             userNameColumnPriv.size());
         for (int i = 0; i < userNameColumnPriv.size(); i++) {
@@ -5276,7 +5254,7 @@ public class ObjectStore implements RawStore, Configurable {
 
       List<HiveObjectPrivilege> privilegeList = privileges.getPrivileges();
 
-      if (privilegeList != null && privilegeList.size() > 0) {
+      if (CollectionUtils.isNotEmpty(privilegeList)) {
         Iterator<HiveObjectPrivilege> privIter = privilegeList.iterator();
         Set<String> privSet = new HashSet<>();
         while (privIter.hasNext()) {
@@ -5468,7 +5446,7 @@ public class ObjectStore implements RawStore, Configurable {
           }
         }
       }
-      if (persistentObjs.size() > 0) {
+      if (CollectionUtils.isNotEmpty(persistentObjs)) {
         pm.makePersistentAll(persistentObjs);
       }
       committed = commitTransaction();
@@ -5491,7 +5469,7 @@ public class ObjectStore implements RawStore, Configurable {
       List<HiveObjectPrivilege> privilegeList = privileges.getPrivileges();
 
 
-      if (privilegeList != null && privilegeList.size() > 0) {
+      if (CollectionUtils.isNotEmpty(privilegeList)) {
         Iterator<HiveObjectPrivilege> privIter = privilegeList.iterator();
 
         while (privIter.hasNext()) {
@@ -5707,7 +5685,7 @@ public class ObjectStore implements RawStore, Configurable {
         }
       }
 
-      if (persistentObjs.size() > 0) {
+      if (CollectionUtils.isNotEmpty(persistentObjs)) {
         if (grantOption) {
           // If grant option specified, only update the privilege, don't remove it.
           // Grant option has already been removed from the privileges in the section above
@@ -7429,8 +7407,8 @@ public class ObjectStore implements RawStore, Configurable {
     QueryWrapper queryWrapper = new QueryWrapper();
 
     try {
-      LOG.info("Updating table level column statistics for db=" + dbName + " tableName=" + tableName
-        + " colName=" + colName);
+      LOG.info("Updating table level column statistics for db={} tableName={}" +
+        " colName={}", tableName, dbName, colName);
       validateTableCols(table, Lists.newArrayList(colName));
 
       if (oldStats != null) {
@@ -7928,9 +7906,8 @@ public class ObjectStore implements RawStore, Configurable {
       InvalidObjectException, InvalidInputException {
     boolean ret = false;
     Query query = null;
-    if (dbName == null) {
-      dbName = Warehouse.DEFAULT_DATABASE_NAME;
-    }
+    dbName = org.apache.commons.lang.StringUtils.defaultString(dbName,
+      Warehouse.DEFAULT_DATABASE_NAME);
     if (tableName == null) {
       throw new InvalidInputException("Table name is null.");
     }
@@ -8006,9 +7983,8 @@ public class ObjectStore implements RawStore, Configurable {
       throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
     boolean ret = false;
     Query query = null;
-    if (dbName == null) {
-      dbName = Warehouse.DEFAULT_DATABASE_NAME;
-    }
+    dbName = org.apache.commons.lang.StringUtils.defaultString(dbName,
+      Warehouse.DEFAULT_DATABASE_NAME);
     if (tableName == null) {
       throw new InvalidInputException("Table name is null.");
     }
@@ -8123,7 +8099,7 @@ public class ObjectStore implements RawStore, Configurable {
         rollbackTransaction();
       }
     }
-    LOG.debug("Done executing addToken with status : " + committed);
+    LOG.debug("Done executing addToken with status : {}", committed);
     return committed && (token == null);
   }
 
@@ -8145,7 +8121,7 @@ public class ObjectStore implements RawStore, Configurable {
         rollbackTransaction();
       }
     }
-    LOG.debug("Done executing removeToken with status : " + committed);
+    LOG.debug("Done executing removeToken with status : {}", committed);
     return committed && (token != null);
   }
 
@@ -8167,7 +8143,7 @@ public class ObjectStore implements RawStore, Configurable {
         rollbackTransaction();
       }
     }
-    LOG.debug("Done executing getToken with status : " + committed);
+    LOG.debug("Done executing getToken with status : {}", committed);
     return (null == token) ? null : token.getTokenStr();
   }
 
@@ -8190,7 +8166,7 @@ public class ObjectStore implements RawStore, Configurable {
       }
       return tokenIdents;
     } finally {
-      LOG.debug("Done executing getAllTokenIdentifers with status : " + committed);
+      LOG.debug("Done executing getAllTokenIdentifers with status : {}", committed);
       rollbackAndCleanup(committed, query);
     }
   }
@@ -8209,7 +8185,7 @@ public class ObjectStore implements RawStore, Configurable {
         rollbackTransaction();
       }
     }
-    LOG.debug("Done executing addMasterKey with status : " + committed);
+    LOG.debug("Done executing addMasterKey with status : {}", committed);
     if (committed) {
       return ((IntIdentity)pm.getObjectId(masterKey)).getKey();
     } else {
@@ -8236,7 +8212,7 @@ public class ObjectStore implements RawStore, Configurable {
     } finally {
       rollbackAndCleanup(committed, query);
     }
-    LOG.debug("Done executing updateMasterKey with status : " + committed);
+    LOG.debug("Done executing updateMasterKey with status : {}", committed);
     if (null == masterKey) {
       throw new NoSuchObjectException("No key found with keyId: " + id);
     }
@@ -8264,7 +8240,7 @@ public class ObjectStore implements RawStore, Configurable {
     } finally {
       rollbackAndCleanup(success, query);
     }
-    LOG.debug("Done executing removeMasterKey with status : " + success);
+    LOG.debug("Done executing removeMasterKey with status : {}", success);
     return (null != masterKey) && success;
   }
 
@@ -8287,7 +8263,7 @@ public class ObjectStore implements RawStore, Configurable {
       }
       return masterKeys;
     } finally {
-      LOG.debug("Done executing getMasterKeys with status : " + committed);
+      LOG.debug("Done executing getMasterKeys with status : {}", committed);
       rollbackAndCleanup(committed, query);
     }
   }
@@ -8321,18 +8297,17 @@ public class ObjectStore implements RawStore, Configurable {
 
     if (dbSchemaVer == null) {
       if (strictValidation) {
-        throw new MetaException("Version information not found in metastore. ");
+        throw new MetaException("Version information not found in metastore.");
       } else {
-        LOG.warn("Version information not found in metastore. "
-            + ConfVars.SCHEMA_VERIFICATION.toString() +
-            " is not enabled so recording the schema version " +
+        LOG.warn("Version information not found in metastore. {} is not " +
+          "enabled so recording the schema version {}", ConfVars.SCHEMA_VERIFICATION,
             hiveSchemaVer);
         setMetaStoreSchemaVersion(hiveSchemaVer,
           "Set by MetaStore " + USER + "@" + HOSTNAME);
       }
     } else {
       if (metastoreSchemaInfo.isVersionCompatible(hiveSchemaVer, dbSchemaVer)) {
-        LOG.debug("Found expected HMS version of " + dbSchemaVer);
+        LOG.debug("Found expected HMS version of {}", dbSchemaVer);
       } else {
         // metastore schema version is different than Hive distribution needs
         if (strictValidation) {
@@ -8340,9 +8315,9 @@ public class ObjectStore implements RawStore, Configurable {
               " does not match metastore's schema version " + dbSchemaVer +
               " Metastore is not upgraded or corrupt");
         } else {
-          LOG.error("Version information found in metastore differs " + dbSchemaVer +
-              " from expected schema version " + hiveSchemaVer +
-              ". Schema verififcation is disabled " + ConfVars.SCHEMA_VERIFICATION);
+          LOG.error("Version information found in metastore differs {} " +
+              "from expected schema version {}. Schema verififcation is disabled {}", 
+              dbSchemaVer, hiveSchemaVer, ConfVars.SCHEMA_VERIFICATION);
           setMetaStoreSchemaVersion(hiveSchemaVer,
             "Set by MetaStore " + USER + "@" + HOSTNAME);
         }
@@ -8411,10 +8386,10 @@ public class ObjectStore implements RawStore, Configurable {
       MetastoreConf.getBoolVar(getConf(), ConfVars.SCHEMA_VERIFICATION_RECORD_VERSION);
     if (!recordVersion) {
       LOG.warn("setMetaStoreSchemaVersion called but recording version is disabled: " +
-        "version = " + schemaVersion + ", comment = " + comment);
+        "version = {}, comment = {}", schemaVersion, comment);
       return;
     }
-    LOG.warn("Setting metastore schema version in db to " + schemaVersion);
+    LOG.warn("Setting metastore schema version in db to {}", schemaVersion);
 
     try {
       mSchemaVer = getMSchemaVersion();
@@ -8448,7 +8423,7 @@ public class ObjectStore implements RawStore, Configurable {
 
   private void debugLog(String message) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug(message + getCallStack());
+      LOG.debug("{} {}", message, getCallStack());
     }
   }
 
@@ -8502,7 +8477,7 @@ public class ObjectStore implements RawStore, Configurable {
     try {
       mdb = getMDatabase(func.getDbName());
     } catch (NoSuchObjectException e) {
-      LOG.error(StringUtils.stringifyException(e));
+      LOG.error("Database does not exist", e);
       throw new InvalidObjectException("Database " + func.getDbName() + " doesn't exist.");
     }
 
@@ -8764,8 +8739,8 @@ public class ObjectStore implements RawStore, Configurable {
           break;
         } catch (Exception e) {
           LOG.info(
-              "Attempting to acquire the DB log notification lock: " + currentRetries + " out of "
-                  + maxRetries + " retries", e);
+              "Attempting to acquire the DB log notification lock: {} out of {}" +
+                " retries", currentRetries, maxRetries, e);
           if (currentRetries >= maxRetries) {
             String message =
                 "Couldn't acquire the DB log notification lock because we reached the maximum"
@@ -8804,7 +8779,7 @@ public class ObjectStore implements RawStore, Configurable {
       Collection<MNotificationNextId> ids = (Collection) objectQuery.execute();
       MNotificationNextId mNotificationNextId = null;
       boolean needToPersistId;
-      if (ids == null || ids.size() == 0) {
+      if (CollectionUtils.isEmpty(ids)) {
         mNotificationNextId = new MNotificationNextId(1L);
         needToPersistId = true;
       } else {
@@ -8836,7 +8811,7 @@ public class ObjectStore implements RawStore, Configurable {
       query = pm.newQuery(MNotificationLog.class, "eventTime < tooOld");
       query.declareParameters("java.lang.Integer tooOld");
       Collection<MNotificationLog> toBeRemoved = (Collection) query.execute(tooOld);
-      if (toBeRemoved != null && toBeRemoved.size() > 0) {
+      if (CollectionUtils.isNotEmpty(toBeRemoved)) {
         pm.deletePersistentAll(toBeRemoved);
       }
       commited = commitTransaction();
@@ -8854,7 +8829,7 @@ public class ObjectStore implements RawStore, Configurable {
       query = pm.newQuery(MNotificationNextId.class);
       Collection<MNotificationNextId> ids = (Collection) query.execute();
       long id = 0;
-      if (ids != null && ids.size() > 0) {
+      if (CollectionUtils.isNotEmpty(ids)) {
         id = ids.iterator().next().getNextEventId() - 1;
       }
       commited = commitTransaction();
@@ -9016,7 +8991,7 @@ public class ObjectStore implements RawStore, Configurable {
       classLoaderResolverMap.set(nc, new HashMap<String, ClassLoaderResolver>());
       LOG.debug("Removed cached classloaders from DataNucleus NucleusContext");
     } catch (Exception e) {
-      LOG.warn("Failed to remove cached classloaders from DataNucleus NucleusContext ", e);
+      LOG.warn("Failed to remove cached classloaders from DataNucleus NucleusContext", e);
     }
   }
 
@@ -9027,8 +9002,8 @@ public class ObjectStore implements RawStore, Configurable {
         long resourcesCleared = clearFieldMap(clri,"resources");
         long loadedClassesCleared = clearFieldMap(clri,"loadedClasses");
         long unloadedClassesCleared = clearFieldMap(clri, "unloadedClasses");
-        LOG.debug("Cleared ClassLoaderResolverImpl: " +
-            resourcesCleared + "," + loadedClassesCleared + "," + unloadedClassesCleared);
+        LOG.debug("Cleared ClassLoaderResolverImpl: {}, {}, {}",
+            resourcesCleared, loadedClassesCleared, unloadedClassesCleared);
       }
     }
   }
@@ -9419,7 +9394,7 @@ public class ObjectStore implements RawStore, Configurable {
 
       List<MConstraint> tabConstraints = listAllTableConstraintsWithOptionalConstraintName(
                                          dbName, tableName, constraintName);
-      if (tabConstraints != null && tabConstraints.size() > 0) {
+      if (CollectionUtils.isNotEmpty(tabConstraints)) {
         pm.deletePersistentAll(tabConstraints);
       } else {
         throw new NoSuchObjectException("The constraint: " + constraintName +