You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ng...@apache.org on 2021/04/06 15:40:14 UTC

[hive] 30/38: HIVE-24396: Some changes with formatters after the rebase (Naveen Gangam)

This is an automated email from the ASF dual-hosted git repository.

ngangam pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git

commit d7a8eb719510da766246cccb699454d754e69fd3
Author: Naveen Gangam <ng...@cloudera.com>
AuthorDate: Thu Mar 4 10:20:33 2021 -0500

    HIVE-24396: Some changes with formatters after the rebase (Naveen Gangam)
---
 .../ddl/database/desc/DescDatabaseFormatter.java   |    23 +-
 .../desc/DescDataConnectorFormatter.java}          |    58 +-
 .../desc/DescDataConnectorOperation.java           |     7 +-
 .../show/ShowDataConnectorsFormatter.java          |    69 +
 .../show/ShowDataConnectorsOperation.java          |     7 +-
 .../metadata/formatting/JsonMetaDataFormatter.java |   704 --
 .../hadoop/hive/metastore/HiveMetaStore.java       | 10368 -------------------
 .../src/main/resources/package.jdo                 |     1 +
 8 files changed, 127 insertions(+), 11110 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/desc/DescDatabaseFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/desc/DescDatabaseFormatter.java
index f6cd633..5e10c3b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/desc/DescDatabaseFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/desc/DescDatabaseFormatter.java
@@ -46,7 +46,8 @@ abstract class DescDatabaseFormatter {
   }
 
   abstract void showDatabaseDescription(DataOutputStream out, String database, String comment, String location,
-      String managedLocation, String ownerName, PrincipalType ownerType, Map<String, String> params)
+      String managedLocation, String ownerName, PrincipalType ownerType, Map<String, String> params,
+      String connectorName, String remoteDbName)
       throws HiveException;
 
   // ------ Implementations ------
@@ -54,7 +55,8 @@ abstract class DescDatabaseFormatter {
   static class JsonDescDatabaseFormatter extends DescDatabaseFormatter {
     @Override
     void showDatabaseDescription(DataOutputStream out, String database, String comment, String location,
-        String managedLocation, String ownerName, PrincipalType ownerType, Map<String, String> params)
+        String managedLocation, String ownerName, PrincipalType ownerType, Map<String, String> params,
+        String connectorName, String remoteDbName)
         throws HiveException {
       MapBuilder builder = MapBuilder.create()
           .put("database", database)
@@ -69,6 +71,12 @@ abstract class DescDatabaseFormatter {
       if (ownerType != null) {
         builder.put("ownerType", ownerType.name());
       }
+      if (null != connectorName) {
+        builder.put("connector_name", connectorName);
+      }
+      if (null != remoteDbName) {
+        builder.put("remote_dbname", remoteDbName);
+      }
       if (MapUtils.isNotEmpty(params)) {
         builder.put("params", params);
       }
@@ -79,7 +87,8 @@ abstract class DescDatabaseFormatter {
   static class TextDescDatabaseFormatter extends DescDatabaseFormatter {
     @Override
     void showDatabaseDescription(DataOutputStream out, String database, String comment, String location,
-        String managedLocation, String ownerName, PrincipalType ownerType, Map<String, String> params)
+        String managedLocation, String ownerName, PrincipalType ownerType, Map<String, String> params,
+        String connectorName, String remoteDbName)
         throws HiveException {
       try {
         out.write(database.getBytes(StandardCharsets.UTF_8));
@@ -104,6 +113,14 @@ abstract class DescDatabaseFormatter {
           out.write(ownerType.name().getBytes(StandardCharsets.UTF_8));
         }
         out.write(Utilities.tabCode);
+        if (connectorName != null) {
+          out.write(connectorName.getBytes(StandardCharsets.UTF_8));
+        }
+        out.write(Utilities.tabCode);
+        if (remoteDbName != null) {
+          out.write(remoteDbName.getBytes(StandardCharsets.UTF_8));
+        }
+        out.write(Utilities.tabCode);
         if (MapUtils.isNotEmpty(params)) {
           out.write(params.toString().getBytes(StandardCharsets.UTF_8));
         }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/desc/DescDatabaseFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/desc/DescDataConnectorFormatter.java
similarity index 65%
copy from ql/src/java/org/apache/hadoop/hive/ql/ddl/database/desc/DescDatabaseFormatter.java
copy to ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/desc/DescDataConnectorFormatter.java
index f6cd633..3865c62 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/desc/DescDatabaseFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/desc/DescDataConnectorFormatter.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.database.desc;
+package org.apache.hadoop.hive.ql.ddl.dataconnector.desc;
 
 import org.apache.commons.collections.MapUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -34,41 +34,41 @@ import java.nio.charset.StandardCharsets;
 import java.util.Map;
 
 /**
- * Formats DESC DATABASES results.
+ * Formats DESC CONNECTOR results.
  */
-abstract class DescDatabaseFormatter {
-  static DescDatabaseFormatter getFormatter(HiveConf conf) {
+abstract class DescDataConnectorFormatter {
+  static DescDataConnectorFormatter getFormatter(HiveConf conf) {
     if (MetaDataFormatUtils.isJson(conf)) {
-      return new JsonDescDatabaseFormatter();
+      return new JsonDescDataConnectorFormatter();
     } else {
-      return new TextDescDatabaseFormatter();
+      return new TextDescDataConnectorFormatter();
     }
   }
 
-  abstract void showDatabaseDescription(DataOutputStream out, String database, String comment, String location,
-      String managedLocation, String ownerName, PrincipalType ownerType, Map<String, String> params)
+  abstract void showDataConnectorDescription(DataOutputStream out, String connector, String type, String url,
+      String ownerName, PrincipalType ownerType, String comment, Map<String, String> params)
       throws HiveException;
 
   // ------ Implementations ------
 
-  static class JsonDescDatabaseFormatter extends DescDatabaseFormatter {
+  static class JsonDescDataConnectorFormatter extends DescDataConnectorFormatter {
     @Override
-    void showDatabaseDescription(DataOutputStream out, String database, String comment, String location,
-        String managedLocation, String ownerName, PrincipalType ownerType, Map<String, String> params)
+    void showDataConnectorDescription(DataOutputStream out, String connector, String type, String url,
+        String ownerName, PrincipalType ownerType, String comment, Map<String, String> params)
         throws HiveException {
       MapBuilder builder = MapBuilder.create()
-          .put("database", database)
-          .put("comment", comment)
-          .put("location", location);
-      if (managedLocation != null) {
-        builder.put("managedLocation", managedLocation);
-      }
+          .put("connector", connector)
+          .put("type", type)
+          .put("url", url);
       if (ownerName != null) {
         builder.put("owner", ownerName);
       }
       if (ownerType != null) {
         builder.put("ownerType", ownerType.name());
       }
+      if (comment != null) {
+        builder.put("comment", comment);
+      }
       if (MapUtils.isNotEmpty(params)) {
         builder.put("params", params);
       }
@@ -76,24 +76,20 @@ abstract class DescDatabaseFormatter {
     }
   }
 
-  static class TextDescDatabaseFormatter extends DescDatabaseFormatter {
+  static class TextDescDataConnectorFormatter extends DescDataConnectorFormatter {
     @Override
-    void showDatabaseDescription(DataOutputStream out, String database, String comment, String location,
-        String managedLocation, String ownerName, PrincipalType ownerType, Map<String, String> params)
+    void showDataConnectorDescription(DataOutputStream out, String connector, String type, String url,
+        String ownerName, PrincipalType ownerType, String comment, Map<String, String> params)
         throws HiveException {
       try {
-        out.write(database.getBytes(StandardCharsets.UTF_8));
-        out.write(Utilities.tabCode);
-        if (comment != null) {
-          out.write(HiveStringUtils.escapeJava(comment).getBytes(StandardCharsets.UTF_8));
-        }
+        out.write(connector.getBytes(StandardCharsets.UTF_8));
         out.write(Utilities.tabCode);
-        if (location != null) {
-          out.write(location.getBytes(StandardCharsets.UTF_8));
+        if (type != null) {
+          out.write(type.getBytes(StandardCharsets.UTF_8));
         }
         out.write(Utilities.tabCode);
-        if (managedLocation != null) {
-          out.write(managedLocation.getBytes(StandardCharsets.UTF_8));
+        if (url != null) {
+          out.write(url.getBytes(StandardCharsets.UTF_8));
         }
         out.write(Utilities.tabCode);
         if (ownerName != null) {
@@ -104,6 +100,10 @@ abstract class DescDatabaseFormatter {
           out.write(ownerType.name().getBytes(StandardCharsets.UTF_8));
         }
         out.write(Utilities.tabCode);
+        if (comment != null) {
+          out.write(HiveStringUtils.escapeJava(comment).getBytes(StandardCharsets.UTF_8));
+        }
+        out.write(Utilities.tabCode);
         if (MapUtils.isNotEmpty(params)) {
           out.write(params.toString().getBytes(StandardCharsets.UTF_8));
         }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/desc/DescDataConnectorOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/desc/DescDataConnectorOperation.java
index de76842..9614f24 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/desc/DescDataConnectorOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/desc/DescDataConnectorOperation.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hive.metastore.api.DataConnector;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.ddl.DDLOperation;
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
-import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.ddl.ShowUtils;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
@@ -40,7 +40,7 @@ public class DescDataConnectorOperation extends DDLOperation<DescDataConnectorDe
 
   @Override
   public int execute() throws HiveException {
-    try (DataOutputStream outStream = DDLUtils.getOutputStream(new Path(desc.getResFile()), context)) {
+    try (DataOutputStream outStream = ShowUtils.getOutputStream(new Path(desc.getResFile()), context)) {
       DataConnector connector = context.getDb().getDataConnector(desc.getConnectorName());
       if (connector == null) {
         throw new HiveException(ErrorMsg.DATACONNECTOR_NOT_EXISTS, desc.getConnectorName());
@@ -51,7 +51,8 @@ public class DescDataConnectorOperation extends DDLOperation<DescDataConnectorDe
         params = new TreeMap<>(connector.getParameters());
       }
 
-      context.getFormatter().showDataConnectorDescription(outStream, connector.getName(), connector.getType(),
+      DescDataConnectorFormatter formatter = DescDataConnectorFormatter.getFormatter(context.getConf());
+      formatter.showDataConnectorDescription(outStream, connector.getName(), connector.getType(),
           connector.getUrl(), connector.getOwnerName(), connector.getOwnerType(), connector.getDescription(), params);
     } catch (Exception e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/show/ShowDataConnectorsFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/show/ShowDataConnectorsFormatter.java
new file mode 100644
index 0000000..7ef6531
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/show/ShowDataConnectorsFormatter.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.ddl.dataconnector.show;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.ddl.ShowUtils;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.formatting.MapBuilder;
+import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+/**
+ * Formats SHOW CONNECTORS results.
+ */
+abstract class ShowDataConnectorsFormatter {
+  static ShowDataConnectorsFormatter getFormatter(HiveConf conf) {
+    if (MetaDataFormatUtils.isJson(conf)) {
+      return new JsonShowDataConnectorsFormatter();
+    } else {
+      return new TextShowDataConnectorsFormatter();
+    }
+  }
+
+  abstract void showDataConnectors(DataOutputStream out, List<String> connectors) throws HiveException;
+
+  // ------ Implementations ------
+
+  static class JsonShowDataConnectorsFormatter extends ShowDataConnectorsFormatter {
+    @Override
+    void showDataConnectors(DataOutputStream out, List<String> connectors) throws HiveException {
+      ShowUtils.asJson(out, MapBuilder.create().put("connectors", connectors).build());
+    }
+  }
+
+  static class TextShowDataConnectorsFormatter extends ShowDataConnectorsFormatter {
+    @Override
+    void showDataConnectors(DataOutputStream out, List<String> connectors) throws HiveException {
+      try {
+        for (String connector : connectors) {
+          out.write(connector.getBytes(StandardCharsets.UTF_8));
+          out.write(Utilities.newLineCode);
+        }
+      } catch (IOException e) {
+        throw new HiveException(e);
+      }
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/show/ShowDataConnectorsOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/show/ShowDataConnectorsOperation.java
index 8ecc9eb..22b10d8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/show/ShowDataConnectorsOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/dataconnector/show/ShowDataConnectorsOperation.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.ddl.DDLOperation;
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
-import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.ddl.ShowUtils;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.UDFLike;
 import org.apache.hadoop.io.IOUtils;
@@ -52,9 +52,10 @@ public class ShowDataConnectorsOperation extends DDLOperation<ShowDataConnectors
     LOG.info("Found {} connector(s) matching the SHOW CONNECTORS statement.", connectors.size());
 
     // write the results in the file
-    DataOutputStream outStream = DDLUtils.getOutputStream(new Path(desc.getResFile()), context);
+    DataOutputStream outStream = ShowUtils.getOutputStream(new Path(desc.getResFile()), context);
     try {
-      context.getFormatter().showDataConnectors(outStream, connectors);
+      ShowDataConnectorsFormatter formatter = ShowDataConnectorsFormatter.getFormatter(context.getConf());
+      formatter.showDataConnectors(outStream, connectors);
     } catch (Exception e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR, "show connectors");
     } finally {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
index 6a5234a..9a245c9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
@@ -66,710 +66,6 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
     asJson(out, mb.build());
   }
 
-  /**
-   * Show a list of tables.
-   */
-  @Override
-  public void showTables(DataOutputStream out, List<String> tables)
-      throws HiveException {
-    asJson(out, MapBuilder.create().put("tables", tables).build());
-  }
-
-  /**
-   * Show a list of tables including table types.
-   */
-  @Override
-  public void showTablesExtended(DataOutputStream out, List<Table> tables) throws HiveException {
-    if (tables.isEmpty()) {
-      return;
-    }
-
-    List<Map<String, Object>> tableDataList = new ArrayList<Map<String, Object>>();
-    for (Table table : tables) {
-      Map<String, Object> tableData = ImmutableMap.of(
-          "Table Name", table.getTableName(),
-          "Table Type", table.getTableType().toString());
-      tableDataList.add(tableData);
-    }
-    asJson(out, ImmutableMap.of("tables", tableDataList));
-  }
-
-  /**
-   * Show a list of materialized views.
-   */
-  @Override
-  public void showMaterializedViews(DataOutputStream out, List<Table> materializedViews) throws HiveException {
-    if (materializedViews.isEmpty()) {
-      return;
-    }
-
-    List<Map<String, Object>> materializedViewDataList = new ArrayList<Map<String, Object>>();
-    for (Table materializedView : materializedViews) {
-      // Currently, we only support manual refresh
-      // TODO: Update whenever we have other modes
-      String refreshMode = "Manual refresh";
-      String timeWindowString = materializedView.getProperty(MATERIALIZED_VIEW_REWRITING_TIME_WINDOW);
-      String mode;
-      if (!StringUtils.isEmpty(timeWindowString)) {
-        long time = HiveConf.toTime(timeWindowString,
-            HiveConf.getDefaultTimeUnit(HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_REWRITING_TIME_WINDOW),
-            TimeUnit.MINUTES);
-        if (time > 0L) {
-          mode = refreshMode + " (Valid for " + time + "min)";
-        } else if (time == 0L) {
-          mode = refreshMode + " (Valid until source tables modified)";
-        } else {
-          mode = refreshMode + " (Valid always)";
-        }
-      } else {
-        mode = refreshMode;
-      }
-
-      Map<String, Object> materializedViewData = ImmutableMap.of(
-          "MV Name", materializedView.getTableName(),
-          "Rewriting Enabled", materializedView.isRewriteEnabled() ? "Yes" : "No",
-          "Mode", mode);
-      materializedViewDataList.add(materializedViewData);
-    }
-    asJson(out, ImmutableMap.of("materialized views", materializedViewDataList));
-  }
-
-  /**
-   * Describe table.
-   */
-  @Override
-  public void describeTable(DataOutputStream out, String colPath, String tableName, Table tbl, Partition part,
-      List<FieldSchema> cols, boolean isFormatted, boolean isExt, boolean isOutputPadded,
-      List<ColumnStatisticsObj> colStats) throws HiveException {
-    MapBuilder builder = MapBuilder.create();
-    builder.put("columns", createColumnsInfo(cols, colStats));
-
-    if (isExt) {
-      if (part != null) {
-        builder.put("partitionInfo", part.getTPartition());
-      }
-      else {
-        builder.put("tableInfo", tbl.getTTable());
-      }
-      if (PrimaryKeyInfo.isPrimaryKeyInfoNotEmpty(tbl.getPrimaryKeyInfo())) {
-        builder.put("primaryKeyInfo", tbl.getPrimaryKeyInfo());
-      }
-      if (ForeignKeyInfo.isForeignKeyInfoNotEmpty(tbl.getForeignKeyInfo())) {
-        builder.put("foreignKeyInfo", tbl.getForeignKeyInfo());
-      }
-      if (UniqueConstraint.isUniqueConstraintNotEmpty(tbl.getUniqueKeyInfo())) {
-        builder.put("uniqueConstraintInfo", tbl.getUniqueKeyInfo());
-      }
-      if (NotNullConstraint.isNotNullConstraintNotEmpty(tbl.getNotNullConstraint())) {
-        builder.put("notNullConstraintInfo", tbl.getNotNullConstraint());
-      }
-      if (DefaultConstraint.isCheckConstraintNotEmpty(tbl.getDefaultConstraint())) {
-        builder.put("defaultConstraintInfo", tbl.getDefaultConstraint());
-      }
-      if (CheckConstraint.isCheckConstraintNotEmpty(tbl.getCheckConstraint())) {
-        builder.put("checkConstraintInfo", tbl.getCheckConstraint());
-      }
-      if (tbl.getStorageHandlerInfo() != null) {
-        builder.put("storageHandlerInfo", tbl.getStorageHandlerInfo().toString());
-      }
-    }
-
-    asJson(out, builder.build());
-  }
-
-  private List<Map<String, Object>> createColumnsInfo(List<FieldSchema> columns,
-      List<ColumnStatisticsObj> columnStatisticsList) {
-    ArrayList<Map<String, Object>> res = new ArrayList<Map<String, Object>>();
-    for (FieldSchema column : columns) {
-      ColumnStatisticsData statistics = getStatistics(column, columnStatisticsList);
-      res.add(createColumnInfo(column, statistics));
-    }
-    return res;
-  }
-
-  private ColumnStatisticsData getStatistics(FieldSchema column, List<ColumnStatisticsObj> columnStatisticsList) {
-    for (ColumnStatisticsObj columnStatistics : columnStatisticsList) {
-      if (column.getName().equals(columnStatistics.getColName())) {
-        return columnStatistics.getStatsData();
-      }
-    }
-
-    return null;
-  }
-
-  private Map<String, Object> createColumnInfo(FieldSchema column, ColumnStatisticsData statistics) {
-    Map<String, Object> result = MapBuilder.create()
-        .put(COLUMN_NAME, column.getName())
-        .put(COLUMN_TYPE, column.getType())
-        .put(COLUMN_COMMENT, column.getComment())
-        .build();
-
-    if (statistics != null) {
-      if (statistics.isSetBinaryStats()) {
-        if (statistics.getBinaryStats().isSetNumNulls()) {
-          result.put(COLUMN_NUM_NULLS, statistics.getBinaryStats().getNumNulls());
-        }
-        if (statistics.getBinaryStats().isSetAvgColLen()) {
-          result.put(COLUMN_AVG_LENGTH, statistics.getBinaryStats().getAvgColLen());
-        }
-        if (statistics.getBinaryStats().isSetMaxColLen()) {
-          result.put(COLUMN_MAX_LENGTH, statistics.getBinaryStats().getMaxColLen());
-        }
-      } else if (statistics.isSetStringStats()) {
-        if (statistics.getStringStats().isSetNumNulls()) {
-          result.put(COLUMN_NUM_NULLS, statistics.getStringStats().getNumNulls());
-        }
-        if (statistics.getStringStats().isSetNumDVs()) {
-          result.put(COLUMN_DISTINCT_COUNT, statistics.getStringStats().getNumDVs());
-        }
-        if (statistics.getStringStats().isSetAvgColLen()) {
-          result.put(COLUMN_AVG_LENGTH, statistics.getStringStats().getAvgColLen());
-        }
-        if (statistics.getStringStats().isSetMaxColLen()) {
-          result.put(COLUMN_MAX_LENGTH, statistics.getStringStats().getMaxColLen());
-        }
-      } else if (statistics.isSetBooleanStats()) {
-        if (statistics.getBooleanStats().isSetNumNulls()) {
-          result.put(COLUMN_NUM_NULLS, statistics.getBooleanStats().getNumNulls());
-        }
-        if (statistics.getBooleanStats().isSetNumTrues()) {
-          result.put(COLUMN_NUM_TRUES, statistics.getBooleanStats().getNumTrues());
-        }
-        if (statistics.getBooleanStats().isSetNumFalses()) {
-          result.put(COLUMN_NUM_FALSES, statistics.getBooleanStats().getNumFalses());
-        }
-      } else if (statistics.isSetDecimalStats()) {
-        if (statistics.getDecimalStats().isSetLowValue()) {
-          result.put(COLUMN_MIN, MetaDataFormatUtils.convertToString(statistics.getDecimalStats().getLowValue()));
-        }
-        if (statistics.getDecimalStats().isSetHighValue()) {
-          result.put(COLUMN_MAX, MetaDataFormatUtils.convertToString(statistics.getDecimalStats().getHighValue()));
-        }
-        if (statistics.getDecimalStats().isSetNumNulls()) {
-          result.put(COLUMN_NUM_NULLS, statistics.getDecimalStats().getNumNulls());
-        }
-        if (statistics.getDecimalStats().isSetNumDVs()) {
-          result.put(COLUMN_DISTINCT_COUNT, statistics.getDecimalStats().getNumDVs());
-        }
-      } else if (statistics.isSetDoubleStats()) {
-        if (statistics.getDoubleStats().isSetLowValue()) {
-          result.put(COLUMN_MIN, statistics.getDoubleStats().getLowValue());
-        }
-        if (statistics.getDoubleStats().isSetHighValue()) {
-          result.put(COLUMN_MAX, statistics.getDoubleStats().getHighValue());
-        }
-        if (statistics.getDoubleStats().isSetNumNulls()) {
-          result.put(COLUMN_NUM_NULLS, statistics.getDoubleStats().getNumNulls());
-        }
-        if (statistics.getDoubleStats().isSetNumDVs()) {
-          result.put(COLUMN_DISTINCT_COUNT, statistics.getDoubleStats().getNumDVs());
-        }
-      } else if (statistics.isSetLongStats()) {
-        if (statistics.getLongStats().isSetLowValue()) {
-          result.put(COLUMN_MIN, statistics.getLongStats().getLowValue());
-        }
-        if (statistics.getLongStats().isSetHighValue()) {
-          result.put(COLUMN_MAX, statistics.getLongStats().getHighValue());
-        }
-        if (statistics.getLongStats().isSetNumNulls()) {
-          result.put(COLUMN_NUM_NULLS, statistics.getLongStats().getNumNulls());
-        }
-        if (statistics.getLongStats().isSetNumDVs()) {
-          result.put(COLUMN_DISTINCT_COUNT, statistics.getLongStats().getNumDVs());
-        }
-      } else if (statistics.isSetDateStats()) {
-        if (statistics.getDateStats().isSetLowValue()) {
-          result.put(COLUMN_MIN, MetaDataFormatUtils.convertToString(statistics.getDateStats().getLowValue()));
-        }
-        if (statistics.getDateStats().isSetHighValue()) {
-          result.put(COLUMN_MAX, MetaDataFormatUtils.convertToString(statistics.getDateStats().getHighValue()));
-        }
-        if (statistics.getDateStats().isSetNumNulls()) {
-          result.put(COLUMN_NUM_NULLS, statistics.getDateStats().getNumNulls());
-        }
-        if (statistics.getDateStats().isSetNumDVs()) {
-          result.put(COLUMN_DISTINCT_COUNT, statistics.getDateStats().getNumDVs());
-        }
-      } else if (statistics.isSetTimestampStats()) {
-        if (statistics.getTimestampStats().isSetLowValue()) {
-          result.put(COLUMN_MIN, MetaDataFormatUtils.convertToString(statistics.getTimestampStats().getLowValue()));
-        }
-        if (statistics.getTimestampStats().isSetHighValue()) {
-          result.put(COLUMN_MAX, MetaDataFormatUtils.convertToString(statistics.getTimestampStats().getHighValue()));
-        }
-        if (statistics.getTimestampStats().isSetNumNulls()) {
-          result.put(COLUMN_NUM_NULLS, statistics.getTimestampStats().getNumNulls());
-        }
-        if (statistics.getTimestampStats().isSetNumDVs()) {
-          result.put(COLUMN_DISTINCT_COUNT, statistics.getTimestampStats().getNumDVs());
-        }
-      }
-    }
-
-    return result;
-  }
-
-  @Override
-  public void showTableStatus(DataOutputStream out, Hive db, HiveConf conf,
-      List<Table> tbls, Map<String, String> part, Partition par)
-          throws HiveException {
-    asJson(out, MapBuilder.create().put(
-        "tables", makeAllTableStatus(db, conf, tbls, part, par)).build());
-  }
-
-  private List<Map<String, Object>> makeAllTableStatus(Hive db, HiveConf conf,
-      List<Table> tbls, Map<String, String> part, Partition par)
-          throws HiveException {
-    try {
-      ArrayList<Map<String, Object>> res = new ArrayList<Map<String, Object>>();
-      for (Table tbl : tbls) {
-        res.add(makeOneTableStatus(tbl, db, conf, part, par));
-      }
-      return res;
-    } catch(IOException e) {
-      throw new HiveException(e);
-    }
-  }
-
-  private Map<String, Object> makeOneTableStatus(Table tbl, Hive db,
-      HiveConf conf, Map<String, String> part, Partition par)
-          throws HiveException, IOException {
-    String tblLoc = null;
-    String inputFormattCls = null;
-    String outputFormattCls = null;
-    if (part != null) {
-      if (par != null) {
-        if (par.getLocation() != null) {
-          tblLoc = par.getDataLocation().toString();
-        }
-        inputFormattCls = par.getInputFormatClass() == null ? null : par.getInputFormatClass().getName();
-        outputFormattCls = par.getOutputFormatClass() == null ? null : par.getOutputFormatClass().getName();
-      }
-    } else {
-      if (tbl.getPath() != null) {
-        tblLoc = tbl.getDataLocation().toString();
-      }
-      inputFormattCls = tbl.getInputFormatClass() == null ? null : tbl.getInputFormatClass().getName();
-      outputFormattCls = tbl.getOutputFormatClass() == null ? null : tbl.getOutputFormatClass().getName();
-    }
-
-    MapBuilder builder = MapBuilder.create();
-
-    builder.put("tableName", tbl.getTableName());
-    builder.put("ownerType", (tbl.getOwnerType() != null) ? tbl.getOwnerType().name() : "null");
-    builder.put("owner", tbl.getOwner());
-    builder.put("location", tblLoc);
-    builder.put("inputFormat", inputFormattCls);
-    builder.put("outputFormat", outputFormattCls);
-    builder.put("columns", createColumnsInfo(tbl.getCols(), new ArrayList<ColumnStatisticsObj>()));
-
-    builder.put("partitioned", tbl.isPartitioned());
-    if (tbl.isPartitioned()) {
-      builder.put("partitionColumns", createColumnsInfo(tbl.getPartCols(), new ArrayList<ColumnStatisticsObj>()));
-    }
-    if(tbl.getTableType() != TableType.VIRTUAL_VIEW) {
-      //tbl.getPath() is null for views
-      putFileSystemsStats(builder, makeTableStatusLocations(tbl, db, par),
-        conf, tbl.getPath());
-    }
-
-    return builder.build();
-  }
-
-  private List<Path> makeTableStatusLocations(Table tbl, Hive db, Partition par)
-      throws HiveException {
-    // output file system information
-    Path tblPath = tbl.getPath();
-    List<Path> locations = new ArrayList<Path>();
-    if (tbl.isPartitioned()) {
-      if (par == null) {
-        for (Partition curPart : db.getPartitions(tbl)) {
-          if (curPart.getLocation() != null) {
-            locations.add(new Path(curPart.getLocation()));
-          }
-        }
-      } else {
-        if (par.getLocation() != null) {
-          locations.add(new Path(par.getLocation()));
-        }
-      }
-    } else {
-      if (tblPath != null) {
-        locations.add(tblPath);
-      }
-    }
-
-    return locations;
-  }
-
-  /**
-   * @param tblPath not NULL
-   * @throws IOException
-   */
-  // Duplicates logic in TextMetaDataFormatter
-  private void putFileSystemsStats(MapBuilder builder, List<Path> locations,
-      HiveConf conf, Path tblPath)
-          throws IOException {
-    long totalFileSize = 0;
-    long maxFileSize = 0;
-    long minFileSize = Long.MAX_VALUE;
-    long lastAccessTime = 0;
-    long lastUpdateTime = 0;
-    int numOfFiles = 0;
-
-    boolean unknown = false;
-    FileSystem fs = tblPath.getFileSystem(conf);
-    // in case all files in locations do not exist
-    try {
-      FileStatus tmpStatus = fs.getFileStatus(tblPath);
-      lastAccessTime = tmpStatus.getAccessTime();
-      lastUpdateTime = tmpStatus.getModificationTime();
-    } catch (IOException e) {
-      LOG.warn(
-          "Cannot access File System. File System status will be unknown: ", e);
-      unknown = true;
-    }
-
-    if (!unknown) {
-      for (Path loc : locations) {
-        try {
-          FileStatus status = fs.getFileStatus(tblPath);
-          FileStatus[] files = fs.listStatus(loc);
-          long accessTime = status.getAccessTime();
-          long updateTime = status.getModificationTime();
-          // no matter loc is the table location or part location, it must be a
-          // directory.
-          if (!status.isDir()) {
-            continue;
-          }
-          if (accessTime > lastAccessTime) {
-            lastAccessTime = accessTime;
-          }
-          if (updateTime > lastUpdateTime) {
-            lastUpdateTime = updateTime;
-          }
-          for (FileStatus currentStatus : files) {
-            if (currentStatus.isDir()) {
-              continue;
-            }
-            numOfFiles++;
-            long fileLen = currentStatus.getLen();
-            totalFileSize += fileLen;
-            if (fileLen > maxFileSize) {
-              maxFileSize = fileLen;
-            }
-            if (fileLen < minFileSize) {
-              minFileSize = fileLen;
-            }
-            accessTime = currentStatus.getAccessTime();
-            updateTime = currentStatus.getModificationTime();
-            if (accessTime > lastAccessTime) {
-              lastAccessTime = accessTime;
-            }
-            if (updateTime > lastUpdateTime) {
-              lastUpdateTime = updateTime;
-            }
-          }
-        } catch (IOException e) {
-          // ignore
-        }
-      }
-    }
-
-    builder
-    .put("totalNumberFiles", numOfFiles, ! unknown)
-    .put("totalFileSize",    totalFileSize, ! unknown)
-    .put("maxFileSize",      maxFileSize, ! unknown)
-    .put("minFileSize",      numOfFiles > 0 ? minFileSize : 0, ! unknown)
-    .put("lastAccessTime",   lastAccessTime, ! (unknown  || lastAccessTime < 0))
-    .put("lastUpdateTime",   lastUpdateTime, ! unknown);
-  }
-
-  /**
-   * Show the table partitions.
-   */
-  @Override
-  public void showTablePartitions(DataOutputStream out, List<String> parts)
-      throws HiveException {
-    asJson(out, MapBuilder.create().put("partitions",
-        makeTablePartions(parts)).build());
-  }
-
-  private List<Map<String, Object>> makeTablePartions(List<String> parts) {
-    ArrayList<Map<String, Object>> res =
-        new ArrayList<Map<String, Object>>(parts.size());
-    for (String part : parts) {
-      res.add(makeOneTablePartition(part));
-    }
-    return res;
-  }
-
-  // This seems like a very wrong implementation.
-  private Map<String, Object> makeOneTablePartition(String partIdent) {
-    ArrayList<Map<String, Object>> res = new ArrayList<Map<String, Object>>();
-
-    ArrayList<String> names = new ArrayList<String>();
-    for (String part : StringUtils.split(partIdent, "/")) {
-      String name = part;
-      String val = null;
-      String[] kv = StringUtils.split(part, "=", 2);
-      if (kv != null) {
-        name = kv[0];
-        if (kv.length > 1) {
-          try {
-            val = URLDecoder.decode(kv[1], StandardCharsets.UTF_8.name());
-          } catch (UnsupportedEncodingException e) {
-          }
-        }
-      }
-      if (val != null) {
-        names.add(name + "='" + val + "'");
-      }
-      else {
-        names.add(name);
-      }
-
-      res.add(MapBuilder.create()
-          .put("columnName", name)
-          .put("columnValue", val)
-          .build());
-    }
-
-    return MapBuilder.create()
-        .put("name", StringUtils.join(names, ","))
-        .put("values", res)
-        .build();
-  }
-
-  /**
-   * Show a list of databases
-   */
-  @Override
-  public void showDatabases(DataOutputStream out, List<String> databases)
-      throws HiveException {
-    asJson(out, MapBuilder.create().put("databases", databases).build());
-  }
-
-  /**
-   * Show the description of a database
-   */
-  @Override
-  public void showDatabaseDescription(DataOutputStream out, String database, String comment,
-      String location, String managedLocation, String ownerName, PrincipalType ownerType, String connectorName, String remoteDbName, Map<String, String> params)
-          throws HiveException {
-    MapBuilder builder = MapBuilder.create().put("database", database).put("comment", comment)
-        .put("location", location);
-    if (null != managedLocation) {
-      builder.put("managedLocation", managedLocation);
-    }
-    if (null != ownerName) {
-      builder.put("owner", ownerName);
-    }
-    if (null != ownerType) {
-      builder.put("ownerType", ownerType.name());
-    }
-    if (null != connectorName) {
-      builder.put("connector_name", connectorName);
-    }
-    if (null != remoteDbName) {
-      builder.put("remote_dbname", remoteDbName);
-    }
-    if (null != params && !params.isEmpty()) {
-      builder.put("params", params);
-    }
-    asJson(out, builder.build());
-  }
-
-  /**
-   * Show a list of dataconnectors
-   */
-  @Override
-  public void showDataConnectors(DataOutputStream out, List<String> connectors)
-      throws HiveException {
-    asJson(out, MapBuilder.create().put("connectors", connectors).build());
-  }
-
-  /**
-   * Show the description of a dataconnector
-   */
-  @Override
-  public void showDataConnectorDescription(DataOutputStream out, String dcName, String type,
-      String url, String ownerName, PrincipalType ownerType, String comment, Map<String, String> params)
-          throws HiveException {
-    MapBuilder builder = MapBuilder.create().put("connector", dcName).put("type", type)
-        .put("url", url);
-    if (null != ownerName) {
-      builder.put("owner", ownerName);
-    }
-    if (null != ownerType) {
-      builder.put("ownerType", ownerType.name());
-    }
-    if (null != comment) {
-      builder.put("comment", comment);
-    }
-    if (null != params && !params.isEmpty()) {
-      builder.put("params", params);
-    }
-    asJson(out, builder.build());
-  }
-
-  @Override
-  public void showResourcePlans(DataOutputStream out, List<WMResourcePlan> resourcePlans)
-      throws HiveException {
-    JsonGenerator generator = null;
-    try {
-      generator = new ObjectMapper().getJsonFactory().createJsonGenerator(out);
-      generator.writeStartArray();
-      for (WMResourcePlan plan : resourcePlans) {
-        generator.writeStartObject();
-        generator.writeStringField("name", plan.getName());
-        generator.writeStringField("status", plan.getStatus().name());
-        if (plan.isSetQueryParallelism()) {
-          generator.writeNumberField("queryParallelism", plan.getQueryParallelism());
-        }
-        if (plan.isSetDefaultPoolPath()) {
-          generator.writeStringField("defaultPoolPath", plan.getDefaultPoolPath());
-        }
-        generator.writeEndObject();
-      }
-      generator.writeEndArray();
-      generator.close();
-    } catch (IOException e) {
-      throw new HiveException(e);
-    } finally {
-      if (generator != null) {
-        IOUtils.closeQuietly(generator);
-      }
-    }
-  }
-
-  /**
-   * Formats a resource plan into a json object, the structure is as follows:
-   * {
-   *    name: "<rp_name>",
-   *    parallelism: "<parallelism>",
-   *    defaultQueue: "<defaultQueue>",
-   *    pools : [
-   *      {
-   *        name: "<pool_name>",
-   *        parallelism: "<parallelism>",
-   *        schedulingPolicy: "<policy>",
-   *        triggers: [
-   *          { name: "<triggerName>", trigger: "<trigExpression>", action: "<actionExpr">}
-   *          ...
-   *        ]
-   *      }
-   *      ...
-   *    ]
-   * }
-   */
-  private static class JsonRPFormatter implements MetaDataFormatUtils.RPFormatter, Closeable {
-    private final JsonGenerator generator;
-
-    JsonRPFormatter(DataOutputStream out) throws IOException {
-      generator = new ObjectMapper().getJsonFactory().createJsonGenerator(out);
-    }
-
-    private void writeNameAndFields(String name, Object ... kvPairs) throws IOException {
-      if (kvPairs.length % 2 != 0) {
-        throw new IllegalArgumentException("Expected pairs");
-      }
-      generator.writeStringField("name", name);
-      for (int i = 0; i < kvPairs.length; i += 2) {
-        generator.writeObjectField(kvPairs[i].toString(), kvPairs[i + 1]);
-      }
-    }
-
-    @Override
-    public void startRP(String rpName, Object ... kvPairs) throws IOException {
-      generator.writeStartObject();
-      writeNameAndFields(rpName, kvPairs);
-    }
-
-    @Override
-    public void endRP() throws IOException {
-      // End the root rp object.
-      generator.writeEndObject();
-    }
-
-    @Override
-    public void startPools() throws IOException {
-      generator.writeArrayFieldStart("pools");
-    }
-
-    @Override
-    public void endPools() throws IOException {
-      // End the pools array.
-      generator.writeEndArray();
-    }
-
-    @Override
-    public void startPool(String poolName, Object ... kvPairs) throws IOException {
-      generator.writeStartObject();
-      writeNameAndFields(poolName, kvPairs);
-    }
-
-    @Override
-    public void startTriggers() throws IOException {
-      generator.writeArrayFieldStart("triggers");
-    }
-
-    @Override
-    public void endTriggers() throws IOException {
-      generator.writeEndArray();
-    }
-
-    @Override
-    public void startMappings() throws IOException {
-      generator.writeArrayFieldStart("mappings");
-    }
-
-    @Override
-    public void endMappings() throws IOException {
-      generator.writeEndArray();
-    }
-
-    @Override
-    public void endPool() throws IOException {
-      generator.writeEndObject();
-    }
-
-    @Override
-    public void formatTrigger(String triggerName, String actionExpression,
-        String triggerExpression) throws IOException {
-      generator.writeStartObject();
-      writeNameAndFields(triggerName, "action", actionExpression, "trigger", triggerExpression);
-      generator.writeEndObject();
-    }
-
-    @Override
-    public void formatMappingType(String type, List<String> names) throws IOException {
-      generator.writeStartObject();
-      generator.writeStringField("type", type);
-      generator.writeArrayFieldStart("values");
-      for (String name : names) {
-        generator.writeString(name);
-      }
-      generator.writeEndArray();
-      generator.writeEndObject();
-    }
-
-    @Override
-    public void close() throws IOException {
-      generator.close();
-    }
-  }
-
-  public void showFullResourcePlan(DataOutputStream out, WMFullResourcePlan resourcePlan)
-      throws HiveException {
-    try (JsonRPFormatter formatter = new JsonRPFormatter(out)) {
-      MetaDataFormatUtils.formatFullRP(formatter, resourcePlan);
-    } catch (IOException e) {
-      throw new HiveException(e);
-    }
->>>>>>> External metastore: clean after rebase
-  }
-
   @Override
   public void showErrors(DataOutputStream out, WMValidateResourcePlanResponse response) throws HiveException {
     try (JsonGenerator generator = new ObjectMapper().getJsonFactory().createJsonGenerator(out)) {
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index d66d928..2af557e 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -15,10374 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore;
-
-import static org.apache.commons.lang3.StringUtils.join;
-import static org.apache.commons.lang3.StringUtils.isBlank;
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_COMMENT;
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
-import static org.apache.hadoop.hive.metastore.Warehouse.getCatalogQualifiedTableName;
-import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
-import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.parseDbName;
-import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.CAT_NAME;
-import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.DB_NAME;
-import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.prependCatalogToDbName;
-import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.prependNotNullCatToDbName;
-
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Modifier;
-import java.lang.reflect.UndeclaredThrowableException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedExceptionAction;
-import java.util.AbstractMap;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.regex.Pattern;
-
-import javax.jdo.JDOException;
-
-import com.codahale.metrics.Counter;
-import com.google.common.base.Supplier;
-import com.google.common.base.Suppliers;
-import com.google.common.collect.Lists;
-
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.collections.CollectionUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.AcidConstants;
-import org.apache.hadoop.hive.common.AcidMetaDataFile;
-import org.apache.hadoop.hive.common.StatsSetupConst;
-import org.apache.hadoop.hive.common.TableName;
-import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
-import org.apache.hadoop.hive.common.ValidWriteIdList;
-import org.apache.hadoop.hive.common.ZooKeeperHiveHelper;
-import org.apache.hadoop.hive.common.ZKDeRegisterWatcher;
-import org.apache.hadoop.hive.common.repl.ReplConst;
-import org.apache.hadoop.hive.metastore.api.*;
-import org.apache.hadoop.hive.metastore.dataconnector.DataConnectorProviderFactory;
-import org.apache.hadoop.hive.metastore.events.*;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf.StatsUpdateMode;
-import org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
-import org.apache.hadoop.hive.metastore.metrics.JvmPauseMonitor;
-import org.apache.hadoop.hive.metastore.metrics.Metrics;
-import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
-import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
-import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.metastore.security.MetastoreDelegationTokenManager;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
-import org.apache.hadoop.hive.metastore.txn.TxnStore;
-import org.apache.hadoop.hive.metastore.txn.TxnUtils;
-import org.apache.hadoop.hive.metastore.utils.FilterUtils;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.hive.metastore.utils.CommonCliOptions;
-import org.apache.hadoop.hive.metastore.utils.FileUtils;
-import org.apache.hadoop.hive.metastore.utils.HdfsUtils;
-import org.apache.hadoop.hive.metastore.utils.JavaUtils;
-import org.apache.hadoop.hive.metastore.utils.LogUtils;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.utils.MetastoreVersionInfo;
-import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.core.LoggerContext;
-import org.apache.thrift.TException;
-import org.apache.thrift.TProcessor;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.protocol.TProtocolFactory;
-import org.apache.thrift.server.ServerContext;
-import org.apache.thrift.server.TServer;
-import org.apache.thrift.server.TServerEventHandler;
-import org.apache.thrift.server.TThreadPoolServer;
-import org.apache.thrift.transport.TServerSocket;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportFactory;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.facebook.fb303.FacebookBase;
-import com.facebook.fb303.fb_status;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Splitter;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * TODO:pc remove application logic to a separate interface.
- */
-public class HiveMetaStore extends ThriftHiveMetastore {
-  public static final Logger LOG = LoggerFactory.getLogger(HiveMetaStore.class);
-  public static final String PARTITION_NUMBER_EXCEED_LIMIT_MSG =
-      "Number of partitions scanned (=%d) on table '%s' exceeds limit (=%d). This is controlled on the metastore server by %s.";
-
-  // boolean that tells if the HiveMetaStore (remote) server is being used.
-  // Can be used to determine if the calls to metastore api (HMSHandler) are being made with
-  // embedded metastore or a remote one
-  private static boolean isMetaStoreRemote = false;
-
-  // Used for testing to simulate method timeout.
-  @VisibleForTesting
-  static boolean TEST_TIMEOUT_ENABLED = false;
-  @VisibleForTesting
-  static long TEST_TIMEOUT_VALUE = -1;
-
-  private static ShutdownHookManager shutdownHookMgr;
-
-  public static final String ADMIN = "admin";
-  public static final String PUBLIC = "public";
-  /** MM write states. */
-  public static final char MM_WRITE_OPEN = 'o', MM_WRITE_COMMITTED = 'c', MM_WRITE_ABORTED = 'a';
-
-  private static HadoopThriftAuthBridge.Server saslServer;
-  private static MetastoreDelegationTokenManager delegationTokenManager;
-  private static boolean useSasl;
-
-  static final String NO_FILTER_STRING = "";
-  static final int UNLIMITED_MAX_PARTITIONS = -1;
-  private static ZooKeeperHiveHelper zooKeeperHelper = null;
-  private static String msHost = null;
-
-  public static boolean isRenameAllowed(Database srcDB, Database destDB) {
-    if (!srcDB.getName().equalsIgnoreCase(destDB.getName())) {
-      if (ReplChangeManager.isSourceOfReplication(srcDB) || ReplChangeManager.isSourceOfReplication(destDB)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  public static class HMSHandler extends FacebookBase implements IHMSHandler {
-    public static final Logger LOG = HiveMetaStore.LOG;
-    private final Configuration conf; // stores datastore (jpox) properties,
-                                     // right now they come from jpox.properties
-
-    // Flag to control that always threads are initialized only once
-    // instead of multiple times
-    private final static AtomicBoolean alwaysThreadsInitialized =
-        new AtomicBoolean(false);
-
-    private static String currentUrl;
-    private FileMetadataManager fileMetadataManager;
-    private PartitionExpressionProxy expressionProxy;
-    private StorageSchemaReader storageSchemaReader;
-    private IMetaStoreMetadataTransformer transformer;
-    private static DataConnectorProviderFactory dataconnectorFactory = null;
-
-    // Variables for metrics
-    // Package visible so that HMSMetricsListener can see them.
-    static AtomicInteger databaseCount, tableCount, partCount;
-
-    private Warehouse wh; // hdfs warehouse
-    private static final ThreadLocal<RawStore> threadLocalMS =
-        new ThreadLocal<RawStore>() {
-          @Override
-          protected RawStore initialValue() {
-            return null;
-          }
-        };
-
-    private static final ThreadLocal<TxnStore> threadLocalTxn = new ThreadLocal<TxnStore>() {
-      @Override
-      protected TxnStore initialValue() {
-        return null;
-      }
-    };
-
-    private static final ThreadLocal<Map<String, com.codahale.metrics.Timer.Context>> timerContexts =
-        new ThreadLocal<Map<String, com.codahale.metrics.Timer.Context>>() {
-      @Override
-      protected Map<String, com.codahale.metrics.Timer.Context> initialValue() {
-        return new HashMap<>();
-      }
-    };
-
-    public static RawStore getRawStore() {
-      return threadLocalMS.get();
-    }
-
-    static void removeRawStore() {
-      threadLocalMS.remove();
-    }
-
-    // Thread local configuration is needed as many threads could make changes
-    // to the conf using the connection hook
-    private static final ThreadLocal<Configuration> threadLocalConf =
-        new ThreadLocal<Configuration>() {
-          @Override
-          protected Configuration initialValue() {
-            return null;
-          }
-        };
-
-    /**
-     * Thread local HMSHandler used during shutdown to notify meta listeners
-     */
-    private static final ThreadLocal<HMSHandler> threadLocalHMSHandler = new ThreadLocal<>();
-
-    /**
-     * Thread local Map to keep track of modified meta conf keys
-     */
-    private static final ThreadLocal<Map<String, String>> threadLocalModifiedConfig =
-        new ThreadLocal<>();
-
-    private static ExecutorService threadPool;
-
-    static final Logger auditLog = LoggerFactory.getLogger(
-        HiveMetaStore.class.getName() + ".audit");
-
-    private static void logAuditEvent(String cmd) {
-      if (cmd == null) {
-        return;
-      }
-
-      UserGroupInformation ugi;
-      try {
-        ugi = SecurityUtils.getUGI();
-      } catch (Exception ex) {
-        throw new RuntimeException(ex);
-      }
-
-      String address = getIPAddress();
-      if (address == null) {
-        address = "unknown-ip-addr";
-      }
-
-      auditLog.info("ugi={}	ip={}	cmd={}	", ugi.getUserName(), address, cmd);
-    }
-
-    public static String getIPAddress() {
-      if (useSasl) {
-        if (saslServer != null && saslServer.getRemoteAddress() != null) {
-          return saslServer.getRemoteAddress().getHostAddress();
-        }
-      } else {
-        // if kerberos is not enabled
-        return getThreadLocalIpAddress();
-      }
-      return null;
-    }
-
-    private static AtomicInteger nextSerialNum = new AtomicInteger();
-    private static ThreadLocal<Integer> threadLocalId = new ThreadLocal<Integer>() {
-      @Override
-      protected Integer initialValue() {
-        return nextSerialNum.getAndIncrement();
-      }
-    };
-
-    // This will only be set if the metastore is being accessed from a metastore Thrift server,
-    // not if it is from the CLI. Also, only if the TTransport being used to connect is an
-    // instance of TSocket. This is also not set when kerberos is used.
-    private static ThreadLocal<String> threadLocalIpAddress = new ThreadLocal<String>() {
-      @Override
-      protected String initialValue() {
-        return null;
-      }
-    };
-
-    /**
-     * Internal function to notify listeners for meta config change events
-     */
-    private void notifyMetaListeners(String key, String oldValue, String newValue) throws MetaException {
-      for (MetaStoreEventListener listener : listeners) {
-        listener.onConfigChange(new ConfigChangeEvent(this, key, oldValue, newValue));
-      }
-
-      if (transactionalListeners.size() > 0) {
-        // All the fields of this event are final, so no reason to create a new one for each
-        // listener
-        ConfigChangeEvent cce = new ConfigChangeEvent(this, key, oldValue, newValue);
-        for (MetaStoreEventListener transactionalListener : transactionalListeners) {
-          transactionalListener.onConfigChange(cce);
-        }
-      }
-    }
-
-    /**
-     * Internal function to notify listeners to revert back to old values of keys
-     * that were modified during setMetaConf. This would get called from HiveMetaStore#cleanupRawStore
-     */
-    private void notifyMetaListenersOnShutDown() {
-      Map<String, String> modifiedConf = threadLocalModifiedConfig.get();
-      if (modifiedConf == null) {
-        // Nothing got modified
-        return;
-      }
-      try {
-        Configuration conf = threadLocalConf.get();
-        if (conf == null) {
-          throw new MetaException("Unexpected: modifiedConf is non-null but conf is null");
-        }
-        // Notify listeners of the changed value
-        for (Entry<String, String> entry : modifiedConf.entrySet()) {
-          String key = entry.getKey();
-          // curr value becomes old and vice-versa
-          String currVal = entry.getValue();
-          String oldVal = conf.get(key);
-          if (!Objects.equals(oldVal, currVal)) {
-            notifyMetaListeners(key, oldVal, currVal);
-          }
-        }
-        logAndAudit("Meta listeners shutdown notification completed.");
-      } catch (MetaException e) {
-        LOG.error("Failed to notify meta listeners on shutdown: ", e);
-      }
-    }
-
-    static void setThreadLocalIpAddress(String ipAddress) {
-      threadLocalIpAddress.set(ipAddress);
-    }
-
-    // This will return null if the metastore is not being accessed from a metastore Thrift server,
-    // or if the TTransport being used to connect is not an instance of TSocket, or if kereberos
-    // is used
-    static String getThreadLocalIpAddress() {
-      return threadLocalIpAddress.get();
-    }
-
-    // Make it possible for tests to check that the right type of PartitionExpressionProxy was
-    // instantiated.
-    @VisibleForTesting
-    PartitionExpressionProxy getExpressionProxy() {
-      return expressionProxy;
-    }
-
-    /**
-     * Use {@link #getThreadId()} instead.
-     * @return thread id
-     */
-    @Deprecated
-    public static Integer get() {
-      return threadLocalId.get();
-    }
-
-    @Override
-    public int getThreadId() {
-      return threadLocalId.get();
-    }
-
-    public HMSHandler(String name) throws MetaException {
-      this(name, MetastoreConf.newMetastoreConf(), true);
-    }
-
-    public HMSHandler(String name, Configuration conf) throws MetaException {
-      this(name, conf, true);
-    }
-
-    public HMSHandler(String name, Configuration conf, boolean init) throws MetaException {
-      super(name);
-      this.conf = conf;
-      isInTest = MetastoreConf.getBoolVar(this.conf, ConfVars.HIVE_IN_TEST);
-      if (threadPool == null) {
-        synchronized (HMSHandler.class) {
-          int numThreads = MetastoreConf.getIntVar(conf, ConfVars.FS_HANDLER_THREADS_COUNT);
-          threadPool = Executors.newFixedThreadPool(numThreads,
-              new ThreadFactoryBuilder().setDaemon(true)
-                  .setNameFormat("HMSHandler #%d").build());
-        }
-      }
-      if (init) {
-        init();
-      }
-    }
-
-    /**
-     * Use {@link #getConf()} instead.
-     * @return Configuration object
-     */
-    @Deprecated
-    public Configuration getHiveConf() {
-      return conf;
-    }
-
-    private ClassLoader classLoader;
-    private AlterHandler alterHandler;
-    private List<MetaStorePreEventListener> preListeners;
-    private List<MetaStoreEventListener> listeners;
-    private List<TransactionalMetaStoreEventListener> transactionalListeners;
-    private List<MetaStoreEndFunctionListener> endFunctionListeners;
-    private List<MetaStoreInitListener> initListeners;
-    private MetaStoreFilterHook filterHook;
-    private boolean isServerFilterEnabled = false;
-
-    private Pattern partitionValidationPattern;
-    private final boolean isInTest;
-
-    {
-      classLoader = Thread.currentThread().getContextClassLoader();
-      if (classLoader == null) {
-        classLoader = Configuration.class.getClassLoader();
-      }
-    }
-
-    @Override
-    public List<TransactionalMetaStoreEventListener> getTransactionalListeners() {
-      return transactionalListeners;
-    }
-
-    @Override
-    public List<MetaStoreEventListener> getListeners() {
-      return listeners;
-    }
-
-    @Override
-    public void init() throws MetaException {
-      initListeners = MetaStoreServerUtils.getMetaStoreListeners(
-          MetaStoreInitListener.class, conf, MetastoreConf.getVar(conf, ConfVars.INIT_HOOKS));
-      for (MetaStoreInitListener singleInitListener: initListeners) {
-          MetaStoreInitContext context = new MetaStoreInitContext();
-          singleInitListener.onInit(context);
-      }
-
-      String alterHandlerName = MetastoreConf.getVar(conf, ConfVars.ALTER_HANDLER);
-      alterHandler = ReflectionUtils.newInstance(JavaUtils.getClass(
-          alterHandlerName, AlterHandler.class), conf);
-      wh = new Warehouse(conf);
-
-      synchronized (HMSHandler.class) {
-        if (currentUrl == null || !currentUrl.equals(MetaStoreInit.getConnectionURL(conf))) {
-          createDefaultDB();
-          createDefaultRoles();
-          addAdminUsers();
-          currentUrl = MetaStoreInit.getConnectionURL(conf);
-        }
-      }
-
-      //Start Metrics
-      if (MetastoreConf.getBoolVar(conf, ConfVars.METRICS_ENABLED)) {
-        LOG.info("Begin calculating metadata count metrics.");
-        Metrics.initialize(conf);
-        databaseCount = Metrics.getOrCreateGauge(MetricsConstants.TOTAL_DATABASES);
-        tableCount = Metrics.getOrCreateGauge(MetricsConstants.TOTAL_TABLES);
-        partCount = Metrics.getOrCreateGauge(MetricsConstants.TOTAL_PARTITIONS);
-        updateMetrics();
-
-      }
-
-      preListeners = MetaStoreServerUtils.getMetaStoreListeners(MetaStorePreEventListener.class,
-          conf, MetastoreConf.getVar(conf, ConfVars.PRE_EVENT_LISTENERS));
-      preListeners.add(0, new TransactionalValidationListener(conf));
-      listeners = MetaStoreServerUtils.getMetaStoreListeners(MetaStoreEventListener.class, conf,
-          MetastoreConf.getVar(conf, ConfVars.EVENT_LISTENERS));
-      listeners.add(new SessionPropertiesListener(conf));
-      transactionalListeners = new ArrayList() {{
-          add(new AcidEventListener(conf));
-      }};
-      transactionalListeners.addAll(MetaStoreServerUtils.getMetaStoreListeners(
-              TransactionalMetaStoreEventListener.class, conf,
-              MetastoreConf.getVar(conf, ConfVars.TRANSACTIONAL_EVENT_LISTENERS)));
-      if (Metrics.getRegistry() != null) {
-        listeners.add(new HMSMetricsListener(conf));
-      }
-
-      boolean canCachedStoreCanUseEvent = false;
-      for (MetaStoreEventListener listener : transactionalListeners) {
-        if (listener.doesAddEventsToNotificationLogTable()) {
-          canCachedStoreCanUseEvent = true;
-          break;
-        }
-      }
-      if (conf.getBoolean(ConfVars.METASTORE_CACHE_CAN_USE_EVENT.getVarname(), false) &&
-              !canCachedStoreCanUseEvent) {
-        throw new MetaException("CahcedStore can not use events for invalidation as there is no " +
-                " TransactionalMetaStoreEventListener to add events to notification table");
-      }
-
-      endFunctionListeners = MetaStoreServerUtils.getMetaStoreListeners(
-          MetaStoreEndFunctionListener.class, conf, MetastoreConf.getVar(conf, ConfVars.END_FUNCTION_LISTENERS));
-
-      String partitionValidationRegex =
-          MetastoreConf.getVar(conf, ConfVars.PARTITION_NAME_WHITELIST_PATTERN);
-      if (partitionValidationRegex != null && !partitionValidationRegex.isEmpty()) {
-        partitionValidationPattern = Pattern.compile(partitionValidationRegex);
-      } else {
-        partitionValidationPattern = null;
-      }
-
-      // We only initialize once the tasks that need to be run periodically. For remote metastore
-      // these threads are started along with the other housekeeping threads only in the leader
-      // HMS.
-      String leaderHost = MetastoreConf.getVar(conf,
-              MetastoreConf.ConfVars.METASTORE_HOUSEKEEPING_LEADER_HOSTNAME);
-      if (!isMetaStoreRemote() && ((leaderHost == null) || leaderHost.trim().isEmpty())) {
-        startAlwaysTaskThreads(conf);
-      } else if (!isMetaStoreRemote()) {
-        LOG.info("Not starting tasks specified by " + ConfVars.TASK_THREADS_ALWAYS.getVarname() +
-                " since " + leaderHost + " is configured to run these tasks.");
-      }
-      expressionProxy = PartFilterExprUtil.createExpressionProxy(conf);
-      fileMetadataManager = new FileMetadataManager(this.getMS(), conf);
-
-      isServerFilterEnabled = getIfServerFilterenabled();
-      filterHook = isServerFilterEnabled ? loadFilterHooks() : null;
-
-      String className = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.METASTORE_METADATA_TRANSFORMER_CLASS);
-      if (className != null && !className.trim().isEmpty()) {
-        Class<?> clazz;
-        try {
-          clazz = conf.getClassByName(className);
-        } catch (ClassNotFoundException e) {
-          LOG.error("Unable to load class " + className, e);
-          throw new IllegalArgumentException(e);
-        }
-        Constructor<?> constructor;
-        try {
-          constructor = clazz.getConstructor(IHMSHandler.class);
-          if (Modifier.isPrivate(constructor.getModifiers()))
-            throw new IllegalArgumentException("Illegal implementation for metadata transformer. Constructor is private");
-          transformer = (IMetaStoreMetadataTransformer) constructor.newInstance(this);
-        } catch (NoSuchMethodException | InstantiationException | IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
-          LOG.error("Unable to create instance of class " + className, e);
-          throw new IllegalArgumentException(e);
-        }
-      }
-      dataconnectorFactory = DataConnectorProviderFactory.getInstance(this);
-    }
-
-    private static void startAlwaysTaskThreads(Configuration conf) throws MetaException {
-      if (alwaysThreadsInitialized.compareAndSet(false, true)) {
-        ThreadPool.initialize(conf);
-        Collection<String> taskNames =
-                MetastoreConf.getStringCollection(conf, ConfVars.TASK_THREADS_ALWAYS);
-        for (String taskName : taskNames) {
-          MetastoreTaskThread task =
-                  JavaUtils.newInstance(JavaUtils.getClass(taskName, MetastoreTaskThread.class));
-          task.setConf(conf);
-          long freq = task.runFrequency(TimeUnit.MILLISECONDS);
-          LOG.info("Scheduling for " + task.getClass().getCanonicalName() + " service with " +
-                  "frequency " + freq + "ms.");
-          // For backwards compatibility, since some threads used to be hard coded but only run if
-          // frequency was > 0
-          if (freq > 0) {
-            ThreadPool.getPool().scheduleAtFixedRate(task, freq, freq, TimeUnit.MILLISECONDS);
-          }
-        }
-      }
-    }
-
-    /**
-     *
-     * Filter is actually enabled only when the configured filter hook is configured, not default, and
-     * enabled in configuration
-     * @return
-     */
-    private boolean getIfServerFilterenabled() throws MetaException{
-      boolean isEnabled = MetastoreConf.getBoolVar(conf, ConfVars.METASTORE_SERVER_FILTER_ENABLED);
-
-      if (!isEnabled) {
-        LOG.info("HMS server filtering is disabled by configuration");
-        return false;
-      }
-
-      String filterHookClassName = MetastoreConf.getVar(conf, ConfVars.FILTER_HOOK);
-
-      if (isBlank(filterHookClassName)) {
-        throw new MetaException("HMS server filtering is enabled but no filter hook is configured");
-      }
-
-      if (filterHookClassName.trim().equalsIgnoreCase(DefaultMetaStoreFilterHookImpl.class.getName())) {
-        throw new MetaException("HMS server filtering is enabled but the filter hook is DefaultMetaStoreFilterHookImpl, which does no filtering");
-      }
-
-      LOG.info("HMS server filtering is enabled. The filter class is " + filterHookClassName);
-      return true;
-    }
-
-    private MetaStoreFilterHook loadFilterHooks() throws IllegalStateException  {
-      String errorMsg = "Unable to load filter hook at HMS server. ";
-
-      String filterHookClassName = MetastoreConf.getVar(conf, ConfVars.FILTER_HOOK);
-      Preconditions.checkState(!isBlank(filterHookClassName));
-
-      try {
-        return (MetaStoreFilterHook)Class.forName(
-            filterHookClassName.trim(), true, JavaUtils.getClassLoader()).getConstructor(
-            Configuration.class).newInstance(conf);
-      } catch (Exception e) {
-        LOG.error(errorMsg, e);
-        throw new IllegalStateException(errorMsg + e.getMessage(), e);
-      }
-    }
-
-    /**
-     * Check if user can access the table associated with the partition. If not, then throw exception
-     * so user cannot access partitions associated with this table
-     * We are not calling Pre event listener for authorization because it requires getting the
-     * table object from DB, more overhead. Instead ,we call filter hook to filter out table if user
-     * has no access. Filter hook only requires table name, not table object. That saves DB access for
-     * table object, and still achieve the same purpose: checking if user can access the specified
-     * table
-     *
-     * @param catName catalog name of the table
-     * @param dbName database name of the table
-     * @param tblName table name
-     * @throws NoSuchObjectException
-     * @throws MetaException
-     */
-    private void authorizeTableForPartitionMetadata(
-        final String catName, final String dbName, final String tblName)
-        throws NoSuchObjectException, MetaException {
-
-      FilterUtils.checkDbAndTableFilters(
-          isServerFilterEnabled, filterHook, catName, dbName, tblName);
-    }
-
-    private static String addPrefix(String s) {
-      return threadLocalId.get() + ": " + s;
-    }
-
-    /**
-     * Set copy of invoking HMSHandler on thread local
-     */
-    private static void setHMSHandler(HMSHandler handler) {
-      if (threadLocalHMSHandler.get() == null) {
-        threadLocalHMSHandler.set(handler);
-      }
-    }
-    @Override
-    public void setConf(Configuration conf) {
-      threadLocalConf.set(conf);
-      RawStore ms = threadLocalMS.get();
-      if (ms != null) {
-        ms.setConf(conf); // reload if DS related configuration is changed
-      }
-    }
-
-    @Override
-    public Configuration getConf() {
-      Configuration conf = threadLocalConf.get();
-      if (conf == null) {
-        conf = new Configuration(this.conf);
-        threadLocalConf.set(conf);
-      }
-      return conf;
-    }
-
-    private Map<String, String> getModifiedConf() {
-      Map<String, String> modifiedConf = threadLocalModifiedConfig.get();
-      if (modifiedConf == null) {
-        modifiedConf = new HashMap<>();
-        threadLocalModifiedConfig.set(modifiedConf);
-      }
-      return modifiedConf;
-    }
-
-    @Override
-    public Warehouse getWh() {
-      return wh;
-    }
-
-    @Override
-    public void setMetaConf(String key, String value) throws MetaException {
-      ConfVars confVar = MetastoreConf.getMetaConf(key);
-      if (confVar == null) {
-        throw new MetaException("Invalid configuration key " + key);
-      }
-      try {
-        confVar.validate(value);
-      } catch (IllegalArgumentException e) {
-        throw new MetaException("Invalid configuration value " + value + " for key " + key +
-            " by " + e.getMessage());
-      }
-      Configuration configuration = getConf();
-      String oldValue = MetastoreConf.get(configuration, key);
-      // Save prev val of the key on threadLocal
-      Map<String, String> modifiedConf = getModifiedConf();
-      if (!modifiedConf.containsKey(key)) {
-        modifiedConf.put(key, oldValue);
-      }
-      // Set invoking HMSHandler on threadLocal, this will be used later to notify
-      // metaListeners in HiveMetaStore#cleanupRawStore
-      setHMSHandler(this);
-      configuration.set(key, value);
-      notifyMetaListeners(key, oldValue, value);
-
-      if (ConfVars.TRY_DIRECT_SQL == confVar) {
-        HMSHandler.LOG.info("Direct SQL optimization = {}",  value);
-      }
-    }
-
-    @Override
-    public String getMetaConf(String key) throws MetaException {
-      ConfVars confVar = MetastoreConf.getMetaConf(key);
-      if (confVar == null) {
-        throw new MetaException("Invalid configuration key " + key);
-      }
-      return getConf().get(key, confVar.getDefaultVal().toString());
-    }
-
-    /**
-     * Get a cached RawStore.
-     *
-     * @return the cached RawStore
-     * @throws MetaException
-     */
-    @Override
-    public RawStore getMS() throws MetaException {
-      Configuration conf = getConf();
-      return getMSForConf(conf);
-    }
-
-    public static RawStore getMSForConf(Configuration conf) throws MetaException {
-      RawStore ms = threadLocalMS.get();
-      if (ms == null) {
-        ms = newRawStoreForConf(conf);
-        try {
-          ms.verifySchema();
-        } catch (MetaException e) {
-          ms.shutdown();
-          throw e;
-        }
-        threadLocalMS.set(ms);
-        ms = threadLocalMS.get();
-        LOG.info("Created RawStore: " + ms + " from thread id: " + Thread.currentThread().getId());
-      }
-      return ms;
-    }
-
-    @Override
-    public TxnStore getTxnHandler() {
-      return getMsThreadTxnHandler(conf);
-    }
-
-    public static TxnStore getMsThreadTxnHandler(Configuration conf) {
-      TxnStore txn = threadLocalTxn.get();
-      if (txn == null) {
-        txn = TxnUtils.getTxnStore(conf);
-        threadLocalTxn.set(txn);
-      }
-      return txn;
-    }
-
-    static RawStore newRawStoreForConf(Configuration conf) throws MetaException {
-      Configuration newConf = new Configuration(conf);
-      String rawStoreClassName = MetastoreConf.getVar(newConf, ConfVars.RAW_STORE_IMPL);
-      LOG.info(addPrefix("Opening raw store with implementation class:" + rawStoreClassName));
-      return RawStoreProxy.getProxy(newConf, conf, rawStoreClassName, threadLocalId.get());
-    }
-
-    @VisibleForTesting
-    public static void createDefaultCatalog(RawStore ms, Warehouse wh) throws MetaException,
-        InvalidOperationException {
-      try {
-        Catalog defaultCat = ms.getCatalog(DEFAULT_CATALOG_NAME);
-        // Null check because in some test cases we get a null from ms.getCatalog.
-        if (defaultCat !=null && defaultCat.getLocationUri().equals("TBD")) {
-          // One time update issue.  When the new 'hive' catalog is created in an upgrade the
-          // script does not know the location of the warehouse.  So we need to update it.
-          LOG.info("Setting location of default catalog, as it hasn't been done after upgrade");
-          defaultCat.setLocationUri(wh.getWhRoot().toString());
-          ms.alterCatalog(defaultCat.getName(), defaultCat);
-        }
-
-      } catch (NoSuchObjectException e) {
-        Catalog cat = new Catalog(DEFAULT_CATALOG_NAME, wh.getWhRoot().toString());
-        long time = System.currentTimeMillis() / 1000;
-        cat.setCreateTime((int) time);
-        cat.setDescription(Warehouse.DEFAULT_CATALOG_COMMENT);
-        ms.createCatalog(cat);
-      }
-    }
-
-    private void createDefaultDB_core(RawStore ms) throws MetaException, InvalidObjectException {
-      try {
-        ms.getDatabase(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME);
-      } catch (NoSuchObjectException e) {
-        LOG.info("Started creating a default database with name: "+DEFAULT_DATABASE_NAME);
-        Database db = new Database(DEFAULT_DATABASE_NAME, DEFAULT_DATABASE_COMMENT,
-            wh.getDefaultDatabasePath(DEFAULT_DATABASE_NAME, true).toString(), null);
-        db.setOwnerName(PUBLIC);
-        db.setOwnerType(PrincipalType.ROLE);
-        db.setCatalogName(DEFAULT_CATALOG_NAME);
-        long time = System.currentTimeMillis() / 1000;
-        db.setCreateTime((int) time);
-        ms.createDatabase(db);
-        LOG.info("Successfully created a default database with name: "+DEFAULT_DATABASE_NAME);
-      }
-    }
-
-    /**
-     * create default database if it doesn't exist.
-     *
-     * This is a potential contention when HiveServer2 using embedded metastore and Metastore
-     * Server try to concurrently invoke createDefaultDB. If one failed, JDOException was caught
-     * for one more time try, if failed again, simply ignored by warning, which meant another
-     * succeeds.
-     *
-     * @throws MetaException
-     */
-    private void createDefaultDB() throws MetaException {
-      try {
-        RawStore ms = getMS();
-        createDefaultCatalog(ms, wh);
-        createDefaultDB_core(ms);
-      } catch (JDOException e) {
-        LOG.warn("Retrying creating default database after error: " + e.getMessage(), e);
-        try {
-          RawStore ms = getMS();
-          createDefaultCatalog(ms, wh);
-          createDefaultDB_core(ms);
-        } catch (InvalidObjectException | InvalidOperationException e1) {
-          throw new MetaException(e1.getMessage());
-        }
-      } catch (InvalidObjectException|InvalidOperationException e) {
-        throw new MetaException(e.getMessage());
-      }
-    }
-
-    /**
-     * create default roles if they don't exist.
-     *
-     * This is a potential contention when HiveServer2 using embedded metastore and Metastore
-     * Server try to concurrently invoke createDefaultRoles. If one failed, JDOException was caught
-     * for one more time try, if failed again, simply ignored by warning, which meant another
-     * succeeds.
-     *
-     * @throws MetaException
-     */
-    private void createDefaultRoles() throws MetaException {
-      try {
-        createDefaultRoles_core();
-      } catch (JDOException e) {
-        LOG.warn("Retrying creating default roles after error: " + e.getMessage(), e);
-        createDefaultRoles_core();
-      }
-    }
-
-    private void createDefaultRoles_core() throws MetaException {
-
-      RawStore ms = getMS();
-      try {
-        ms.addRole(ADMIN, ADMIN);
-      } catch (InvalidObjectException e) {
-        LOG.debug(ADMIN +" role already exists",e);
-      } catch (NoSuchObjectException e) {
-        // This should never be thrown.
-        LOG.warn("Unexpected exception while adding " +ADMIN+" roles" , e);
-      }
-      LOG.info("Added "+ ADMIN+ " role in metastore");
-      try {
-        ms.addRole(PUBLIC, PUBLIC);
-      } catch (InvalidObjectException e) {
-        LOG.debug(PUBLIC + " role already exists",e);
-      } catch (NoSuchObjectException e) {
-        // This should never be thrown.
-        LOG.warn("Unexpected exception while adding "+PUBLIC +" roles" , e);
-      }
-      LOG.info("Added "+PUBLIC+ " role in metastore");
-      // now grant all privs to admin
-      PrivilegeBag privs = new PrivilegeBag();
-      privs.addToPrivileges(new HiveObjectPrivilege( new HiveObjectRef(HiveObjectType.GLOBAL, null,
-        null, null, null), ADMIN, PrincipalType.ROLE, new PrivilegeGrantInfo("All", 0, ADMIN,
-          PrincipalType.ROLE, true), "SQL"));
-      try {
-        ms.grantPrivileges(privs);
-      } catch (InvalidObjectException e) {
-        // Surprisingly these privs are already granted.
-        LOG.debug("Failed while granting global privs to admin", e);
-      } catch (NoSuchObjectException e) {
-        // Unlikely to be thrown.
-        LOG.warn("Failed while granting global privs to admin", e);
-      }
-    }
-
-    /**
-     * add admin users if they don't exist.
-     *
-     * This is a potential contention when HiveServer2 using embedded metastore and Metastore
-     * Server try to concurrently invoke addAdminUsers. If one failed, JDOException was caught for
-     * one more time try, if failed again, simply ignored by warning, which meant another succeeds.
-     *
-     * @throws MetaException
-     */
-    private void addAdminUsers() throws MetaException {
-      try {
-        addAdminUsers_core();
-      } catch (JDOException e) {
-        LOG.warn("Retrying adding admin users after error: " + e.getMessage(), e);
-        addAdminUsers_core();
-      }
-    }
-
-    private void addAdminUsers_core() throws MetaException {
-
-      // now add pre-configured users to admin role
-      String userStr = MetastoreConf.getVar(conf,ConfVars.USERS_IN_ADMIN_ROLE,"").trim();
-      if (userStr.isEmpty()) {
-        LOG.info("No user is added in admin role, since config is empty");
-        return;
-      }
-      // Since user names need to be valid unix user names, per IEEE Std 1003.1-2001 they cannot
-      // contain comma, so we can safely split above string on comma.
-
-     Iterator<String> users = Splitter.on(",").trimResults().omitEmptyStrings().split(userStr).iterator();
-      if (!users.hasNext()) {
-        LOG.info("No user is added in admin role, since config value "+ userStr +
-          " is in incorrect format. We accept comma separated list of users.");
-        return;
-      }
-      Role adminRole;
-      RawStore ms = getMS();
-      try {
-        adminRole = ms.getRole(ADMIN);
-      } catch (NoSuchObjectException e) {
-        LOG.error("Failed to retrieve just added admin role",e);
-        return;
-      }
-      while (users.hasNext()) {
-        String userName = users.next();
-        try {
-          ms.grantRole(adminRole, userName, PrincipalType.USER, ADMIN, PrincipalType.ROLE, true);
-          LOG.info("Added " + userName + " to admin role");
-        } catch (NoSuchObjectException e) {
-          LOG.error("Failed to add "+ userName + " in admin role",e);
-        } catch (InvalidObjectException e) {
-          LOG.debug(userName + " already in admin role", e);
-        }
-      }
-    }
-
-    private static void logAndAudit(final String m) {
-      LOG.debug("{}: {}", threadLocalId.get(), m);
-      logAuditEvent(m);
-    }
-
-    private String startFunction(String function, String extraLogInfo) {
-      incrementCounter(function);
-      logAndAudit((getThreadLocalIpAddress() == null ? "" : "source:" + getThreadLocalIpAddress() + " ") +
-          function + extraLogInfo);
-      com.codahale.metrics.Timer timer =
-          Metrics.getOrCreateTimer(MetricsConstants.API_PREFIX + function);
-      if (timer != null) {
-        // Timer will be null we aren't using the metrics
-        timerContexts.get().put(function, timer.time());
-      }
-      Counter counter = Metrics.getOrCreateCounter(MetricsConstants.ACTIVE_CALLS + function);
-      if (counter != null) {
-        counter.inc();
-      }
-      return function;
-    }
-
-    private String startFunction(String function) {
-      return startFunction(function, "");
-    }
-
-    private void startTableFunction(String function, String catName, String db, String tbl) {
-      startFunction(function, " : tbl=" +
-          TableName.getQualified(catName, db, tbl));
-    }
-
-    private void startMultiTableFunction(String function, String db, List<String> tbls) {
-      String tableNames = join(tbls, ",");
-      startFunction(function, " : db=" + db + " tbls=" + tableNames);
-    }
-
-    private void startPartitionFunction(String function, String cat, String db, String tbl,
-                                        List<String> partVals) {
-      startFunction(function, " : tbl=" +
-          TableName.getQualified(cat, db, tbl) + "[" + join(partVals, ",") + "]");
-    }
-
-    private void startPartitionFunction(String function, String catName, String db, String tbl,
-                                        Map<String, String> partName) {
-      startFunction(function, " : tbl=" +
-          TableName.getQualified(catName, db, tbl) + "partition=" + partName);
-    }
-
-    private void endFunction(String function, boolean successful, Exception e) {
-      endFunction(function, successful, e, null);
-    }
-    private void endFunction(String function, boolean successful, Exception e,
-                            String inputTableName) {
-      endFunction(function, new MetaStoreEndFunctionContext(successful, e, inputTableName));
-    }
-
-    private void endFunction(String function, MetaStoreEndFunctionContext context) {
-      com.codahale.metrics.Timer.Context timerContext = timerContexts.get().remove(function);
-      if (timerContext != null) {
-        long timeTaken = timerContext.stop();
-        LOG.debug((getThreadLocalIpAddress() == null ? "" : "source:" + getThreadLocalIpAddress() + " ") +
-            function + "time taken(ns): " + timeTaken);
-      }
-      Counter counter = Metrics.getOrCreateCounter(MetricsConstants.ACTIVE_CALLS + function);
-      if (counter != null) {
-        counter.dec();
-      }
-
-      for (MetaStoreEndFunctionListener listener : endFunctionListeners) {
-        listener.onEndFunction(function, context);
-      }
-    }
-
-    @Override
-    public fb_status getStatus() {
-      return fb_status.ALIVE;
-    }
-
-    @Override
-    public void shutdown() {
-      cleanupRawStore();
-      PerfLogger.getPerfLogger(false).cleanupPerfLogMetrics();
-      ThreadPool.shutdown();
-    }
-
-    @Override
-    public AbstractMap<String, Long> getCounters() {
-      AbstractMap<String, Long> counters = super.getCounters();
-
-      // Allow endFunctionListeners to add any counters they have collected
-      if (endFunctionListeners != null) {
-        for (MetaStoreEndFunctionListener listener : endFunctionListeners) {
-          listener.exportCounters(counters);
-        }
-      }
-
-      return counters;
-    }
-
-    @Override
-    public void create_catalog(CreateCatalogRequest rqst)
-        throws AlreadyExistsException, InvalidObjectException, MetaException {
-      Catalog catalog = rqst.getCatalog();
-      startFunction("create_catalog", ": " + catalog.toString());
-      boolean success = false;
-      Exception ex = null;
-      try {
-        try {
-          getMS().getCatalog(catalog.getName());
-          throw new AlreadyExistsException("Catalog " + catalog.getName() + " already exists");
-        } catch (NoSuchObjectException e) {
-          // expected
-        }
-
-        if (!MetaStoreUtils.validateName(catalog.getName(), null)) {
-          throw new InvalidObjectException(catalog.getName() + " is not a valid catalog name");
-        }
-
-        if (catalog.getLocationUri() == null) {
-          throw new InvalidObjectException("You must specify a path for the catalog");
-        }
-
-        RawStore ms = getMS();
-        Path catPath = new Path(catalog.getLocationUri());
-        boolean madeDir = false;
-        Map<String, String> transactionalListenersResponses = Collections.emptyMap();
-        try {
-          firePreEvent(new PreCreateCatalogEvent(this, catalog));
-          if (!wh.isDir(catPath)) {
-            if (!wh.mkdirs(catPath)) {
-              throw new MetaException("Unable to create catalog path " + catPath +
-                  ", failed to create catalog " + catalog.getName());
-            }
-            madeDir = true;
-          }
-          // set the create time of catalog
-          long time = System.currentTimeMillis() / 1000;
-          catalog.setCreateTime((int) time);
-          ms.openTransaction();
-          ms.createCatalog(catalog);
-
-          // Create a default database inside the catalog
-          Database db = new Database(DEFAULT_DATABASE_NAME,
-              "Default database for catalog " + catalog.getName(), catalog.getLocationUri(),
-              Collections.emptyMap());
-          db.setCatalogName(catalog.getName());
-          create_database_core(ms, db);
-
-          if (!transactionalListeners.isEmpty()) {
-            transactionalListenersResponses =
-                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                    EventType.CREATE_CATALOG,
-                    new CreateCatalogEvent(true, this, catalog));
-          }
-
-          success = ms.commitTransaction();
-        } finally {
-          if (!success) {
-            ms.rollbackTransaction();
-            if (madeDir) {
-              wh.deleteDir(catPath, true, false, false);
-            }
-          }
-
-          if (!listeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(listeners,
-                EventType.CREATE_CATALOG,
-                new CreateCatalogEvent(success, this, catalog),
-                null,
-                transactionalListenersResponses, ms);
-          }
-        }
-        success = true;
-      } catch (AlreadyExistsException|InvalidObjectException|MetaException e) {
-        ex = e;
-        throw e;
-      } finally {
-        endFunction("create_catalog", success, ex);
-      }
-    }
-
-    @Override
-    public void alter_catalog(AlterCatalogRequest rqst) throws TException {
-      startFunction("alter_catalog " + rqst.getName());
-      boolean success = false;
-      Exception ex = null;
-      RawStore ms = getMS();
-      Map<String, String> transactionalListenersResponses = Collections.emptyMap();
-      GetCatalogResponse oldCat = null;
-
-      try {
-        oldCat = get_catalog(new GetCatalogRequest(rqst.getName()));
-        // Above should have thrown NoSuchObjectException if there is no such catalog
-        assert oldCat != null && oldCat.getCatalog() != null;
-        firePreEvent(new PreAlterCatalogEvent(oldCat.getCatalog(), rqst.getNewCat(), this));
-
-        ms.openTransaction();
-        ms.alterCatalog(rqst.getName(), rqst.getNewCat());
-
-        if (!transactionalListeners.isEmpty()) {
-          transactionalListenersResponses =
-              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                  EventType.ALTER_CATALOG,
-                  new AlterCatalogEvent(oldCat.getCatalog(), rqst.getNewCat(), true, this));
-        }
-
-        success = ms.commitTransaction();
-      } catch (MetaException|NoSuchObjectException e) {
-        ex = e;
-        throw e;
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-
-        if ((null != oldCat) && (!listeners.isEmpty())) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-              EventType.ALTER_CATALOG,
-              new AlterCatalogEvent(oldCat.getCatalog(), rqst.getNewCat(), success, this),
-              null, transactionalListenersResponses, ms);
-        }
-        endFunction("alter_catalog", success, ex);
-      }
-
-    }
-
-    @Override
-    public GetCatalogResponse get_catalog(GetCatalogRequest rqst)
-        throws NoSuchObjectException, TException {
-      String catName = rqst.getName();
-      startFunction("get_catalog", ": " + catName);
-      Catalog cat = null;
-      Exception ex = null;
-      try {
-        cat = getMS().getCatalog(catName);
-        firePreEvent(new PreReadCatalogEvent(this, cat));
-        return new GetCatalogResponse(cat);
-      } catch (MetaException|NoSuchObjectException e) {
-        ex = e;
-        throw e;
-      } finally {
-        endFunction("get_catalog", cat != null, ex);
-      }
-    }
-
-    @Override
-    public GetCatalogsResponse get_catalogs() throws MetaException {
-      startFunction("get_catalogs");
-
-      List<String> ret = null;
-      Exception ex = null;
-      try {
-        ret = getMS().getCatalogs();
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } finally {
-        endFunction("get_catalog", ret != null, ex);
-      }
-      return new GetCatalogsResponse(ret == null ? Collections.emptyList() : ret);
-
-    }
-
-    @Override
-    public void drop_catalog(DropCatalogRequest rqst)
-        throws NoSuchObjectException, InvalidOperationException, MetaException {
-      String catName = rqst.getName();
-      startFunction("drop_catalog", ": " + catName);
-      if (DEFAULT_CATALOG_NAME.equalsIgnoreCase(catName)) {
-        endFunction("drop_catalog", false, null);
-        throw new MetaException("Can not drop " + DEFAULT_CATALOG_NAME + " catalog");
-      }
-
-      boolean success = false;
-      Exception ex = null;
-      try {
-        dropCatalogCore(catName);
-        success = true;
-      } catch (NoSuchObjectException|InvalidOperationException|MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("drop_catalog", success, ex);
-      }
-
-    }
-
-    private void dropCatalogCore(String catName)
-        throws MetaException, NoSuchObjectException, InvalidOperationException {
-      boolean success = false;
-      Catalog cat = null;
-      Map<String, String> transactionalListenerResponses = Collections.emptyMap();
-      RawStore ms = getMS();
-      try {
-        ms.openTransaction();
-        cat = ms.getCatalog(catName);
-
-        firePreEvent(new PreDropCatalogEvent(this, cat));
-
-        List<String> allDbs = get_databases(prependNotNullCatToDbName(catName, null));
-        if (allDbs != null && !allDbs.isEmpty()) {
-          // It might just be the default, in which case we can drop that one if it's empty
-          if (allDbs.size() == 1 && allDbs.get(0).equals(DEFAULT_DATABASE_NAME)) {
-            try {
-              drop_database_core(ms, catName, DEFAULT_DATABASE_NAME, true, false);
-            } catch (InvalidOperationException e) {
-              // This means there are tables of something in the database
-              throw new InvalidOperationException("There are still objects in the default " +
-                  "database for catalog " + catName);
-            } catch (InvalidObjectException|IOException|InvalidInputException e) {
-              MetaException me = new MetaException("Error attempt to drop default database for " +
-                  "catalog " + catName);
-              me.initCause(e);
-              throw me;
-            }
-          } else {
-            throw new InvalidOperationException("There are non-default databases in the catalog " +
-                catName + " so it cannot be dropped.");
-          }
-        }
-
-        ms.dropCatalog(catName) ;
-        if (!transactionalListeners.isEmpty()) {
-          transactionalListenerResponses =
-              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                  EventType.DROP_CATALOG,
-                  new DropCatalogEvent(true, this, cat));
-        }
-
-        success = ms.commitTransaction();
-      } finally {
-        if (success) {
-          wh.deleteDir(wh.getDnsPath(new Path(cat.getLocationUri())), false, false, false);
-        } else {
-          ms.rollbackTransaction();
-        }
-
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-              EventType.DROP_CATALOG,
-              new DropCatalogEvent(success, this, cat),
-              null,
-              transactionalListenerResponses, ms);
-        }
-      }
-    }
-
-    static boolean isDbReplicationTarget(Database db) {
-      if (db.getParameters() == null) {
-        return false;
-      }
-
-      if (!db.getParameters().containsKey(ReplConst.REPL_TARGET_DB_PROPERTY)) {
-        return false;
-      }
-
-      return !db.getParameters().get(ReplConst.REPL_TARGET_DB_PROPERTY).trim().isEmpty();
-    }
-
-    // Assumes that the catalog has already been set.
-    private void create_database_core(RawStore ms, final Database db)
-        throws AlreadyExistsException, InvalidObjectException, MetaException {
-      if (!MetaStoreUtils.validateName(db.getName(), conf)) {
-        throw new InvalidObjectException(db.getName() + " is not a valid database name");
-      }
-
-      Catalog cat = null;
-      try {
-        cat = getMS().getCatalog(db.getCatalogName());
-      } catch (NoSuchObjectException e) {
-        LOG.error("No such catalog " + db.getCatalogName());
-        throw new InvalidObjectException("No such catalog " + db.getCatalogName());
-      }
-      boolean skipAuthorization = false;
-      String passedInURI = db.getLocationUri();
-      String passedInManagedURI = db.getManagedLocationUri();
-      if (passedInURI == null && passedInManagedURI == null) {
-        skipAuthorization = true;
-      }
-      final Path defaultDbExtPath = wh.getDefaultDatabasePath(db.getName(), true);
-      final Path defaultDbMgdPath = wh.getDefaultDatabasePath(db.getName(), false);
-      final Path dbExtPath = (passedInURI != null) ? wh.getDnsPath(new Path(passedInURI)) : wh.determineDatabasePath(cat, db);
-      final Path dbMgdPath = (passedInManagedURI != null) ? wh.getDnsPath(new Path(passedInManagedURI)) : null;
-
-      if ((defaultDbExtPath.equals(dbExtPath) && defaultDbMgdPath.equals(dbMgdPath)) &&
-          ((dbMgdPath == null) || dbMgdPath.equals(defaultDbMgdPath))) {
-        skipAuthorization = true;
-      }
-
-      if ( skipAuthorization ) {
-        //null out to skip authorizer URI check
-        db.setLocationUri(null);
-        db.setManagedLocationUri(null);
-      }else{
-        db.setLocationUri(dbExtPath.toString());
-        if (dbMgdPath != null) {
-          db.setManagedLocationUri(dbMgdPath.toString());
-        }
-      }
-      if (db.getOwnerName() == null){
-        try {
-          db.setOwnerName(SecurityUtils.getUGI().getShortUserName());
-        }catch (Exception e){
-          LOG.warn("Failed to get owner name for create database operation.", e);
-        }
-      }
-      long time = System.currentTimeMillis()/1000;
-      db.setCreateTime((int) time);
-      boolean success = false;
-      boolean madeManagedDir = false;
-      boolean madeExternalDir = false;
-      boolean isReplicated = isDbReplicationTarget(db);
-      Map<String, String> transactionalListenersResponses = Collections.emptyMap();
-      try {
-        firePreEvent(new PreCreateDatabaseEvent(db, this));
-        //reinstate location uri for metastore db.
-        if (skipAuthorization == true){
-          db.setLocationUri(dbExtPath.toString());
-          if (dbMgdPath != null)
-            db.setManagedLocationUri(dbMgdPath.toString());
-        }
-        if (db.getCatalogName() != null && !db.getCatalogName().
-            equals(Warehouse.DEFAULT_CATALOG_NAME)) {
-          if (!wh.isDir(dbExtPath)) {
-            LOG.debug("Creating database path " + dbExtPath);
-            if (!wh.mkdirs(dbExtPath)) {
-              throw new MetaException("Unable to create database path " + dbExtPath +
-                  ", failed to create database " + db.getName());
-            }
-            madeExternalDir = true;
-          }
-        } else {
-          if (dbMgdPath != null) {
-            try {
-              // Since this may be done as random user (if doAs=true) he may not have access
-              // to the managed directory. We run this as an admin user
-              madeManagedDir = UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction<Boolean>() {
-                @Override public Boolean run() throws MetaException {
-                  if (!wh.isDir(dbMgdPath)) {
-                    LOG.info("Creating database path in managed directory " + dbMgdPath);
-                    if (!wh.mkdirs(dbMgdPath)) {
-                      throw new MetaException("Unable to create database managed path " + dbMgdPath + ", failed to create database " + db.getName());
-                    }
-                    return true;
-                  }
-                  return false;
-                }
-              });
-              if (madeManagedDir) {
-                LOG.info("Created database path in managed directory " + dbMgdPath);
-              } else {
-                throw new MetaException(
-                    "Unable to create database managed directory " + dbMgdPath + ", failed to create database " + db.getName());
-              }
-            } catch (IOException | InterruptedException e) {
-              throw new MetaException(
-                  "Unable to create database managed directory " + dbMgdPath + ", failed to create database " + db.getName() + ":" + e.getMessage());
-            }
-          }
-          if (dbExtPath != null) {
-            try {
-              madeExternalDir = UserGroupInformation.getCurrentUser().doAs(new PrivilegedExceptionAction<Boolean>() {
-                @Override public Boolean run() throws MetaException {
-                  if (!wh.isDir(dbExtPath)) {
-                    LOG.info("Creating database path in external directory " + dbExtPath);
-                    return wh.mkdirs(dbExtPath);
-                  }
-                  return false;
-                }
-              });
-              if (madeExternalDir) {
-                LOG.info("Created database path in external directory " + dbExtPath);
-              } else {
-                LOG.warn("Failed to create external path " + dbExtPath + " for database " + db.getName() + ". This may result in access not being allowed if the "
-                    + "StorageBasedAuthorizationProvider is enabled");
-              }
-            } catch (IOException | InterruptedException | UndeclaredThrowableException e) {
-              throw new MetaException("Failed to create external path " + dbExtPath + " for database " + db.getName() + ". This may result in access not being allowed if the "
-                  + "StorageBasedAuthorizationProvider is enabled: " + e.getMessage());
-            }
-          } else {
-            LOG.info("Database external path won't be created since the external warehouse directory is not defined");
-          }
-        }
-
-        ms.openTransaction();
-        ms.createDatabase(db);
-
-        if (!transactionalListeners.isEmpty()) {
-          transactionalListenersResponses =
-              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                    EventType.CREATE_DATABASE,
-                                                    new CreateDatabaseEvent(db, true, this, isReplicated));
-        }
-
-        success = ms.commitTransaction();
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-
-          if (db.getCatalogName() != null && !db.getCatalogName().
-              equals(Warehouse.DEFAULT_CATALOG_NAME)) {
-            if (madeManagedDir) {
-              wh.deleteDir(dbMgdPath, true, db);
-            }
-          } else {
-            if (madeManagedDir) {
-              try {
-                UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction<Void>() {
-                  @Override public Void run() throws Exception {
-                    wh.deleteDir(dbMgdPath, true, db);
-                    return null;
-                  }
-                });
-              } catch (IOException | InterruptedException e) {
-                LOG.error(
-                    "Couldn't delete managed directory " + dbMgdPath + " after " + "it was created for database " + db.getName() + " " + e.getMessage());
-              }
-            }
-
-            if (madeExternalDir) {
-              try {
-                UserGroupInformation.getCurrentUser().doAs(new PrivilegedExceptionAction<Void>() {
-                  @Override public Void run() throws Exception {
-                    wh.deleteDir(dbExtPath, true, db);
-                    return null;
-                  }
-                });
-              } catch (IOException | InterruptedException e) {
-                LOG.error("Couldn't delete external directory " + dbExtPath + " after " + "it was created for database "
-                    + db.getName() + " " + e.getMessage());
-              }
-            }
-          }
-        }
-
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                EventType.CREATE_DATABASE,
-                                                new CreateDatabaseEvent(db, success, this, isReplicated),
-                                                null,
-                                                transactionalListenersResponses, ms);
-        }
-      }
-    }
-
-    @Override
-    public void create_database(final Database db)
-        throws AlreadyExistsException, InvalidObjectException, MetaException {
-      startFunction("create_database", ": " + db.toString());
-      boolean success = false;
-      Exception ex = null;
-      if (!db.isSetCatalogName()) {
-        db.setCatalogName(getDefaultCatalog(conf));
-      }
-      try {
-        try {
-          if (null != get_database_core(db.getCatalogName(), db.getName())) {
-            throw new AlreadyExistsException("Database " + db.getName() + " already exists");
-          }
-        } catch (NoSuchObjectException e) {
-          // expected
-        }
-
-        if (TEST_TIMEOUT_ENABLED) {
-          try {
-            Thread.sleep(TEST_TIMEOUT_VALUE);
-          } catch (InterruptedException e) {
-            // do nothing
-          }
-          Deadline.checkTimeout();
-        }
-        create_database_core(getMS(), db);
-        success = true;
-      } catch (MetaException | InvalidObjectException | AlreadyExistsException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("create_database", success, ex);
-      }
-    }
-
-    @Override
-    public Database get_database(final String name)
-        throws NoSuchObjectException, MetaException {
-      GetDatabaseRequest request = new GetDatabaseRequest();
-      String[] parsedDbName = parseDbName(name, conf);
-      request.setName(parsedDbName[DB_NAME]);
-      if (parsedDbName[CAT_NAME] != null)
-          request.setCatalogName(parsedDbName[CAT_NAME]);
-        return get_database_req(request);
-    }
-
-    @Override
-    public Database get_database_core(String catName, final String name) throws NoSuchObjectException, MetaException {
-      Database db = null;
-      if (name == null) {
-        throw new MetaException("Database name cannot be null.");
-      }
-      try {
-        db = getMS().getDatabase(catName, name);
-      } catch (MetaException | NoSuchObjectException e) {
-        throw e;
-      } catch (Exception e) {
-        assert (e instanceof RuntimeException);
-        throw (RuntimeException) e;
-      }
-      return db;
-    }
-
-    @Override
-    public Database get_database_req(GetDatabaseRequest request) throws NoSuchObjectException, MetaException {
-      startFunction("get_database", ": " + request.getName());
-      Database db = null;
-      Exception ex = null;
-      if (request.getName() == null) {
-        throw new MetaException("Database name cannot be null.");
-      }
-      List<String> processorCapabilities = request.getProcessorCapabilities();
-      String processorId = request.getProcessorIdentifier();
-      try {
-        db = getMS().getDatabase(request.getCatalogName(), request.getName());
-        firePreEvent(new PreReadDatabaseEvent(db, this));
-        if (processorCapabilities != null && transformer != null) {
-          db = transformer.transformDatabase(db, processorCapabilities, processorId);
-        }
-      } catch (MetaException | NoSuchObjectException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        assert (e instanceof RuntimeException);
-        throw (RuntimeException) e;
-      } finally {
-        endFunction("get_database", db != null, ex);
-      }
-      return db;
-    }
-
-    @Override
-    public void alter_database(final String dbName, final Database newDB) throws TException {
-      startFunction("alter_database " + dbName);
-      boolean success = false;
-      Exception ex = null;
-      RawStore ms = getMS();
-      Database oldDB = null;
-      Map<String, String> transactionalListenersResponses = Collections.emptyMap();
-
-      // Perform the same URI normalization as create_database_core.
-      if (newDB.getLocationUri() != null) {
-        newDB.setLocationUri(wh.getDnsPath(new Path(newDB.getLocationUri())).toString());
-      }
-
-      String[] parsedDbName = parseDbName(dbName, conf);
-
-      // We can replicate into an empty database, in which case newDB will have indication that
-      // it's target of replication but not oldDB. But replication flow will never alter a
-      // database so that oldDB indicates that it's target or replication but not the newDB. So,
-      // relying solely on newDB to check whether the database is target of replication works.
-      boolean isReplicated = isDbReplicationTarget(newDB);
-      try {
-        oldDB = get_database_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME]);
-        if (oldDB == null) {
-          throw new MetaException("Could not alter database \"" + parsedDbName[DB_NAME] +
-              "\". Could not retrieve old definition.");
-        }
-        firePreEvent(new PreAlterDatabaseEvent(oldDB, newDB, this));
-
-        ms.openTransaction();
-        ms.alterDatabase(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], newDB);
-
-        if (!transactionalListeners.isEmpty()) {
-          transactionalListenersResponses =
-              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                  EventType.ALTER_DATABASE,
-                  new AlterDatabaseEvent(oldDB, newDB, true, this, isReplicated));
-        }
-
-        success = ms.commitTransaction();
-      } catch (MetaException|NoSuchObjectException e) {
-        ex = e;
-        throw e;
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-
-        if ((null != oldDB) && (!listeners.isEmpty())) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-              EventType.ALTER_DATABASE,
-              new AlterDatabaseEvent(oldDB, newDB, success, this, isReplicated),
-              null,
-              transactionalListenersResponses, ms);
-        }
-        endFunction("alter_database", success, ex);
-      }
-    }
-
-    private void drop_database_core(RawStore ms, String catName,
-        final String name, final boolean deleteData, final boolean cascade)
-        throws NoSuchObjectException, InvalidOperationException, MetaException,
-        IOException, InvalidObjectException, InvalidInputException {
-      boolean success = false;
-      Database db = null;
-      List<Path> tablePaths = new ArrayList<>();
-      List<Path> partitionPaths = new ArrayList<>();
-      Map<String, String> transactionalListenerResponses = Collections.emptyMap();
-      if (name == null) {
-        throw new MetaException("Database name cannot be null.");
-      }
-      boolean isReplicated = false;
-      try {
-        ms.openTransaction();
-        db = ms.getDatabase(catName, name);
-        if (db.getType() == DatabaseType.REMOTE) {
-          success = drop_remote_database_core(ms, db);
-          return;
-        }
-        isReplicated = isDbReplicationTarget(db);
-
-        if (!isInTest && ReplChangeManager.isSourceOfReplication(db)) {
-          throw new InvalidOperationException("can not drop a database which is a source of replication");
-        }
-
-        firePreEvent(new PreDropDatabaseEvent(db, this));
-        String catPrependedName = MetaStoreUtils.prependCatalogToDbName(catName, name, conf);
-
-        Set<String> uniqueTableNames = new HashSet<>(get_all_tables(catPrependedName));
-        List<String> allFunctions = get_functions(catPrependedName, "*");
-        ListStoredProcedureRequest request = new ListStoredProcedureRequest(catName);
-        request.setDbName(name);
-        List<String> allProcedures = get_all_stored_procedures(request);
-
-        if (!cascade) {
-          if (!uniqueTableNames.isEmpty()) {
-            throw new InvalidOperationException(
-                "Database " + db.getName() + " is not empty. One or more tables exist.");
-          }
-          if (!allFunctions.isEmpty()) {
-            throw new InvalidOperationException(
-                "Database " + db.getName() + " is not empty. One or more functions exist.");
-          }
-          if (!allProcedures.isEmpty()) {
-            throw new InvalidOperationException(
-                    "Database " + db.getName() + " is not empty. One or more stored procedures exist.");
-          }
-        }
-        Path path = new Path(db.getLocationUri()).getParent();
-        if (!wh.isWritable(path)) {
-          throw new MetaException("Database not dropped since " +
-              path + " is not writable by " +
-              SecurityUtils.getUser());
-        }
-
-        Path databasePath = wh.getDnsPath(wh.getDatabasePath(db));
-
-        // drop any functions before dropping db
-        for (String funcName : allFunctions) {
-          drop_function(catPrependedName, funcName);
-        }
-
-        for (String procName : allProcedures) {
-          drop_stored_procedure(new StoredProcedureRequest(catName, name, procName));
-        }
-
-        final int tableBatchSize = MetastoreConf.getIntVar(conf,
-            ConfVars.BATCH_RETRIEVE_MAX);
-
-        // First pass will drop the materialized views
-        List<String> materializedViewNames = getTablesByTypeCore(catName, name, ".*",
-            TableType.MATERIALIZED_VIEW.toString());
-        int startIndex = 0;
-        // retrieve the tables from the metastore in batches to alleviate memory constraints
-        while (startIndex < materializedViewNames.size()) {
-          int endIndex = Math.min(startIndex + tableBatchSize, materializedViewNames.size());
-
-          List<Table> materializedViews;
-          try {
-            materializedViews = ms.getTableObjectsByName(catName, name, materializedViewNames.subList(startIndex, endIndex));
-          } catch (UnknownDBException e) {
-            throw new MetaException(e.getMessage());
-          }
-
-          if (materializedViews != null && !materializedViews.isEmpty()) {
-            for (Table materializedView : materializedViews) {
-              if (materializedView.getSd().getLocation() != null) {
-                Path materializedViewPath = wh.getDnsPath(new Path(materializedView.getSd().getLocation()));
-                if (!wh.isWritable(materializedViewPath.getParent())) {
-                  throw new MetaException("Database metadata not deleted since table: " +
-                      materializedView.getTableName() + " has a parent location " + materializedViewPath.getParent() +
-                      " which is not writable by " + SecurityUtils.getUser());
-                }
-
-                if (!FileUtils.isSubdirectory(databasePath.toString(),
-                    materializedViewPath.toString())) {
-                  tablePaths.add(materializedViewPath);
-                }
-              }
-              // Drop the materialized view but not its data
-              drop_table(name, materializedView.getTableName(), false);
-              // Remove from all tables
-              uniqueTableNames.remove(materializedView.getTableName());
-            }
-          }
-          startIndex = endIndex;
-        }
-
-        // drop tables before dropping db
-        List<String> allTables = new ArrayList<>(uniqueTableNames);
-        startIndex = 0;
-        // retrieve the tables from the metastore in batches to alleviate memory constraints
-        while (startIndex < allTables.size()) {
-          int endIndex = Math.min(startIndex + tableBatchSize, allTables.size());
-
-          List<Table> tables;
-          try {
-            tables = ms.getTableObjectsByName(catName, name, allTables.subList(startIndex, endIndex));
-          } catch (UnknownDBException e) {
-            throw new MetaException(e.getMessage());
-          }
-
-          if (tables != null && !tables.isEmpty()) {
-            for (Table table : tables) {
-
-              // If the table is not external and it might not be in a subdirectory of the database
-              // add it's locations to the list of paths to delete
-              Path tablePath = null;
-              boolean tableDataShouldBeDeleted = checkTableDataShouldBeDeleted(table, deleteData);
-              if (table.getSd().getLocation() != null && tableDataShouldBeDeleted) {
-                tablePath = wh.getDnsPath(new Path(table.getSd().getLocation()));
-                if (!wh.isWritable(tablePath.getParent())) {
-                  throw new MetaException("Database metadata not deleted since table: " +
-                      table.getTableName() + " has a parent location " + tablePath.getParent() +
-                      " which is not writable by " + SecurityUtils.getUser());
-                }
-
-                if (!FileUtils.isSubdirectory(databasePath.toString(), tablePath.toString())) {
-                  tablePaths.add(tablePath);
-                }
-              }
-
-              // For each partition in each table, drop the partitions and get a list of
-              // partitions' locations which might need to be deleted
-              partitionPaths = dropPartitionsAndGetLocations(ms, catName, name, table.getTableName(),
-                  tablePath, tableDataShouldBeDeleted);
-
-              // Drop the table but not its data
-              drop_table(MetaStoreUtils.prependCatalogToDbName(table.getCatName(), table.getDbName(), conf),
-                  table.getTableName(), false);
-            }
-          }
-
-          startIndex = endIndex;
-        }
-
-        if (ms.dropDatabase(catName, name)) {
-          if (!transactionalListeners.isEmpty()) {
-            transactionalListenerResponses =
-                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                      EventType.DROP_DATABASE,
-                                                      new DropDatabaseEvent(db, true, this, isReplicated));
-          }
-
-          success = ms.commitTransaction();
-        }
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        } else if (deleteData) {
-          // Delete the data in the partitions which have other locations
-          deletePartitionData(partitionPaths, false, db);
-          // Delete the data in the tables which have other locations
-          for (Path tablePath : tablePaths) {
-            deleteTableData(tablePath, false, db);
-          }
-          // Delete the data in the database
-          try {
-            if (db.getManagedLocationUri() != null) {
-              wh.deleteDir(new Path(db.getManagedLocationUri()), true, db);
-            }
-          } catch (Exception e) {
-            LOG.error("Failed to delete database directory: " + db.getManagedLocationUri() +
-                " " + e.getMessage());
-          }
-          // Delete the data in the database's location only if it is a legacy db path?
-          try {
-            wh.deleteDir(new Path(db.getLocationUri()), true, db);
-          } catch (Exception e) {
-            LOG.error("Failed to delete database directory: " + db.getLocationUri() +
-                " " + e.getMessage());
-          }
-          // it is not a terrible thing even if the data is not deleted
-        }
-
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                EventType.DROP_DATABASE,
-                                                new DropDatabaseEvent(db, success, this, isReplicated),
-                                                null,
-                                                transactionalListenerResponses, ms);
-        }
-      }
-    }
-
-    private boolean drop_remote_database_core(RawStore ms, final Database db) throws MetaException, NoSuchObjectException {
-      boolean success = false;
-      firePreEvent(new PreDropDatabaseEvent(db, this));
-
-      if (ms.dropDatabase(db.getCatalogName(), db.getName())) {
-        success = ms.commitTransaction();
-      }
-      return success;
-    }
-
-    @Override
-    public void drop_database(final String dbName, final boolean deleteData, final boolean cascade)
-        throws NoSuchObjectException, InvalidOperationException, MetaException {
-      startFunction("drop_database", ": " + dbName);
-      String[] parsedDbName = parseDbName(dbName, conf);
-      if (DEFAULT_CATALOG_NAME.equalsIgnoreCase(parsedDbName[CAT_NAME]) &&
-          DEFAULT_DATABASE_NAME.equalsIgnoreCase(parsedDbName[DB_NAME])) {
-        endFunction("drop_database", false, null);
-        throw new MetaException("Can not drop " + DEFAULT_DATABASE_NAME + " database in catalog "
-            + DEFAULT_CATALOG_NAME);
-      }
-
-      boolean success = false;
-      Exception ex = null;
-      try {
-        drop_database_core(getMS(), parsedDbName[CAT_NAME], parsedDbName[DB_NAME], deleteData,
-            cascade);
-        success = true;
-      } catch (NoSuchObjectException|InvalidOperationException|MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("drop_database", success, ex);
-      }
-    }
-
-
-    @Override
-    public List<String> get_databases(final String pattern) throws MetaException {
-      startFunction("get_databases", ": " + pattern);
-
-      String[] parsedDbNamed = parseDbName(pattern, conf);
-      List<String> ret = null;
-      Exception ex = null;
-      try {
-        if (parsedDbNamed[DB_NAME] == null) {
-          ret = getMS().getAllDatabases(parsedDbNamed[CAT_NAME]);
-          ret = FilterUtils.filterDbNamesIfEnabled(isServerFilterEnabled, filterHook, ret);
-        } else {
-          ret = getMS().getDatabases(parsedDbNamed[CAT_NAME], parsedDbNamed[DB_NAME]);
-          ret = FilterUtils.filterDbNamesIfEnabled(isServerFilterEnabled, filterHook, ret);
-        }
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_databases", ret != null, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public List<String> get_all_databases() throws MetaException {
-      // get_databases filters results already. No need to filter here
-      return get_databases(MetaStoreUtils.prependCatalogToDbName(null, null, conf));
-    }
-
-    private void create_type_core(final RawStore ms, final Type type)
-        throws AlreadyExistsException, MetaException, InvalidObjectException {
-      if (!MetaStoreUtils.validateName(type.getName(), null)) {
-        throw new InvalidObjectException("Invalid type name");
-      }
-
-      boolean success = false;
-      try {
-        ms.openTransaction();
-        if (is_type_exists(ms, type.getName())) {
-          throw new AlreadyExistsException("Type " + type.getName() + " already exists");
-        }
-        ms.createType(type);
-        success = ms.commitTransaction();
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-      }
-    }
-
-    private void create_dataconnector_core(RawStore ms, final DataConnector connector)
-        throws AlreadyExistsException, InvalidObjectException, MetaException {
-      if (!MetaStoreUtils.validateName(connector.getName(), conf)) {
-        throw new InvalidObjectException(connector.getName() + " is not a valid dataconnector name");
-      }
-
-      if (connector.getOwnerName() == null){
-        try {
-          connector.setOwnerName(SecurityUtils.getUGI().getShortUserName());
-        }catch (Exception e){
-          LOG.warn("Failed to get owner name for create dataconnector operation.", e);
-        }
-      }
-      long time = System.currentTimeMillis()/1000;
-      connector.setCreateTime((int) time);
-      boolean success = false;
-      Map<String, String> transactionalListenersResponses = Collections.emptyMap();
-      try {
-        firePreEvent(new PreCreateDataConnectorEvent(connector, this));
-
-        ms.openTransaction();
-        ms.createDataConnector(connector);
-
-        if (!transactionalListeners.isEmpty()) {
-          transactionalListenersResponses =
-              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                  EventType.CREATE_DATACONNECTOR,
-                  new CreateDataConnectorEvent(connector, true, this));
-        }
-
-        success = ms.commitTransaction();
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-              EventType.CREATE_DATACONNECTOR,
-              new CreateDataConnectorEvent(connector, success, this),
-              null,
-              transactionalListenersResponses, ms);
-        }
-      }
-    }
-
-    @Override
-    public void create_dataconnector(final DataConnector connector)
-        throws AlreadyExistsException, InvalidObjectException, MetaException {
-      startFunction("create_dataconnector", ": " + connector.toString());
-      boolean success = false;
-      Exception ex = null;
-      try {
-        try {
-          if (null != get_dataconnector_core(connector.getName())) {
-            throw new AlreadyExistsException("DataConnector " + connector.getName() + " already exists");
-          }
-        } catch (NoSuchObjectException e) {
-          // expected
-        }
-
-        if (TEST_TIMEOUT_ENABLED) {
-          try {
-            Thread.sleep(TEST_TIMEOUT_VALUE);
-          } catch (InterruptedException e) {
-            // do nothing
-          }
-          Deadline.checkTimeout();
-        }
-        create_dataconnector_core(getMS(), connector);
-        success = true;
-      } catch (MetaException | InvalidObjectException | AlreadyExistsException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("create_connector", success, ex);
-      }
-    }
-
-    @Override
-    public DataConnector get_dataconnector_core(final String name) throws NoSuchObjectException, MetaException {
-      DataConnector connector = null;
-      if (name == null) {
-        throw new MetaException("Data connector name cannot be null.");
-      }
-      try {
-        connector = getMS().getDataConnector(name);
-      } catch (MetaException | NoSuchObjectException e) {
-        throw e;
-      } catch (Exception e) {
-        assert (e instanceof RuntimeException);
-        throw (RuntimeException) e;
-      }
-      return connector;
-    }
-
-    @Override
-    public DataConnector get_dataconnector_req(GetDataConnectorRequest request) throws NoSuchObjectException, MetaException {
-      startFunction("get_dataconnector", ": " + request.getConnectorName());
-      DataConnector connector = null;
-      Exception ex = null;
-      try {
-        connector = get_dataconnector_core(request.getConnectorName());
-      } catch (MetaException | NoSuchObjectException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        assert (e instanceof RuntimeException);
-        throw (RuntimeException) e;
-      } finally {
-        endFunction("get_dataconnector", connector != null, ex);
-      }
-      return connector;
-    }
-
-    @Override
-    public void alter_dataconnector(final String dcName, final DataConnector newDC) throws TException {
-      startFunction("alter_dataconnector " + dcName);
-      boolean success = false;
-      Exception ex = null;
-      RawStore ms = getMS();
-      DataConnector oldDC = null;
-      Map<String, String> transactionalListenersResponses = Collections.emptyMap();
-
-      try {
-        oldDC = get_dataconnector_core(dcName);
-        if (oldDC == null) {
-          throw new MetaException("Could not alter dataconnector \"" + dcName +
-              "\". Could not retrieve old definition.");
-        }
-        // firePreEvent(new PreAlterDatabaseEvent(oldDC, newDC, this));
-
-        ms.openTransaction();
-        ms.alterDataConnector(dcName, newDC);
-
-        /*
-        if (!transactionalListeners.isEmpty()) {
-          transactionalListenersResponses =
-              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                  EventType.ALTER_DATACONNECTOR,
-                  new AlterDataConnectorEvent(oldDC, newDC, true, this));
-        }
-         */
-
-        success = ms.commitTransaction();
-      } catch (MetaException|NoSuchObjectException e) {
-        ex = e;
-        throw e;
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-/*
-        if ((null != oldDC) && (!listeners.isEmpty())) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-              EventType.ALTER_DATACONNECTOR,
-              new AlterDataConnectorEvent(oldDC, newDC, success, this),
-              null,
-              transactionalListenersResponses, ms);
-        }
- */
-        endFunction("alter_database", success, ex);
-      }
-    }
-
-    @Override
-    public List<String> get_dataconnectors() throws MetaException {
-      startFunction("get_dataconnectors");
-
-      List<String> ret = null;
-      Exception ex = null;
-      try {
-          ret = getMS().getAllDataConnectors();
-          ret = FilterUtils.filterDataConnectorsIfEnabled(isServerFilterEnabled, filterHook, ret);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_dataconnectors", ret != null, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public void drop_dataconnector(final String dcName, boolean ifNotExists, boolean checkReferences) throws NoSuchObjectException, InvalidOperationException, MetaException {
-      startFunction("drop_dataconnector", ": " + dcName);
-      boolean success = false;
-      DataConnector connector = null;
-      Exception ex = null;
-      RawStore ms = getMS();
-      try {
-        ms.openTransaction();
-        connector = getMS().getDataConnector(dcName);
-
-        if (connector == null) {
-          if (!ifNotExists) {
-            throw new NoSuchObjectException("DataConnector " + dcName + " doesn't exist");
-          } else {
-            return;
-          }
-        }
-        // TODO find DBs with references to this connector
-        // if any existing references and checkReferences=true, do not drop
-
-        // firePreEvent(new PreDropTableEvent(tbl, deleteData, this));
-
-        if (!ms.dropDataConnector(dcName)) {
-          throw new MetaException("Unable to drop dataconnector " + dcName);
-        } else {
-/*
-          if (!transactionalListeners.isEmpty()) {
-            transactionalListenerResponses =
-                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                    EventType.DROP_TABLE,
-                    new DropTableEvent(tbl, true, deleteData,
-                        this, isReplicated),
-                    envContext);
-          }
-
- */
-          success = ms.commitTransaction();
-        }
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-/*
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-              EventType.DROP_TABLE,
-              new DropTableEvent(tbl, success, deleteData, this, isReplicated),
-              envContext,
-              transactionalListenerResponses, ms);
-        }
-
- */
-        endFunction("drop_dataconnector", success, ex);
-      }
-    }
-
-
-
-    @Override
-    public boolean create_type(final Type type) throws AlreadyExistsException,
-        MetaException, InvalidObjectException {
-      startFunction("create_type", ": " + type.toString());
-      boolean success = false;
-      Exception ex = null;
-      try {
-        create_type_core(getMS(), type);
-        success = true;
-      } catch (MetaException | InvalidObjectException | AlreadyExistsException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("create_type", success, ex);
-      }
-
-      return success;
-    }
-
-    @Override
-    public Type get_type(final String name) throws MetaException, NoSuchObjectException {
-      startFunction("get_type", ": " + name);
-
-      Type ret = null;
-      Exception ex = null;
-      try {
-        ret = getMS().getType(name);
-        if (null == ret) {
-          throw new NoSuchObjectException("Type \"" + name + "\" not found.");
-        }
-      } catch (Exception e) {
-        ex = e;
-        throwMetaException(e);
-      } finally {
-        endFunction("get_type", ret != null, ex);
-      }
-      return ret;
-    }
-
-    private boolean is_type_exists(RawStore ms, String typeName)
-        throws MetaException {
-      return (ms.getType(typeName) != null);
-    }
-
-    @Override
-    public boolean drop_type(final String name) throws MetaException, NoSuchObjectException {
-      startFunction("drop_type", ": " + name);
-
-      boolean success = false;
-      Exception ex = null;
-      try {
-        // TODO:pc validate that there are no types that refer to this
-        success = getMS().dropType(name);
-      } catch (Exception e) {
-        ex = e;
-        throwMetaException(e);
-      } finally {
-        endFunction("drop_type", success, ex);
-      }
-      return success;
-    }
-
-    @Override
-    public Map<String, Type> get_type_all(String name) throws MetaException {
-      // TODO Auto-generated method stub
-      startFunction("get_type_all", ": " + name);
-      endFunction("get_type_all", false, null);
-      throw new MetaException("Not yet implemented");
-    }
-
-    private void create_table_core(final RawStore ms, final Table tbl,
-        final EnvironmentContext envContext)
-            throws AlreadyExistsException, MetaException,
-            InvalidObjectException, NoSuchObjectException, InvalidInputException {
-      CreateTableRequest req = new CreateTableRequest(tbl);
-      req.setEnvContext(envContext);
-      create_table_core(ms, req);
-    }
-
-    private void create_table_core(final RawStore ms, final Table tbl,
-        final EnvironmentContext envContext, List<SQLPrimaryKey> primaryKeys,
-        List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints,
-        List<SQLNotNullConstraint> notNullConstraints, List<SQLDefaultConstraint> defaultConstraints,
-                                   List<SQLCheckConstraint> checkConstraints,
-                                   List<String> processorCapabilities, String processorIdentifier)
-        throws AlreadyExistsException, MetaException,
-        InvalidObjectException, NoSuchObjectException, InvalidInputException {
-      CreateTableRequest req = new CreateTableRequest(tbl);
-      if (envContext != null)
-        req.setEnvContext(envContext);
-      if (primaryKeys != null)
-        req.setPrimaryKeys(primaryKeys);
-      if (foreignKeys != null)
-        req.setForeignKeys(foreignKeys);
-      if (uniqueConstraints != null)
-        req.setUniqueConstraints(uniqueConstraints);
-      if (notNullConstraints != null)
-        req.setNotNullConstraints(notNullConstraints);
-      if (defaultConstraints != null)
-        req.setDefaultConstraints(defaultConstraints);
-      if (checkConstraints != null)
-        req.setCheckConstraints(checkConstraints);
-      if (processorCapabilities != null) {
-        req.setProcessorCapabilities(processorCapabilities);
-        req.setProcessorIdentifier(processorIdentifier);
-      }
-      create_table_core(ms, req);
-    }
-
-    private void create_table_core(final RawStore ms, final CreateTableRequest req)
-        throws AlreadyExistsException, MetaException,
-              InvalidObjectException, NoSuchObjectException, InvalidInputException {
-      ColumnStatistics colStats = null;
-      Table tbl = req.getTable();
-      EnvironmentContext envContext = req.getEnvContext();
-      SQLAllTableConstraints constraints = new SQLAllTableConstraints();
-      constraints.setPrimaryKeys(req.getPrimaryKeys());
-      constraints.setForeignKeys(req.getForeignKeys());
-      constraints.setUniqueConstraints(req.getUniqueConstraints());
-      constraints.setDefaultConstraints(req.getDefaultConstraints());
-      constraints.setCheckConstraints(req.getCheckConstraints());
-      constraints.setNotNullConstraints(req.getNotNullConstraints());
-      List<String> processorCapabilities = req.getProcessorCapabilities();
-      String processorId = req.getProcessorIdentifier();
-
-      // To preserve backward compatibility throw MetaException in case of null database
-      if (tbl.getDbName() == null) {
-        throw new MetaException("Null database name is not allowed");
-      }
-
-      if (!MetaStoreUtils.validateName(tbl.getTableName(), conf)) {
-        throw new InvalidObjectException(tbl.getTableName()
-            + " is not a valid object name");
-      }
-
-      if (!tbl.isSetCatName()) {
-        tbl.setCatName(getDefaultCatalog(conf));
-      }
-
-      Database db = get_database_core(tbl.getCatName(), tbl.getDbName());
-      if (db != null && db.getType().equals(DatabaseType.REMOTE)) {
-        DataConnectorProviderFactory.getDataConnectorProvider(db).createTable(tbl);
-        return;
-      }
-
-      if (transformer != null && !isInTest) {
-        tbl = transformer.transformCreateTable(tbl, processorCapabilities, processorId);
-      }
-
-      // If the given table has column statistics, save it here. We will update it later.
-      // We don't want it to be part of the Table object being created, lest the create table
-      // event will also have the col stats which we don't want.
-      if (tbl.isSetColStats()) {
-        colStats = tbl.getColStats();
-        tbl.unsetColStats();
-      }
-
-      String validate = MetaStoreServerUtils.validateTblColumns(tbl.getSd().getCols());
-      if (validate != null) {
-        throw new InvalidObjectException("Invalid column " + validate);
-      }
-      if (tbl.getPartitionKeys() != null) {
-        validate = MetaStoreServerUtils.validateTblColumns(tbl.getPartitionKeys());
-        if (validate != null) {
-          throw new InvalidObjectException("Invalid partition column " + validate);
-        }
-      }
-      if (tbl.isSetId()) {
-        LOG.debug("Id shouldn't be set but table {}.{} has the Id set to {}. Id is ignored.", tbl.getDbName(),
-            tbl.getTableName(), tbl.getId());
-        tbl.unsetId();
-      }
-      SkewedInfo skew = tbl.getSd().getSkewedInfo();
-      if (skew != null) {
-        validate = MetaStoreServerUtils.validateSkewedColNames(skew.getSkewedColNames());
-        if (validate != null) {
-          throw new InvalidObjectException("Invalid skew column " + validate);
-        }
-        validate = MetaStoreServerUtils.validateSkewedColNamesSubsetCol(
-            skew.getSkewedColNames(), tbl.getSd().getCols());
-        if (validate != null) {
-          throw new InvalidObjectException("Invalid skew column " + validate);
-        }
-      }
-
-      Map<String, String> transactionalListenerResponses = Collections.emptyMap();
-      Path tblPath = null;
-      boolean success = false, madeDir = false;
-      boolean isReplicated = false;
-      try {
-        firePreEvent(new PreCreateTableEvent(tbl, this));
-
-        ms.openTransaction();
-        isReplicated = isDbReplicationTarget(db);
-
-        // get_table checks whether database exists, it should be moved here
-        if (is_table_exists(ms, tbl.getCatName(), tbl.getDbName(), tbl.getTableName())) {
-          throw new AlreadyExistsException("Table " + getCatalogQualifiedTableName(tbl)
-              + " already exists");
-        }
-
-        if (!TableType.VIRTUAL_VIEW.toString().equals(tbl.getTableType())) {
-          if (tbl.getSd().getLocation() == null
-              || tbl.getSd().getLocation().isEmpty()) {
-            tblPath = wh.getDefaultTablePath(db, tbl);
-          } else {
-            if (!isExternal(tbl) && !MetaStoreUtils.isNonNativeTable(tbl)) {
-              LOG.warn("Location: " + tbl.getSd().getLocation()
-                  + " specified for non-external table:" + tbl.getTableName());
-            }
-            tblPath = wh.getDnsPath(new Path(tbl.getSd().getLocation()));
-          }
-          tbl.getSd().setLocation(tblPath.toString());
-        }
-
-        if (tblPath != null) {
-          if (!wh.isDir(tblPath)) {
-            if (!wh.mkdirs(tblPath)) {
-              throw new MetaException(tblPath
-                  + " is not a directory or unable to create one");
-            }
-            madeDir = true;
-          }
-        }
-        if (MetastoreConf.getBoolVar(conf, ConfVars.STATS_AUTO_GATHER) &&
-            !MetaStoreUtils.isView(tbl)) {
-          MetaStoreServerUtils.updateTableStatsSlow(db, tbl, wh, madeDir, false, envContext);
-        }
-
-        // set create time
-        long time = System.currentTimeMillis() / 1000;
-        tbl.setCreateTime((int) time);
-        if (tbl.getParameters() == null ||
-            tbl.getParameters().get(hive_metastoreConstants.DDL_TIME) == null) {
-          tbl.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(time));
-        }
-
-        if (CollectionUtils.isEmpty(constraints.getPrimaryKeys()) && CollectionUtils.isEmpty(constraints.getForeignKeys())
-                && CollectionUtils.isEmpty(constraints.getUniqueConstraints())&& CollectionUtils.isEmpty(constraints.getNotNullConstraints())&& CollectionUtils.isEmpty(constraints.getDefaultConstraints())
-            && CollectionUtils.isEmpty(constraints.getCheckConstraints())) {
-          ms.createTable(tbl);
-        } else {
-          final String catName = tbl.getCatName();
-          // Check that constraints have catalog name properly set first
-          if (CollectionUtils.isNotEmpty(constraints.getPrimaryKeys()) && !constraints.getPrimaryKeys().get(0).isSetCatName()) {
-            constraints.getPrimaryKeys().forEach(constraint -> constraint.setCatName(catName));
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getForeignKeys()) && !constraints.getForeignKeys().get(0).isSetCatName()) {
-            constraints.getForeignKeys().forEach(constraint -> constraint.setCatName(catName));
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getUniqueConstraints()) && !constraints.getUniqueConstraints().get(0).isSetCatName()) {
-            constraints.getUniqueConstraints().forEach(constraint -> constraint.setCatName(catName));
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getNotNullConstraints()) && !constraints.getNotNullConstraints().get(0).isSetCatName()) {
-            constraints.getNotNullConstraints().forEach(constraint -> constraint.setCatName(catName));
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getDefaultConstraints()) && !constraints.getDefaultConstraints().get(0).isSetCatName()) {
-            constraints.getDefaultConstraints().forEach(constraint -> constraint.setCatName(catName));
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getCheckConstraints()) && !constraints.getCheckConstraints().get(0).isSetCatName()) {
-            constraints.getCheckConstraints().forEach(constraint -> constraint.setCatName(catName));
-          }
-          // Set constraint name if null before sending to listener
-          constraints = ms.createTableWithConstraints(tbl, constraints);
-
-        }
-
-        if (!transactionalListeners.isEmpty()) {
-          transactionalListenerResponses = MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-              EventType.CREATE_TABLE, new CreateTableEvent(tbl, true, this, isReplicated), envContext);
-          if (CollectionUtils.isNotEmpty(constraints.getPrimaryKeys())) {
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners, EventType.ADD_PRIMARYKEY,
-                new AddPrimaryKeyEvent(constraints.getPrimaryKeys(), true, this), envContext);
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getForeignKeys())) {
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners, EventType.ADD_FOREIGNKEY,
-                new AddForeignKeyEvent(constraints.getForeignKeys(), true, this), envContext);
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getUniqueConstraints())) {
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners, EventType.ADD_UNIQUECONSTRAINT,
-                new AddUniqueConstraintEvent(constraints.getUniqueConstraints(), true, this), envContext);
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getNotNullConstraints())) {
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners, EventType.ADD_NOTNULLCONSTRAINT,
-                new AddNotNullConstraintEvent(constraints.getNotNullConstraints(), true, this), envContext);
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getCheckConstraints())) {
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners, EventType.ADD_CHECKCONSTRAINT,
-                new AddCheckConstraintEvent(constraints.getCheckConstraints(), true, this), envContext);
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getDefaultConstraints())) {
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners, EventType.ADD_DEFAULTCONSTRAINT,
-                new AddDefaultConstraintEvent(constraints.getDefaultConstraints(), true, this), envContext);
-          }
-        }
-
-        success = ms.commitTransaction();
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-          if (madeDir) {
-            wh.deleteDir(tblPath, true, false, ReplChangeManager.shouldEnableCm(db, tbl));
-          }
-        }
-
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners, EventType.CREATE_TABLE,
-              new CreateTableEvent(tbl, success, this, isReplicated), envContext,
-                  transactionalListenerResponses, ms);
-          if (CollectionUtils.isNotEmpty(constraints.getPrimaryKeys())) {
-            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ADD_PRIMARYKEY,
-                new AddPrimaryKeyEvent(constraints.getPrimaryKeys(), success, this), envContext);
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getForeignKeys())) {
-            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ADD_FOREIGNKEY,
-                new AddForeignKeyEvent(constraints.getForeignKeys(), success, this), envContext);
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getUniqueConstraints())) {
-            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ADD_UNIQUECONSTRAINT,
-                new AddUniqueConstraintEvent(constraints.getUniqueConstraints(), success, this), envContext);
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getNotNullConstraints())) {
-            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ADD_NOTNULLCONSTRAINT,
-                new AddNotNullConstraintEvent(constraints.getNotNullConstraints(), success, this), envContext);
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getDefaultConstraints())) {
-            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ADD_DEFAULTCONSTRAINT,
-                new AddDefaultConstraintEvent(constraints.getDefaultConstraints(), success, this), envContext);
-          }
-          if (CollectionUtils.isNotEmpty(constraints.getCheckConstraints())) {
-            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ADD_CHECKCONSTRAINT,
-                new AddCheckConstraintEvent(constraints.getCheckConstraints(), success, this), envContext);
-          }
-        }
-      }
-
-      // If the table has column statistics, update it into the metastore. We need a valid
-      // writeId list to update column statistics for a transactional table. But during bootstrap
-      // replication, where we use this feature, we do not have a valid writeId list which was
-      // used to update the stats. But we know for sure that the writeId associated with the
-      // stats was valid then (otherwise stats update would have failed on the source). So, craft
-      // a valid transaction list with only that writeId and use it to update the stats.
-      if (colStats != null) {
-        long writeId = tbl.getWriteId();
-        String validWriteIds = null;
-        if (writeId > 0) {
-          ValidWriteIdList validWriteIdList =
-                  new ValidReaderWriteIdList(TableName.getDbTable(tbl.getDbName(),
-                                                                  tbl.getTableName()),
-                                              new long[0], new BitSet(), writeId);
-          validWriteIds = validWriteIdList.toString();
-        }
-        updateTableColumnStatsInternal(colStats, validWriteIds, tbl.getWriteId());
-      }
-    }
-
-    @Override
-    public void create_table(final Table tbl) throws AlreadyExistsException,
-        MetaException, InvalidObjectException, InvalidInputException {
-      create_table_with_environment_context(tbl, null);
-    }
-
-    @Override
-    public void create_table_with_environment_context(final Table tbl,
-        final EnvironmentContext envContext)
-        throws AlreadyExistsException, MetaException, InvalidObjectException,
-            InvalidInputException {
-      startFunction("create_table", ": " + tbl.toString());
-      boolean success = false;
-      Exception ex = null;
-      try {
-        create_table_core(getMS(), tbl, envContext);
-        success = true;
-      } catch (NoSuchObjectException e) {
-        LOG.warn("create_table_with_environment_context got ", e);
-        ex = e;
-        throw new InvalidObjectException(e.getMessage());
-      } catch (MetaException | InvalidObjectException | AlreadyExistsException | InvalidInputException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("create_table", success, ex, tbl.getTableName());
-      }
-    }
-
-    @Override
-    public void create_table_req(final CreateTableRequest req)
-            throws AlreadyExistsException, MetaException, InvalidObjectException,
-            InvalidInputException {
-      Table tbl = req.getTable();
-      startFunction("create_table_req", ": " + tbl.toString());
-      boolean success = false;
-      Exception ex = null;
-      try {
-        create_table_core(getMS(), req);
-        success = true;
-      } catch (NoSuchObjectException e) {
-        LOG.warn("create_table_req got ", e);
-        ex = e;
-        throw new InvalidObjectException(e.getMessage());
-      } catch (MetaException | InvalidObjectException | AlreadyExistsException | InvalidInputException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("create_table_req", success, ex, tbl.getTableName());
-      }
-    }
-
-    @Override
-    public void create_table_with_constraints(final Table tbl,
-        final List<SQLPrimaryKey> primaryKeys, final List<SQLForeignKey> foreignKeys,
-        List<SQLUniqueConstraint> uniqueConstraints,
-        List<SQLNotNullConstraint> notNullConstraints,
-        List<SQLDefaultConstraint> defaultConstraints,
-        List<SQLCheckConstraint> checkConstraints)
-        throws AlreadyExistsException, MetaException, InvalidObjectException,
-            InvalidInputException {
-      startFunction("create_table", ": " + tbl.toString());
-      boolean success = false;
-      Exception ex = null;
-      try {
-        CreateTableRequest req = new CreateTableRequest(tbl);
-        req.setPrimaryKeys(primaryKeys);
-        req.setForeignKeys(foreignKeys);
-        req.setUniqueConstraints(uniqueConstraints);
-        req.setNotNullConstraints(notNullConstraints);
-        req.setDefaultConstraints(defaultConstraints);
-        req.setCheckConstraints(checkConstraints);
-        create_table_req(req);
-        success = true;
-      } catch (MetaException | InvalidObjectException | AlreadyExistsException |
-              InvalidInputException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("create_table_with_constraints", success, ex, tbl.getTableName());
-      }
-    }
-
-    @Override
-    public void drop_constraint(DropConstraintRequest req)
-        throws MetaException, InvalidObjectException {
-      String catName = req.isSetCatName() ? req.getCatName() : getDefaultCatalog(conf);
-      String dbName = req.getDbname();
-      String tableName = req.getTablename();
-      String constraintName = req.getConstraintname();
-      startFunction("drop_constraint", ": " + constraintName);
-      boolean success = false;
-      Exception ex = null;
-      RawStore ms = getMS();
-      try {
-        ms.openTransaction();
-        ms.dropConstraint(catName, dbName, tableName, constraintName);
-        if (transactionalListeners.size() > 0) {
-          DropConstraintEvent dropConstraintEvent = new DropConstraintEvent(catName, dbName,
-              tableName, constraintName, true, this);
-          for (MetaStoreEventListener transactionalListener : transactionalListeners) {
-            transactionalListener.onDropConstraint(dropConstraintEvent);
-          }
-        }
-        success = ms.commitTransaction();
-      } catch (NoSuchObjectException e) {
-        ex = e;
-        throw new InvalidObjectException(e.getMessage());
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        } else {
-          for (MetaStoreEventListener listener : listeners) {
-            DropConstraintEvent dropConstraintEvent = new DropConstraintEvent(catName, dbName,
-                tableName, constraintName, true, this);
-            listener.onDropConstraint(dropConstraintEvent);
-          }
-        }
-        endFunction("drop_constraint", success, ex, constraintName);
-      }
-    }
-
-    @Override
-    public void add_primary_key(AddPrimaryKeyRequest req)
-      throws MetaException, InvalidObjectException {
-      List<SQLPrimaryKey> primaryKeyCols = req.getPrimaryKeyCols();
-      String constraintName = (CollectionUtils.isNotEmpty(primaryKeyCols)) ?
-        primaryKeyCols.get(0).getPk_name() : "null";
-      startFunction("add_primary_key", ": " + constraintName);
-      boolean success = false;
-      Exception ex = null;
-      if (CollectionUtils.isNotEmpty(primaryKeyCols) && !primaryKeyCols.get(0).isSetCatName()) {
-        String defaultCat = getDefaultCatalog(conf);
-        primaryKeyCols.forEach(pk -> pk.setCatName(defaultCat));
-      }
-      RawStore ms = getMS();
-      try {
-        ms.openTransaction();
-        List<SQLPrimaryKey> primaryKeys = ms.addPrimaryKeys(primaryKeyCols);
-        if (transactionalListeners.size() > 0) {
-          if (CollectionUtils.isNotEmpty(primaryKeys)) {
-            AddPrimaryKeyEvent addPrimaryKeyEvent = new AddPrimaryKeyEvent(primaryKeys, true, this);
-            for (MetaStoreEventListener transactionalListener : transactionalListeners) {
-              transactionalListener.onAddPrimaryKey(addPrimaryKeyEvent);
-            }
-          }
-        }
-        success = ms.commitTransaction();
-      } catch (MetaException | InvalidObjectException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        } else if (primaryKeyCols != null && primaryKeyCols.size() > 0) {
-          for (MetaStoreEventListener listener : listeners) {
-            AddPrimaryKeyEvent addPrimaryKeyEvent = new AddPrimaryKeyEvent(primaryKeyCols, true, this);
-            listener.onAddPrimaryKey(addPrimaryKeyEvent);
-          }
-        }
-        endFunction("add_primary_key", success, ex, constraintName);
-      }
-    }
-
-    @Override
-    public void add_foreign_key(AddForeignKeyRequest req)
-      throws MetaException, InvalidObjectException {
-      List<SQLForeignKey> foreignKeys = req.getForeignKeyCols();
-      String constraintName = CollectionUtils.isNotEmpty(foreignKeys) ?
-        foreignKeys.get(0).getFk_name() : "null";
-      startFunction("add_foreign_key", ": " + constraintName);
-      boolean success = false;
-      Exception ex = null;
-      if (CollectionUtils.isNotEmpty(foreignKeys) && !foreignKeys.get(0).isSetCatName()) {
-        String defaultCat = getDefaultCatalog(conf);
-        foreignKeys.forEach(pk -> pk.setCatName(defaultCat));
-      }
-      RawStore ms = getMS();
-      try {
-        ms.openTransaction();
-        foreignKeys = ms.addForeignKeys(foreignKeys);
-        if (transactionalListeners.size() > 0) {
-          if (CollectionUtils.isNotEmpty(foreignKeys)) {
-            AddForeignKeyEvent addForeignKeyEvent = new AddForeignKeyEvent(foreignKeys, true, this);
-            for (MetaStoreEventListener transactionalListener : transactionalListeners) {
-              transactionalListener.onAddForeignKey(addForeignKeyEvent);
-            }
-          }
-        }
-        success = ms.commitTransaction();
-      } catch (MetaException | InvalidObjectException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        } else if (CollectionUtils.isNotEmpty(foreignKeys)) {
-          for (MetaStoreEventListener listener : listeners) {
-            AddForeignKeyEvent addForeignKeyEvent = new AddForeignKeyEvent(foreignKeys, true, this);
-            listener.onAddForeignKey(addForeignKeyEvent);
-          }
-        }
-        endFunction("add_foreign_key", success, ex, constraintName);
-      }
-    }
-
-    @Override
-    public void add_unique_constraint(AddUniqueConstraintRequest req)
-      throws MetaException, InvalidObjectException {
-      List<SQLUniqueConstraint> uniqueConstraints = req.getUniqueConstraintCols();
-      String constraintName = (uniqueConstraints != null && uniqueConstraints.size() > 0) ?
-              uniqueConstraints.get(0).getUk_name() : "null";
-      startFunction("add_unique_constraint", ": " + constraintName);
-      boolean success = false;
-      Exception ex = null;
-      if (!uniqueConstraints.isEmpty() && !uniqueConstraints.get(0).isSetCatName()) {
-        String defaultCat = getDefaultCatalog(conf);
-        uniqueConstraints.forEach(pk -> pk.setCatName(defaultCat));
-      }
-      RawStore ms = getMS();
-      try {
-        ms.openTransaction();
-        uniqueConstraints = ms.addUniqueConstraints(uniqueConstraints);
-        if (transactionalListeners.size() > 0) {
-          if (CollectionUtils.isNotEmpty(uniqueConstraints)) {
-            AddUniqueConstraintEvent addUniqueConstraintEvent = new AddUniqueConstraintEvent(uniqueConstraints, true, this);
-            for (MetaStoreEventListener transactionalListener : transactionalListeners) {
-              transactionalListener.onAddUniqueConstraint(addUniqueConstraintEvent);
-            }
-          }
-        }
-        success = ms.commitTransaction();
-      } catch (MetaException | InvalidObjectException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        } else if (CollectionUtils.isNotEmpty(uniqueConstraints)) {
-          for (MetaStoreEventListener listener : listeners) {
-            AddUniqueConstraintEvent addUniqueConstraintEvent = new AddUniqueConstraintEvent(uniqueConstraints, true, this);
-            listener.onAddUniqueConstraint(addUniqueConstraintEvent);
-          }
-        }
-        endFunction("add_unique_constraint", success, ex, constraintName);
-      }
-    }
-
-    @Override
-    public void add_not_null_constraint(AddNotNullConstraintRequest req)
-      throws MetaException, InvalidObjectException {
-      List<SQLNotNullConstraint> notNullConstraints = req.getNotNullConstraintCols();
-      String constraintName = (notNullConstraints != null && notNullConstraints.size() > 0) ?
-              notNullConstraints.get(0).getNn_name() : "null";
-      startFunction("add_not_null_constraint", ": " + constraintName);
-      boolean success = false;
-      Exception ex = null;
-      if (!notNullConstraints.isEmpty() && !notNullConstraints.get(0).isSetCatName()) {
-        String defaultCat = getDefaultCatalog(conf);
-        notNullConstraints.forEach(pk -> pk.setCatName(defaultCat));
-      }
-      RawStore ms = getMS();
-      try {
-        ms.openTransaction();
-        notNullConstraints = ms.addNotNullConstraints(notNullConstraints);
-
-        if (transactionalListeners.size() > 0) {
-          if (CollectionUtils.isNotEmpty(notNullConstraints)) {
-            AddNotNullConstraintEvent addNotNullConstraintEvent = new AddNotNullConstraintEvent(notNullConstraints, true, this);
-            for (MetaStoreEventListener transactionalListener : transactionalListeners) {
-              transactionalListener.onAddNotNullConstraint(addNotNullConstraintEvent);
-            }
-          }
-        }
-        success = ms.commitTransaction();
-      } catch (MetaException | InvalidObjectException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        } else if (CollectionUtils.isNotEmpty(notNullConstraints)) {
-          for (MetaStoreEventListener listener : listeners) {
-            AddNotNullConstraintEvent addNotNullConstraintEvent = new AddNotNullConstraintEvent(notNullConstraints, true, this);
-            listener.onAddNotNullConstraint(addNotNullConstraintEvent);
-          }
-        }
-        endFunction("add_not_null_constraint", success, ex, constraintName);
-      }
-    }
-
-    @Override
-    public void add_default_constraint(AddDefaultConstraintRequest req) throws MetaException, InvalidObjectException {
-      List<SQLDefaultConstraint> defaultConstraints = req.getDefaultConstraintCols();
-      String constraintName =
-          CollectionUtils.isNotEmpty(defaultConstraints) ? defaultConstraints.get(0).getDc_name() : "null";
-      startFunction("add_default_constraint", ": " + constraintName);
-      boolean success = false;
-      Exception ex = null;
-      if (!defaultConstraints.isEmpty() && !defaultConstraints.get(0).isSetCatName()) {
-        String defaultCat = getDefaultCatalog(conf);
-        defaultConstraints.forEach(pk -> pk.setCatName(defaultCat));
-      }
-      RawStore ms = getMS();
-      try {
-        ms.openTransaction();
-        defaultConstraints = ms.addDefaultConstraints(defaultConstraints);
-        if (transactionalListeners.size() > 0) {
-          if (CollectionUtils.isNotEmpty(defaultConstraints)) {
-            AddDefaultConstraintEvent addDefaultConstraintEvent =
-                new AddDefaultConstraintEvent(defaultConstraints, true, this);
-            for (MetaStoreEventListener transactionalListener : transactionalListeners) {
-              transactionalListener.onAddDefaultConstraint(addDefaultConstraintEvent);
-            }
-          }
-        }
-        success = ms.commitTransaction();
-      } catch (MetaException | InvalidObjectException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        } else if (CollectionUtils.isNotEmpty(defaultConstraints)) {
-          for (MetaStoreEventListener listener : listeners) {
-            AddDefaultConstraintEvent addDefaultConstraintEvent =
-                new AddDefaultConstraintEvent(defaultConstraints, true, this);
-            listener.onAddDefaultConstraint(addDefaultConstraintEvent);
-          }
-        }
-        endFunction("add_default_constraint", success, ex, constraintName);
-      }
-    }
-
-    @Override
-    public void add_check_constraint(AddCheckConstraintRequest req)
-        throws MetaException, InvalidObjectException {
-      List<SQLCheckConstraint> checkConstraints= req.getCheckConstraintCols();
-      String constraintName = CollectionUtils.isNotEmpty(checkConstraints) ?
-          checkConstraints.get(0).getDc_name() : "null";
-      startFunction("add_check_constraint", ": " + constraintName);
-      boolean success = false;
-      Exception ex = null;
-      if (!checkConstraints.isEmpty() && !checkConstraints.get(0).isSetCatName()) {
-        String defaultCat = getDefaultCatalog(conf);
-        checkConstraints.forEach(pk -> pk.setCatName(defaultCat));
-      }
-      RawStore ms = getMS();
-      try {
-        ms.openTransaction();
-        checkConstraints = ms.addCheckConstraints(checkConstraints);
-        if (transactionalListeners.size() > 0) {
-          if (CollectionUtils.isNotEmpty(checkConstraints)) {
-            AddCheckConstraintEvent addcheckConstraintEvent = new AddCheckConstraintEvent(checkConstraints, true, this);
-            for (MetaStoreEventListener transactionalListener : transactionalListeners) {
-             transactionalListener.onAddCheckConstraint(addcheckConstraintEvent);
-            }
-          }
-        }
-        success = ms.commitTransaction();
-      } catch (MetaException | InvalidObjectException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        } else if (CollectionUtils.isNotEmpty(checkConstraints)) {
-          for (MetaStoreEventListener listener : listeners) {
-            AddCheckConstraintEvent addCheckConstraintEvent = new AddCheckConstraintEvent(checkConstraints, true, this);
-            listener.onAddCheckConstraint(addCheckConstraintEvent);
-          }
-        }
-        endFunction("add_check_constraint", success, ex, constraintName);
-      }
-    }
-
-    private boolean is_table_exists(RawStore ms, String catName, String dbname, String name)
-        throws MetaException {
-      return (ms.getTable(catName, dbname, name, null) != null);
-    }
-
-    private boolean drop_table_core(final RawStore ms, final String catName, final String dbname,
-                                    final String name, final boolean deleteData,
-                                    final EnvironmentContext envContext, final String indexName)
-        throws NoSuchObjectException, MetaException, IOException, InvalidObjectException,
-        InvalidInputException {
-      boolean success = false;
-      boolean tableDataShouldBeDeleted = false;
-      Path tblPath = null;
-      List<Path> partPaths = null;
-      Table tbl = null;
-      boolean ifPurge = false;
-      Map<String, String> transactionalListenerResponses = Collections.emptyMap();
-      Database db = null;
-      boolean isReplicated = false;
-      try {
-        ms.openTransaction();
-        db = ms.getDatabase(catName, dbname);
-        if (db.getType() == DatabaseType.REMOTE) {
-          return DataConnectorProviderFactory.getDataConnectorProvider(db).dropTable(name);
-        }
-        isReplicated = isDbReplicationTarget(db);
-
-        // drop any partitions
-        tbl = get_table_core(catName, dbname, name, null);
-        if (tbl == null) {
-          throw new NoSuchObjectException(name + " doesn't exist");
-        }
-
-        // Check if table is part of a materialized view.
-        // If it is, it cannot be dropped.
-        List<String> isPartOfMV = ms.isPartOfMaterializedView(catName, dbname, name);
-        if (!isPartOfMV.isEmpty()) {
-            throw new MetaException(String.format("Cannot drop table as it is used in the following materialized" +
-                    " views %s%n", isPartOfMV));
-        }
-
-        if (tbl.getSd() == null) {
-          throw new MetaException("Table metadata is corrupted");
-        }
-        ifPurge = isMustPurge(envContext, tbl);
-
-        firePreEvent(new PreDropTableEvent(tbl, deleteData, this));
-
-        tableDataShouldBeDeleted = checkTableDataShouldBeDeleted(tbl, deleteData);
-        if (tbl.getSd().getLocation() != null) {
-          tblPath = new Path(tbl.getSd().getLocation());
-          if (!wh.isWritable(tblPath.getParent())) {
-            String target = indexName == null ? "Table" : "Index table";
-            throw new MetaException(target + " metadata not deleted since " +
-                tblPath.getParent() + " is not writable by " +
-                SecurityUtils.getUser());
-          }
-        }
-
-        // Drop the partitions and get a list of locations which need to be deleted
-        partPaths = dropPartitionsAndGetLocations(ms, catName, dbname, name, tblPath,
-            tableDataShouldBeDeleted);
-
-        // Drop any constraints on the table
-        ms.dropConstraint(catName, dbname, name, null, true);
-
-        if (!ms.dropTable(catName, dbname, name)) {
-          String tableName = TableName.getQualified(catName, dbname, name);
-          throw new MetaException(indexName == null ? "Unable to drop table " + tableName:
-              "Unable to drop index table " + tableName + " for index " + indexName);
-        } else {
-          if (!transactionalListeners.isEmpty()) {
-            transactionalListenerResponses =
-                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                      EventType.DROP_TABLE,
-                                                      new DropTableEvent(tbl, true, deleteData,
-                                                              this, isReplicated),
-                                                      envContext);
-          }
-          success = ms.commitTransaction();
-        }
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        } else if (tableDataShouldBeDeleted) {
-          // Data needs deletion. Check if trash may be skipped.
-          // Delete the data in the partitions which have other locations
-          deletePartitionData(partPaths, ifPurge, ReplChangeManager.shouldEnableCm(db, tbl));
-          // Delete the data in the table
-          deleteTableData(tblPath, ifPurge, ReplChangeManager.shouldEnableCm(db, tbl));
-        }
-
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                EventType.DROP_TABLE,
-                                                new DropTableEvent(tbl, success, deleteData, this, isReplicated),
-                                                envContext,
-                                                transactionalListenerResponses, ms);
-        }
-      }
-      return success;
-    }
-
-    private boolean checkTableDataShouldBeDeleted(Table tbl, boolean deleteData) {
-      if (deleteData && isExternal(tbl)) {
-        // External table data can be deleted if EXTERNAL_TABLE_PURGE is true
-        return isExternalTablePurge(tbl);
-      }
-      return deleteData;
-    }
-
-    /**
-     * Deletes the data in a table's location, if it fails logs an error
-     *
-     * @param tablePath
-     * @param ifPurge completely purge the table (skipping trash) while removing
-     *                data from warehouse
-     * @param shouldEnableCm If cm should be enabled
-     */
-    private void deleteTableData(Path tablePath, boolean ifPurge, boolean shouldEnableCm) {
-      if (tablePath != null) {
-        deleteDataExcludeCmroot(tablePath, ifPurge, shouldEnableCm);
-      }
-    }
-
-    /**
-     * Deletes the data in a table's location, if it fails logs an error.
-     *
-     * @param tablePath
-     * @param ifPurge completely purge the table (skipping trash) while removing
-     *                data from warehouse
-     * @param db Database
-     */
-    private void deleteTableData(Path tablePath, boolean ifPurge, Database db) {
-      if (tablePath != null) {
-        try {
-          wh.deleteDir(tablePath, true, ifPurge, db);
-        } catch (Exception e) {
-          LOG.error("Failed to delete table directory: " + tablePath +
-                  " " + e.getMessage());
-        }
-      }
-    }
-
-    /**
-    * Give a list of partitions' locations, tries to delete each one
-    * and for each that fails logs an error.
-    *
-    * @param partPaths
-    * @param ifPurge completely purge the partition (skipping trash) while
-    *                removing data from warehouse
-    * @param shouldEnableCm If cm should be enabled
-    */
-    private void deletePartitionData(List<Path> partPaths, boolean ifPurge, boolean shouldEnableCm) {
-      if (partPaths != null && !partPaths.isEmpty()) {
-        for (Path partPath : partPaths) {
-          deleteDataExcludeCmroot(partPath, ifPurge, shouldEnableCm);
-        }
-      }
-    }
-
-    /**
-     * Give a list of partitions' locations, tries to delete each one
-     * and for each that fails logs an error.
-     *
-     * @param partPaths
-     * @param ifPurge completely purge the partition (skipping trash) while
-     *                removing data from warehouse
-     * @param db Database
-     */
-    private void deletePartitionData(List<Path> partPaths, boolean ifPurge, Database db) {
-      if (partPaths != null && !partPaths.isEmpty()) {
-        for (Path partPath : partPaths) {
-          try {
-            wh.deleteDir(partPath, true, ifPurge, db);
-          } catch (Exception e) {
-            LOG.error("Failed to delete partition directory: " + partPath +
-                    " " + e.getMessage());
-          }
-        }
-      }
-    }
-
-    /**
-     * Delete data from path excluding cmdir
-     * and for each that fails logs an error.
-     *
-     * @param path
-     * @param ifPurge completely purge the partition (skipping trash) while
-     *                removing data from warehouse
-     * @param shouldEnableCm If cm should be enabled
-     */
-    private void deleteDataExcludeCmroot(Path path, boolean ifPurge, boolean shouldEnableCm) {
-      try {
-        if (shouldEnableCm) {
-          //Don't delete cmdir if its inside the partition path
-          FileStatus[] statuses = path.getFileSystem(conf).listStatus(path,
-                  ReplChangeManager.CMROOT_PATH_FILTER);
-          for (final FileStatus status : statuses) {
-            wh.deleteDir(status.getPath(), true, ifPurge, shouldEnableCm);
-          }
-          //Check if table directory is empty, delete it
-          FileStatus[] statusWithoutFilter = path.getFileSystem(conf).listStatus(path);
-          if (statusWithoutFilter.length == 0) {
-            wh.deleteDir(path, true, ifPurge, shouldEnableCm);
-          }
-        } else {
-          //If no cm delete the complete table directory
-          wh.deleteDir(path, true, ifPurge, shouldEnableCm);
-        }
-      } catch (Exception e) {
-        LOG.error("Failed to delete directory: " + path +
-                " " + e.getMessage());
-      }
-    }
-
-    /**
-     * Deletes the partitions specified by catName, dbName, tableName. If checkLocation is true, for
-     * locations of partitions which may not be subdirectories of tablePath checks to make sure the
-     * locations are writable.
-     *
-     * Drops the metadata for each partition.
-     *
-     * Provides a list of locations of partitions which may not be subdirectories of tablePath.
-     *
-     * @param ms RawStore to use for metadata retrieval and delete
-     * @param catName The catName
-     * @param dbName The dbName
-     * @param tableName The tableName
-     * @param tablePath The tablePath of which subdirectories does not have to be checked
-     * @param checkLocation Should we check the locations at all
-     * @return The list of the Path objects to delete (only in case checkLocation is true)
-     * @throws MetaException
-     * @throws IOException
-     * @throws NoSuchObjectException
-     */
-    private List<Path> dropPartitionsAndGetLocations(RawStore ms, String catName, String dbName,
-        String tableName, Path tablePath, boolean checkLocation)
-        throws MetaException, IOException, NoSuchObjectException {
-      int batchSize = MetastoreConf.getIntVar(conf, ConfVars.BATCH_RETRIEVE_OBJECTS_MAX);
-      String tableDnsPath = null;
-      if (tablePath != null) {
-        tableDnsPath = wh.getDnsPath(tablePath).toString();
-      }
-
-      List<Path> partPaths = new ArrayList<>();
-      while (true) {
-        Map<String, String> partitionLocations = ms.getPartitionLocations(catName, dbName, tableName,
-            tableDnsPath, batchSize);
-        if (partitionLocations == null || partitionLocations.isEmpty()) {
-          // No more partitions left to drop. Return with the collected path list to delete.
-          return partPaths;
-        }
-
-        if (checkLocation) {
-          for (String partName : partitionLocations.keySet()) {
-            String pathString = partitionLocations.get(partName);
-            if (pathString != null) {
-              Path partPath = wh.getDnsPath(new Path(pathString));
-              // Double check here. Maybe Warehouse.getDnsPath revealed relationship between the
-              // path objects
-              if (tableDnsPath == null ||
-                      !FileUtils.isSubdirectory(tableDnsPath, partPath.toString())) {
-                if (!wh.isWritable(partPath.getParent())) {
-                  throw new MetaException("Table metadata not deleted since the partition "
-                      + partName + " has parent location " + partPath.getParent()
-                      + " which is not writable by " + SecurityUtils.getUser());
-                }
-                partPaths.add(partPath);
-              }
-            }
-          }
-        }
-
-        for (MetaStoreEventListener listener : listeners) {
-          //No drop part listener events fired for public listeners historically, for drop table case.
-          //Limiting to internal listeners for now, to avoid unexpected calls for public listeners.
-          if (listener instanceof HMSMetricsListener) {
-            for (@SuppressWarnings("unused") String partName : partitionLocations.keySet()) {
-              listener.onDropPartition(null);
-            }
-          }
-        }
-
-        ms.dropPartitions(catName, dbName, tableName, new ArrayList<>(partitionLocations.keySet()));
-      }
-    }
-
-    @Override
-    public void drop_table(final String dbname, final String name, final boolean deleteData)
-        throws NoSuchObjectException, MetaException {
-      drop_table_with_environment_context(dbname, name, deleteData, null);
-    }
-
-    @Override
-    public void drop_table_with_environment_context(final String dbname, final String name,
-        final boolean deleteData, final EnvironmentContext envContext)
-        throws NoSuchObjectException, MetaException {
-      String[] parsedDbName = parseDbName(dbname, conf);
-      startTableFunction("drop_table", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], name);
-
-      boolean success = false;
-      Exception ex = null;
-      try {
-        success = drop_table_core(getMS(), parsedDbName[CAT_NAME], parsedDbName[DB_NAME], name,
-            deleteData, envContext, null);
-      } catch (IOException e) {
-        ex = e;
-        throw new MetaException(e.getMessage());
-      } catch (Exception e) {
-        ex = e;
-        throwMetaException(e);
-      } finally {
-        endFunction("drop_table", success, ex, name);
-      }
-
-    }
-
-    private void updateStatsForTruncate(Map<String,String> props, EnvironmentContext environmentContext) {
-      if (null == props) {
-        return;
-      }
-      for (String stat : StatsSetupConst.SUPPORTED_STATS) {
-        String statVal = props.get(stat);
-        if (statVal != null) {
-          //In the case of truncate table, we set the stats to be 0.
-          props.put(stat, "0");
-        }
-      }
-      //first set basic stats to true
-      StatsSetupConst.setBasicStatsState(props, StatsSetupConst.TRUE);
-      environmentContext.putToProperties(StatsSetupConst.STATS_GENERATED, StatsSetupConst.TASK);
-      environmentContext.putToProperties(StatsSetupConst.DO_NOT_POPULATE_QUICK_STATS, StatsSetupConst.TRUE);
-      //then invalidate column stats
-      StatsSetupConst.clearColumnStatsState(props);
-      return;
-    }
-
-    private void alterPartitionForTruncate(RawStore ms, String catName, String dbName, String tableName,
-        Table table, Partition partition, String validWriteIds, long writeId) throws Exception {
-      EnvironmentContext environmentContext = new EnvironmentContext();
-      updateStatsForTruncate(partition.getParameters(), environmentContext);
-
-      if (!transactionalListeners.isEmpty()) {
-        MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                EventType.ALTER_PARTITION,
-                new AlterPartitionEvent(partition, partition, table, true, true,
-                        writeId, this));
-      }
-
-      if (!listeners.isEmpty()) {
-        MetaStoreListenerNotifier.notifyEvent(listeners,
-                EventType.ALTER_PARTITION,
-                new AlterPartitionEvent(partition, partition, table, true, true,
-                        writeId, this));
-      }
-
-      if (writeId > 0) {
-        partition.setWriteId(writeId);
-      }
-      alterHandler.alterPartition(ms, wh, catName, dbName, tableName, null, partition,
-          environmentContext, this, validWriteIds);
-    }
-
-    private void alterTableStatsForTruncate(RawStore ms, String catName, String dbName,
-        String tableName, Table table, List<String> partNames,
-        String validWriteIds, long writeId) throws Exception {
-      if (partNames == null) {
-        if (0 != table.getPartitionKeysSize()) {
-          for (Partition partition : ms.getPartitions(catName, dbName, tableName, -1)) {
-            alterPartitionForTruncate(ms, catName, dbName, tableName, table, partition,
-                validWriteIds, writeId);
-          }
-        } else {
-          EnvironmentContext environmentContext = new EnvironmentContext();
-          updateStatsForTruncate(table.getParameters(), environmentContext);
-
-          boolean isReplicated = isDbReplicationTarget(ms.getDatabase(catName, dbName));
-          if (!transactionalListeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                    EventType.ALTER_TABLE,
-                    new AlterTableEvent(table, table, true, true,
-                            writeId, this, isReplicated));
-          }
-
-          if (!listeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(listeners,
-                    EventType.ALTER_TABLE,
-                    new AlterTableEvent(table, table, true, true,
-                            writeId, this, isReplicated));
-          }
-
-          // TODO: this should actually pass thru and set writeId for txn stats.
-          if (writeId > 0) {
-            table.setWriteId(writeId);
-          }
-          alterHandler.alterTable(ms, wh, catName, dbName, tableName, table,
-              environmentContext, this, validWriteIds);
-        }
-      } else {
-        for (Partition partition : ms.getPartitionsByNames(catName, dbName, tableName, partNames)) {
-          alterPartitionForTruncate(ms, catName, dbName, tableName, table, partition,
-              validWriteIds, writeId);
-        }
-      }
-      return;
-    }
-
-    private List<Path> getLocationsForTruncate(final RawStore ms,
-                                               final String catName,
-                                               final String dbName,
-                                               final String tableName,
-                                               final Table table,
-                                               final List<String> partNames) throws Exception {
-      List<Path> locations = new ArrayList<>();
-      if (partNames == null) {
-        if (0 != table.getPartitionKeysSize()) {
-          for (Partition partition : ms.getPartitions(catName, dbName, tableName, -1)) {
-            locations.add(new Path(partition.getSd().getLocation()));
-          }
-        } else {
-          locations.add(new Path(table.getSd().getLocation()));
-        }
-      } else {
-        for (Partition partition : ms.getPartitionsByNames(catName, dbName, tableName, partNames)) {
-          locations.add(new Path(partition.getSd().getLocation()));
-        }
-      }
-      return locations;
-    }
-
-    @Override
-    public CmRecycleResponse cm_recycle(final CmRecycleRequest request) throws MetaException {
-      wh.recycleDirToCmPath(new Path(request.getDataPath()), request.isPurge());
-      return new CmRecycleResponse();
-    }
-
-    @Override
-    public void truncate_table(final String dbName, final String tableName, List<String> partNames)
-      throws NoSuchObjectException, MetaException {
-      // Deprecated path, won't work for txn tables.
-      truncateTableInternal(dbName, tableName, partNames, null, -1);
-    }
-
-    @Override
-    public TruncateTableResponse truncate_table_req(TruncateTableRequest req)
-        throws MetaException, TException {
-      truncateTableInternal(req.getDbName(), req.getTableName(), req.getPartNames(),
-          req.getValidWriteIdList(), req.getWriteId());
-      return new TruncateTableResponse();
-    }
-
-    private void truncateTableInternal(String dbName, String tableName, List<String> partNames,
-        String validWriteIds, long writeId) throws MetaException, NoSuchObjectException {
-      try {
-        String[] parsedDbName = parseDbName(dbName, conf);
-        Table tbl = get_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName);
-
-        boolean truncateFiles = !TxnUtils.isTransactionalTable(tbl) ||
-            !MetastoreConf.getBoolVar(getConf(), MetastoreConf.ConfVars.TRUNCATE_ACID_USE_BASE);
-        // This is not transactional
-        for (Path location : getLocationsForTruncate(getMS(), parsedDbName[CAT_NAME],
-            parsedDbName[DB_NAME], tableName, tbl, partNames)) {
-          FileSystem fs = location.getFileSystem(getConf());
-          if (truncateFiles) {
-            truncateDataFiles(tbl, parsedDbName, location, fs);
-          } else {
-            // For Acid tables we don't need to delete the old files, only write an empty baseDir.
-            // Compaction and cleaner will take care of the rest
-            addTruncateBaseFile(location, writeId, fs);
-          }
-        }
-
-        // Alter the table/partition stats and also notify truncate table event
-        alterTableStatsForTruncate(getMS(), parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-            tableName, tbl, partNames, validWriteIds, writeId);
-      } catch (IOException e) {
-        throw new MetaException(e.getMessage());
-      } catch (MetaException | NoSuchObjectException e) {
-        throw e;
-      } catch (Exception e) {
-        throw newMetaException(e);
-      }
-    }
-
-    /**
-     * Add an empty baseDir with a truncate metadatafile
-     * @param location partition or table directory
-     * @param writeId allocated writeId
-     * @param fs FileSystem
-     * @throws Exception
-     */
-    private void addTruncateBaseFile(Path location, long writeId, FileSystem fs) throws Exception {
-      Path basePath = new Path(location, AcidConstants.baseDir(writeId));
-      fs.mkdirs(basePath);
-      // We can not leave the folder empty, otherwise it will be skipped at some file listing in AcidUtils
-      // No need for a data file, a simple metadata is enough
-      AcidMetaDataFile.writeToFile(fs, basePath, AcidMetaDataFile.DataFormat.TRUNCATED);
-    }
-
-    private void truncateDataFiles(Table tbl, String[] parsedDbName, Path location, FileSystem fs)
-        throws IOException, MetaException, NoSuchObjectException {
-      boolean isSkipTrash = MetaStoreUtils.isSkipTrash(tbl.getParameters());
-      Database db = get_database_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME]);
-      if (!HdfsUtils.isPathEncrypted(getConf(), fs.getUri(), location) &&
-          !FileUtils.pathHasSnapshotSubDir(location, fs)) {
-        HdfsUtils.HadoopFileStatus status = new HdfsUtils.HadoopFileStatus(getConf(), fs, location);
-        FileStatus targetStatus = fs.getFileStatus(location);
-        String targetGroup = targetStatus == null ? null : targetStatus.getGroup();
-        wh.deleteDir(location, true, isSkipTrash, ReplChangeManager.shouldEnableCm(db, tbl));
-        fs.mkdirs(location);
-        HdfsUtils.setFullFileStatus(getConf(), status, targetGroup, fs, location, false);
-      } else {
-        FileStatus[] statuses = fs.listStatus(location, FileUtils.HIDDEN_FILES_PATH_FILTER);
-        if (statuses == null || statuses.length == 0) {
-          return;
-        }
-        for (final FileStatus status : statuses) {
-          wh.deleteDir(status.getPath(), true, isSkipTrash, ReplChangeManager.shouldEnableCm(db, tbl));
-        }
-      }
-    }
-
-    /**
-     * Is this an external table?
-     *
-     * @param table
-     *          Check if this table is external.
-     * @return True if the table is external, otherwise false.
-     */
-    private boolean isExternal(Table table) {
-      return MetaStoreUtils.isExternalTable(table);
-    }
-
-    private boolean isExternalTablePurge(Table table) {
-      return MetaStoreUtils.isExternalTablePurge(table);
-    }
-
-    @Override
-    @Deprecated
-    public Table get_table(final String dbname, final String name) throws MetaException,
-        NoSuchObjectException {
-      String[] parsedDbName = parseDbName(dbname, conf);
-      return getTableInternal(
-            parsedDbName[CAT_NAME], parsedDbName[DB_NAME], name, null, null, false, null, null, null);
-    }
-
-    @Override
-    public List<ExtendedTableInfo> get_tables_ext(final GetTablesExtRequest req) throws MetaException {
-      List<String> tables = new ArrayList<String>();
-      List<ExtendedTableInfo> ret = new ArrayList<ExtendedTableInfo>();
-      String catalog  = req.getCatalog();
-      String database = req.getDatabase();
-      String pattern  = req.getTableNamePattern();
-      List<String> processorCapabilities = req.getProcessorCapabilities();
-      int limit = req.getLimit();
-      String processorId  = req.getProcessorIdentifier();
-      List<Table> tObjects = new ArrayList<>();
-
-      startTableFunction("get_tables_ext", catalog, database, pattern);
-      Exception ex = null;
-      try {
-        tables = getMS().getTables(catalog, database, pattern, null, limit);
-        LOG.debug("get_tables_ext:getTables() returned " + tables.size());
-        tables = FilterUtils.filterTableNamesIfEnabled(isServerFilterEnabled, filterHook,
-                catalog, database, tables);
-
-        if (tables.size() > 0) {
-          tObjects = getMS().getTableObjectsByName(catalog, database, tables);
-          LOG.debug("get_tables_ext:getTableObjectsByName() returned " + tObjects.size());
-          if (processorCapabilities == null || processorCapabilities.size() == 0 ||
-                processorCapabilities.contains("MANAGERAWMETADATA")) {
-            LOG.info("Skipping translation for processor with " + processorId);
-          } else {
-            if (transformer != null) {
-              Map<Table, List<String>> retMap = transformer.transform(tObjects, processorCapabilities, processorId);
-
-              for (Map.Entry<Table, List<String>> entry : retMap.entrySet())  {
-                LOG.debug("Table " + entry.getKey().getTableName() + " requires " + Arrays.toString((entry.getValue()).toArray()));
-                ret.add(convertTableToExtendedTable(entry.getKey(), entry.getValue(), req.getRequestedFields()));
-              }
-            } else {
-              for (Table table : tObjects) {
-                ret.add(convertTableToExtendedTable(table, processorCapabilities, req.getRequestedFields()));
-              }
-            }
-          }
-        }
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_tables_ext", ret != null, ex);
-      }
-      return ret;
-    }
-
-    private ExtendedTableInfo convertTableToExtendedTable (Table table,
-          List<String> processorCapabilities, int mask) {
-      ExtendedTableInfo extTable = new ExtendedTableInfo(table.getTableName());
-      if ((mask & GetTablesExtRequestFields.ACCESS_TYPE.getValue()) == GetTablesExtRequestFields.ACCESS_TYPE.getValue())
-        extTable.setAccessType(table.getAccessType());
-
-      if ((mask & GetTablesExtRequestFields.PROCESSOR_CAPABILITIES.getValue())
-             == GetTablesExtRequestFields.PROCESSOR_CAPABILITIES.getValue()) {
-        extTable.setRequiredReadCapabilities(table.getRequiredReadCapabilities());
-        extTable.setRequiredWriteCapabilities(table.getRequiredWriteCapabilities());
-      }
-
-      return extTable;
-    }
-
-    @Override
-    public GetTableResult get_table_req(GetTableRequest req) throws MetaException,
-        NoSuchObjectException {
-      String catName = req.isSetCatName() ? req.getCatName() : getDefaultCatalog(conf);
-      return new GetTableResult(getTableInternal(catName, req.getDbName(), req.getTblName(),
-          req.getCapabilities(), req.getValidWriteIdList(), req.isGetColumnStats(), req.getEngine(),
-          req.getProcessorCapabilities(), req.getProcessorIdentifier()));
-    }
-
-    /**
-     * This function retrieves table from metastore. If getColumnStats flag is true,
-     * then engine should be specified so the table is retrieve with the column stats
-     * for that engine.
-     */
-    private Table getTableInternal(String catName, String dbname, String name,
-        ClientCapabilities capabilities, String writeIdList, boolean getColumnStats, String engine,
-        List<String> processorCapabilities, String processorId)
-        throws MetaException, NoSuchObjectException {
-      Preconditions.checkArgument(!getColumnStats || engine != null,
-          "To retrieve column statistics with a table, engine parameter cannot be null");
-
-      if (isInTest) {
-        assertClientHasCapability(capabilities, ClientCapability.TEST_CAPABILITY,
-            "Hive tests", "get_table_req");
-      }
-
-      Table t = null;
-      startTableFunction("get_table", catName, dbname, name);
-      Exception ex = null;
-      try {
-        t = get_table_core(catName, dbname, name, writeIdList, getColumnStats, engine);
-        if (MetaStoreUtils.isInsertOnlyTableParam(t.getParameters())) {
-          assertClientHasCapability(capabilities, ClientCapability.INSERT_ONLY_TABLES,
-              "insert-only tables", "get_table_req");
-        }
-
-        if (processorCapabilities == null || processorCapabilities.size() == 0 ||
-              processorCapabilities.contains("MANAGERAWMETADATA")) {
-          LOG.info("Skipping translation for processor with " + processorId);
-        } else {
-          if (transformer != null) {
-            List<Table> tList = new ArrayList<>();
-            tList.add(t);
-            Map<Table, List<String>> ret = transformer.transform(tList, processorCapabilities, processorId);
-            if (ret.size() > 1) {
-              LOG.warn("Unexpected resultset size:" + ret.size());
-              throw new MetaException("Unexpected result from metadata transformer:return list size is " + ret.size());
-            }
-            t = (Table)(ret.keySet().iterator().next());
-          }
-        }
-
-        firePreEvent(new PreReadTableEvent(t, this));
-      } catch (MetaException | NoSuchObjectException e) {
-        ex = e;
-        throw e;
-      } finally {
-        endFunction("get_table", t != null, ex, name);
-      }
-      return t;
-    }
-
-    @Override
-    public List<TableMeta> get_table_meta(String dbnames, String tblNames, List<String> tblTypes)
-        throws MetaException, NoSuchObjectException {
-      List<TableMeta> t = null;
-      String[] parsedDbName = parseDbName(dbnames, conf);
-      startTableFunction("get_table_metas", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblNames);
-      Exception ex = null;
-      try {
-        t = getMS().getTableMeta(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblNames, tblTypes);
-        t = FilterUtils.filterTableMetasIfEnabled(isServerFilterEnabled, filterHook,
-            parsedDbName[CAT_NAME], parsedDbName[DB_NAME], t);
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_table_metas", t != null, ex);
-      }
-      return t;
-    }
-
-    @Override
-    public Table get_table_core(
-        final String catName,
-        final String dbname,
-        final String name)
-        throws MetaException, NoSuchObjectException {
-      return get_table_core(catName, dbname, name, null);
-    }
-
-    @Override
-    public Table get_table_core(
-        final String catName,
-        final String dbname,
-        final String name,
-        final String writeIdList)
-        throws MetaException, NoSuchObjectException {
-      return get_table_core(catName, dbname, name, writeIdList, false, null);
-    }
-
-    /**
-     * This function retrieves table from metastore. If getColumnStats flag is true,
-     * then engine should be specified so the table is retrieve with the column stats
-     * for that engine.
-     */
-    public Table get_table_core(final String catName,
-        final String dbname,
-        final String name,
-        final String writeIdList,
-        boolean getColumnStats, String engine)
-        throws MetaException, NoSuchObjectException {
-      Preconditions.checkArgument(!getColumnStats || engine != null,
-          "To retrieve column statistics with a table, engine parameter cannot be null");
-
-      Database db = null;
-      Table t = null;
-      try {
-        db = get_database_core(catName, dbname);
-      } catch (Exception e) { /* appears exception is not thrown currently if db doesnt exist */ }
-
-      if (db != null) {
-        if (db.getType().equals(DatabaseType.REMOTE)) {
-          t = DataConnectorProviderFactory.getDataConnectorProvider(db).getTable(name);
-          if (t == null) {
-            throw new NoSuchObjectException(TableName.getQualified(catName, dbname, name) +
-              " table not found");
-          }
-          t.setDbName(dbname);
-          return t;
-        }
-      }
-
-      try {
-        t = getMS().getTable(catName, dbname, name, writeIdList);
-        if (t == null) {
-          throw new NoSuchObjectException(TableName.getQualified(catName, dbname, name) +
-            " table not found");
-        }
-
-        // If column statistics was requested and is valid fetch it.
-        if (getColumnStats) {
-          ColumnStatistics colStats = getMS().getTableColumnStatistics(catName, dbname, name,
-                  StatsSetupConst.getColumnsHavingStats(t.getParameters()), engine, writeIdList);
-          if (colStats != null) {
-            t.setColStats(colStats);
-          }
-        }
-      } catch (Exception e) {
-        throwMetaException(e);
-      }
-      return t;
-    }
-
-    /**
-     * Gets multiple tables from the hive metastore.
-     *
-     * @param dbName
-     *          The name of the database in which the tables reside
-     * @param tableNames
-     *          The names of the tables to get.
-     *
-     * @return A list of tables whose names are in the the list "names" and
-     *         are retrievable from the database specified by "dbnames."
-     *         There is no guarantee of the order of the returned tables.
-     *         If there are duplicate names, only one instance of the table will be returned.
-     * @throws MetaException
-     * @throws InvalidOperationException
-     * @throws UnknownDBException
-     */
-    @Override
-    @Deprecated
-    public List<Table> get_table_objects_by_name(final String dbName, final List<String> tableNames)
-        throws MetaException, InvalidOperationException, UnknownDBException {
-      String[] parsedDbName = parseDbName(dbName, conf);
-      return getTableObjectsInternal(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableNames, null);
-    }
-
-    @Override
-    public GetTablesResult get_table_objects_by_name_req(GetTablesRequest req) throws TException {
-      String catName = req.isSetCatName() ? req.getCatName() : getDefaultCatalog(conf);
-      return new GetTablesResult(getTableObjectsInternal(catName, req.getDbName(),
-          req.getTblNames(), req.getCapabilities()));
-    }
-
-    private List<Table> getTableObjectsInternal(String catName, String dbName,
-                                                List<String> tableNames,
-                                                ClientCapabilities capabilities)
-            throws MetaException, InvalidOperationException, UnknownDBException {
-      if (isInTest) {
-        assertClientHasCapability(capabilities, ClientCapability.TEST_CAPABILITY,
-            "Hive tests", "get_table_objects_by_name_req");
-      }
-      List<Table> tables = new ArrayList<>();
-      startMultiTableFunction("get_multi_table", dbName, tableNames);
-      Exception ex = null;
-      int tableBatchSize = MetastoreConf.getIntVar(conf,
-          ConfVars.BATCH_RETRIEVE_MAX);
-
-      try {
-        if (dbName == null || dbName.isEmpty()) {
-          throw new UnknownDBException("DB name is null or empty");
-        }
-        if (tableNames == null) {
-          throw new InvalidOperationException(dbName + " cannot find null tables");
-        }
-
-        // The list of table names could contain duplicates. RawStore.getTableObjectsByName()
-        // only guarantees returning no duplicate table objects in one batch. If we need
-        // to break into multiple batches, remove duplicates first.
-        List<String> distinctTableNames = tableNames;
-        if (distinctTableNames.size() > tableBatchSize) {
-          List<String> lowercaseTableNames = new ArrayList<>();
-          for (String tableName : tableNames) {
-            lowercaseTableNames.add(org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier(tableName));
-          }
-          distinctTableNames = new ArrayList<>(new HashSet<>(lowercaseTableNames));
-        }
-
-        RawStore ms = getMS();
-        int startIndex = 0;
-        // Retrieve the tables from the metastore in batches. Some databases like
-        // Oracle cannot have over 1000 expressions in a in-list
-        while (startIndex < distinctTableNames.size()) {
-          int endIndex = Math.min(startIndex + tableBatchSize, distinctTableNames.size());
-          tables.addAll(ms.getTableObjectsByName(catName, dbName, distinctTableNames.subList(
-              startIndex, endIndex)));
-          startIndex = endIndex;
-        }
-        for (Table t : tables) {
-          if (MetaStoreUtils.isInsertOnlyTableParam(t.getParameters())) {
-            assertClientHasCapability(capabilities, ClientCapability.INSERT_ONLY_TABLES,
-                "insert-only tables", "get_table_req");
-          }
-        }
-
-        FilterUtils.filterTablesIfEnabled(isServerFilterEnabled, filterHook, tables);
-      } catch (MetaException | InvalidOperationException | UnknownDBException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_multi_table", tables != null, ex, join(tableNames, ","));
-      }
-      return tables;
-    }
-
-    @Override
-    public Materialization get_materialization_invalidation_info(final CreationMetadata cm, final String validTxnList) throws MetaException {
-      return getTxnHandler().getMaterializationInvalidationInfo(cm, validTxnList);
-    }
-
-    @Override
-    public void update_creation_metadata(String catName, final String dbName, final String tableName, CreationMetadata cm) throws MetaException {
-      getMS().updateCreationMetadata(catName, dbName, tableName, cm);
-    }
-
-    private void assertClientHasCapability(ClientCapabilities client,
-        ClientCapability value, String what, String call) throws MetaException {
-      if (!doesClientHaveCapability(client, value)) {
-        throw new MetaException("Your client does not appear to support " + what + ". To skip"
-            + " capability checks, please set " + ConfVars.CAPABILITY_CHECK.toString()
-            + " to false. This setting can be set globally, or on the client for the current"
-            + " metastore session. Note that this may lead to incorrect results, data loss,"
-            + " undefined behavior, etc. if your client is actually incompatible. You can also"
-            + " specify custom client capabilities via " + call + " API.");
-      }
-    }
-
-    private boolean doesClientHaveCapability(ClientCapabilities client, ClientCapability value) {
-      if (!MetastoreConf.getBoolVar(getConf(), ConfVars.CAPABILITY_CHECK)) {
-        return true;
-      }
-      return (client != null && client.isSetValues() && client.getValues().contains(value));
-    }
-
-    @Override
-    public List<String> get_table_names_by_filter(
-        final String dbName, final String filter, final short maxTables)
-        throws MetaException, InvalidOperationException, UnknownDBException {
-      List<String> tables = null;
-      startFunction("get_table_names_by_filter", ": db = " + dbName + ", filter = " + filter);
-      Exception ex = null;
-      String[] parsedDbName = parseDbName(dbName, conf);
-      try {
-        if (parsedDbName[CAT_NAME] == null || parsedDbName[CAT_NAME].isEmpty() ||
-            parsedDbName[DB_NAME] == null || parsedDbName[DB_NAME].isEmpty()) {
-          throw new UnknownDBException("DB name is null or empty");
-        }
-        if (filter == null) {
-          throw new InvalidOperationException(filter + " cannot apply null filter");
-        }
-        tables = getMS().listTableNamesByFilter(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], filter, maxTables);
-        tables = FilterUtils.filterTableNamesIfEnabled(
-            isServerFilterEnabled, filterHook, parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tables);
-
-      } catch (MetaException | InvalidOperationException | UnknownDBException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_table_names_by_filter", tables != null, ex, join(tables, ","));
-      }
-      return tables;
-    }
-
-    private Partition append_partition_common(RawStore ms, String catName, String dbName,
-                                              String tableName, List<String> part_vals,
-                                              EnvironmentContext envContext)
-            throws InvalidObjectException, AlreadyExistsException, MetaException, NoSuchObjectException {
-
-      Partition part = new Partition();
-      boolean success = false, madeDir = false;
-      Path partLocation = null;
-      Table tbl = null;
-      Map<String, String> transactionalListenerResponses = Collections.emptyMap();
-      Database db = null;
-      try {
-        ms.openTransaction();
-        part.setCatName(catName);
-        part.setDbName(dbName);
-        part.setTableName(tableName);
-        part.setValues(part_vals);
-
-        MetaStoreServerUtils.validatePartitionNameCharacters(part_vals, partitionValidationPattern);
-
-        tbl = ms.getTable(part.getCatName(), part.getDbName(), part.getTableName(), null);
-        if (tbl == null) {
-          throw new InvalidObjectException(
-              "Unable to add partition because table or database do not exist");
-        }
-        if (tbl.getSd().getLocation() == null) {
-          throw new MetaException(
-              "Cannot append a partition to a view");
-        }
-
-        db = get_database_core(catName, dbName);
-
-        firePreEvent(new PreAddPartitionEvent(tbl, part, this));
-
-        part.setSd(tbl.getSd().deepCopy());
-        partLocation = new Path(tbl.getSd().getLocation(), Warehouse
-            .makePartName(tbl.getPartitionKeys(), part_vals));
-        part.getSd().setLocation(partLocation.toString());
-
-        Partition old_part;
-        try {
-          old_part = ms.getPartition(part.getCatName(), part.getDbName(), part
-              .getTableName(), part.getValues());
-        } catch (NoSuchObjectException e) {
-          // this means there is no existing partition
-          old_part = null;
-        }
-        if (old_part != null) {
-          throw new AlreadyExistsException("Partition already exists:" + part);
-        }
-
-        if (!wh.isDir(partLocation)) {
-          if (!wh.mkdirs(partLocation)) {
-            throw new MetaException(partLocation
-                + " is not a directory or unable to create one");
-          }
-          madeDir = true;
-        }
-
-        // set create time
-        long time = System.currentTimeMillis() / 1000;
-        part.setCreateTime((int) time);
-        part.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(time));
-
-        if (canUpdateStats(tbl)) {
-          MetaStoreServerUtils.updatePartitionStatsFast(part, tbl, wh, madeDir, false, envContext, true);
-        }
-
-        if (ms.addPartition(part)) {
-          if (!transactionalListeners.isEmpty()) {
-            transactionalListenerResponses =
-                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                      EventType.ADD_PARTITION,
-                                                      new AddPartitionEvent(tbl, part, true, this),
-                                                      envContext);
-          }
-
-          success = ms.commitTransaction();
-        }
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-          if (madeDir) {
-            wh.deleteDir(partLocation, true, false, ReplChangeManager.shouldEnableCm(db, tbl));
-          }
-        }
-
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                EventType.ADD_PARTITION,
-                                                new AddPartitionEvent(tbl, part, success, this),
-                                                envContext,
-                                                transactionalListenerResponses, ms);
-        }
-      }
-      return part;
-    }
-
-    private void firePreEvent(PreEventContext event) throws MetaException {
-      for (MetaStorePreEventListener listener : preListeners) {
-        try {
-          listener.onEvent(event);
-        } catch (NoSuchObjectException e) {
-          throw new MetaException(e.getMessage());
-        } catch (InvalidOperationException e) {
-          throw new MetaException(e.getMessage());
-        }
-      }
-    }
-
-    @Override
-    public Partition append_partition(final String dbName, final String tableName,
-        final List<String> part_vals) throws InvalidObjectException,
-        AlreadyExistsException, MetaException {
-      return append_partition_with_environment_context(dbName, tableName, part_vals, null);
-    }
-
-    @Override
-    public Partition append_partition_with_environment_context(final String dbName,
-        final String tableName, final List<String> part_vals, final EnvironmentContext envContext)
-        throws InvalidObjectException, AlreadyExistsException, MetaException {
-      if (part_vals == null || part_vals.isEmpty()) {
-        throw new MetaException("The partition values must not be null or empty.");
-      }
-      String[] parsedDbName = parseDbName(dbName, conf);
-      startPartitionFunction("append_partition", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, part_vals);
-      if (LOG.isDebugEnabled()) {
-        for (String part : part_vals) {
-          LOG.debug(part);
-        }
-      }
-
-      Partition ret = null;
-      Exception ex = null;
-      try {
-        ret = append_partition_common(getMS(), parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, part_vals, envContext);
-      } catch (MetaException | InvalidObjectException | AlreadyExistsException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("append_partition", ret != null, ex, tableName);
-      }
-      return ret;
-    }
-
-    private static class PartValEqWrapperLite {
-      List<String> values;
-      String location;
-
-      PartValEqWrapperLite(Partition partition) {
-        this.values = partition.isSetValues()? partition.getValues() : null;
-        if (partition.getSd() != null) {
-          this.location = partition.getSd().getLocation();
-        }
-      }
-
-      @Override
-      public int hashCode() {
-        return values == null ? 0 : values.hashCode();
-      }
-
-      @Override
-      public boolean equals(Object obj) {
-        if (this == obj) {
-          return true;
-        }
-        if (obj == null || !(obj instanceof PartValEqWrapperLite)) {
-          return false;
-        }
-
-        List<String> lhsValues = this.values;
-        List<String> rhsValues = ((PartValEqWrapperLite)obj).values;
-
-        if (lhsValues == null || rhsValues == null) {
-          return lhsValues == rhsValues;
-        }
-
-        if (lhsValues.size() != rhsValues.size()) {
-          return false;
-        }
-
-        for (int i=0; i<lhsValues.size(); ++i) {
-          String lhsValue = lhsValues.get(i);
-          String rhsValue = rhsValues.get(i);
-
-          if ((lhsValue == null && rhsValue != null)
-              || (lhsValue != null && !lhsValue.equals(rhsValue))) {
-            return false;
-          }
-        }
-
-        return true;
-      }
-    }
-
-    private List<Partition> add_partitions_core(final RawStore ms, String catName,
-        String dbName, String tblName, List<Partition> parts, final boolean ifNotExists)
-        throws TException {
-      logAndAudit("add_partitions");
-      boolean success = false;
-      // Ensures that the list doesn't have dups, and keeps track of directories we have created.
-      final Map<PartValEqWrapperLite, Boolean> addedPartitions = new ConcurrentHashMap<>();
-      final List<Partition> newParts = new ArrayList<>();
-      final List<Partition> existingParts = new ArrayList<>();
-      Table tbl = null;
-      Map<String, String> transactionalListenerResponses = Collections.emptyMap();
-      Database db = null;
-
-      List<ColumnStatistics> partsColStats = new ArrayList<>(parts.size());
-      List<Long> partsWriteIds = new ArrayList<>(parts.size());
-
-      throwUnsupportedExceptionIfRemoteDB(dbName, "add_partitions");
-
-      try {
-        ms.openTransaction();
-        tbl = ms.getTable(catName, dbName, tblName, null);
-        if (tbl == null) {
-          throw new InvalidObjectException("Unable to add partitions because "
-              + TableName.getQualified(catName, dbName, tblName) +
-              " does not exist");
-        }
-
-        db = ms.getDatabase(catName, dbName);
-
-        if (!parts.isEmpty()) {
-          firePreEvent(new PreAddPartitionEvent(tbl, parts, this));
-        }
-
-        Set<PartValEqWrapperLite> partsToAdd = new HashSet<>(parts.size());
-        List<Partition> partitionsToAdd = new ArrayList<>(parts.size());
-        List<FieldSchema> partitionKeys = tbl.getPartitionKeys();
-        for (final Partition part : parts) {
-          // Collect partition column stats to be updated if present. Partition objects passed down
-          // here at the time of replication may have statistics in them, which is required to be
-          // updated in the metadata. But we don't want it to be part of the Partition object when
-          // it's being created or altered, lest it becomes part of the notification event.
-          if (part.isSetColStats()) {
-            partsColStats.add(part.getColStats());
-            part.unsetColStats();
-            partsWriteIds.add(part.getWriteId());
-          }
-
-          // Iterate through the partitions and validate them. If one of the partitions is
-          // incorrect, an exception will be thrown before the threads which create the partition
-          // folders are submitted. This way we can be sure that no partition and no partition
-          // folder will be created if the list contains an invalid partition.
-          if (validatePartition(part, catName, tblName, dbName, partsToAdd, ms, ifNotExists,
-              partitionKeys)) {
-            partitionsToAdd.add(part);
-          } else {
-            existingParts.add(part);
-          }
-        }
-
-        newParts.addAll(createPartitionFolders(partitionsToAdd, tbl, addedPartitions));
-
-        if (!newParts.isEmpty()) {
-          ms.addPartitions(catName, dbName, tblName, newParts);
-        }
-
-        // Notification is generated for newly created partitions only. The subset of partitions
-        // that already exist (existingParts), will not generate notifications.
-        if (!transactionalListeners.isEmpty()) {
-          transactionalListenerResponses =
-              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                    EventType.ADD_PARTITION,
-                                                    new AddPartitionEvent(tbl, newParts, true, this));
-        }
-
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-                  EventType.ADD_PARTITION,
-                  new AddPartitionEvent(tbl, newParts, true, this),
-                  null,
-                  transactionalListenerResponses, ms);
-
-          if (!existingParts.isEmpty()) {
-            // The request has succeeded but we failed to add these partitions.
-            MetaStoreListenerNotifier.notifyEvent(listeners,
-                    EventType.ADD_PARTITION,
-                    new AddPartitionEvent(tbl, existingParts, false, this),
-                    null, null, ms);
-          }
-        }
-
-        // Update partition column statistics if available. We need a valid writeId list to
-        // update column statistics for a transactional table. But during bootstrap replication,
-        // where we use this feature, we do not have a valid writeId list which was used to
-        // update the stats. But we know for sure that the writeId associated with the stats was
-        // valid then (otherwise stats update would have failed on the source). So, craft a valid
-        // transaction list with only that writeId and use it to update the stats.
-        int cnt = 0;
-        for (ColumnStatistics partColStats: partsColStats) {
-          long writeId = partsWriteIds.get(cnt++);
-          String validWriteIds = null;
-          if (writeId > 0) {
-            ValidWriteIdList validWriteIdList =
-                    new ValidReaderWriteIdList(TableName.getDbTable(tbl.getDbName(),
-                            tbl.getTableName()),
-                            new long[0], new BitSet(), writeId);
-            validWriteIds = validWriteIdList.toString();
-          }
-          updatePartitonColStatsInternal(tbl, partColStats, validWriteIds, writeId);
-        }
-
-        success = ms.commitTransaction();
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-          cleanupPartitionFolders(addedPartitions, db);
-
-          if (!listeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                  EventType.ADD_PARTITION,
-                                                  new AddPartitionEvent(tbl, parts, false, this),
-                                                  null, null, ms);
-          }
-        }
-      }
-
-      return newParts;
-    }
-
-    /**
-     * Remove the newly created partition folders. The values in the addedPartitions map indicates
-     * whether or not the location of the partition was newly created. If the value is false, the
-     * partition folder will not be removed.
-     * @param addedPartitions
-     * @throws MetaException
-     * @throws IllegalArgumentException
-     */
-    private void cleanupPartitionFolders(final Map<PartValEqWrapperLite, Boolean> addedPartitions,
-        Database db) throws MetaException, IllegalArgumentException {
-      for (Map.Entry<PartValEqWrapperLite, Boolean> e : addedPartitions.entrySet()) {
-        if (e.getValue()) {
-          // we just created this directory - it's not a case of pre-creation, so we nuke.
-          wh.deleteDir(new Path(e.getKey().location), true, db);
-        }
-      }
-    }
-
-    /**
-     * Validate a partition before creating it. The validation checks
-     * <ul>
-     * <li>if the database and table names set in the partition are not null and they are matching
-     * with the expected values set in the tblName and dbName parameters.</li>
-     * <li>if the partition values are set.</li>
-     * <li>if none of the partition values is null.</li>
-     * <li>if the partition values are matching with the pattern set in the
-     * 'metastore.partition.name.whitelist.pattern' configuration property.</li>
-     * <li>if the partition doesn't already exist. If the partition already exists, an exception
-     * will be thrown if the ifNotExists parameter is false, otherwise it will be just ignored.</li>
-     * <li>if the partsToAdd set doesn't contain the partition. The partsToAdd set contains the
-     * partitions which are already validated. If the set contains the current partition, it means
-     * that the partition is tried to be added multiple times in the same batch. Please note that
-     * the set will be updated with the current partition if the validation was successful.</li>
-     * </ul>
-     * @param part
-     * @param catName
-     * @param tblName
-     * @param dbName
-     * @param partsToAdd
-     * @param ms
-     * @param ifNotExists
-     * @return
-     * @throws MetaException
-     * @throws TException
-     */
-    private boolean validatePartition(final Partition part, final String catName,
-        final String tblName, final String dbName, final Set<PartValEqWrapperLite> partsToAdd,
-        final RawStore ms, final boolean ifNotExists, List<FieldSchema> partitionKeys) throws MetaException, TException {
-
-      if (part.getDbName() == null || part.getTableName() == null) {
-        throw new MetaException("The database and table name must be set in the partition.");
-      }
-
-      if (!part.getTableName().equalsIgnoreCase(tblName)
-          || !part.getDbName().equalsIgnoreCase(dbName)) {
-        String errorMsg = String.format(
-            "Partition does not belong to target table %s. It belongs to the table %s.%s : %s",
-            TableName.getQualified(catName, dbName, tblName), part.getDbName(),
-            part.getTableName(), part.toString());
-        throw new MetaException(errorMsg);
-      }
-
-      if (part.getValues() == null || part.getValues().isEmpty()) {
-        throw new MetaException("The partition values cannot be null or empty.");
-      }
-
-      if (part.getValues().contains(null)) {
-        throw new MetaException("Partition value cannot be null.");
-      }
-
-      boolean shouldAdd = startAddPartition(ms, part, partitionKeys, ifNotExists);
-      if (!shouldAdd) {
-        LOG.info("Not adding partition {} as it already exists", part);
-        return false;
-      }
-
-      if (!partsToAdd.add(new PartValEqWrapperLite(part))) {
-        // Technically, for ifNotExists case, we could insert one and discard the other
-        // because the first one now "exists", but it seems better to report the problem
-        // upstream as such a command doesn't make sense.
-        throw new MetaException("Duplicate partitions in the list: " + part);
-      }
-      return true;
-    }
-
-    /**
-     * Create the location folders for the partitions. For each partition a separate thread will be
-     * started to create the folder. The method will wait until all threads are finished and returns
-     * the partitions whose folders were created successfully. If an error occurs during the
-     * execution of a thread, a MetaException will be thrown.
-     * @param partitionsToAdd
-     * @param table
-     * @param addedPartitions
-     * @return
-     * @throws MetaException
-     */
-    private List<Partition> createPartitionFolders(final List<Partition> partitionsToAdd,
-        final Table table, final Map<PartValEqWrapperLite, Boolean> addedPartitions)
-        throws MetaException {
-
-      final AtomicBoolean failureOccurred = new AtomicBoolean(false);
-      final List<Future<Partition>> partFutures = new ArrayList<>(partitionsToAdd.size());
-      final Map<PartValEqWrapperLite, Boolean> addedParts = new ConcurrentHashMap<>();
-
-      final UserGroupInformation ugi;
-      try {
-        ugi = UserGroupInformation.getCurrentUser();
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-
-      for (final Partition partition : partitionsToAdd) {
-        initializePartitionParameters(table, partition);
-
-        partFutures.add(threadPool.submit(() -> {
-          if (failureOccurred.get()) {
-            return null;
-          }
-          ugi.doAs((PrivilegedExceptionAction<Partition>) () -> {
-            try {
-              boolean madeDir = createLocationForAddedPartition(table, partition);
-              addedParts.put(new PartValEqWrapperLite(partition), madeDir);
-              initializeAddedPartition(table, partition, madeDir, null);
-            } catch (MetaException e) {
-              throw new IOException(e.getMessage(), e);
-            }
-            return null;
-          });
-          return partition;
-        }));
-      }
-
-      List<Partition> newParts = new ArrayList<>(partitionsToAdd.size());
-      String errorMessage = null;
-      for (Future<Partition> partFuture : partFutures) {
-        try {
-          Partition part = partFuture.get();
-          if (part != null && !failureOccurred.get()) {
-            newParts.add(part);
-          }
-        } catch (ExecutionException e) {
-          // If an exception is thrown in the execution of a task, set the failureOccurred flag to
-          // true. This flag is visible in the tasks and if its value is true, the partition
-          // folders won't be created.
-          // Then iterate through the remaining tasks and wait for them to finish. The tasks which
-          // are started before the flag got set will then finish creating the partition folders.
-          // The tasks which are started after the flag got set, won't create the partition
-          // folders, to avoid unnecessary work.
-          // This way it is sure that all tasks are finished, when entering the finally part where
-          // the partition folders are cleaned up. It won't happen that a task is still running
-          // when cleaning up the folders, so it is sure we won't have leftover folders.
-          // Canceling the other tasks would be also an option but during testing it turned out
-          // that it is not a trustworthy solution to avoid leftover folders.
-          failureOccurred.compareAndSet(false, true);
-          errorMessage = e.getMessage();
-        } catch (InterruptedException e) {
-          failureOccurred.compareAndSet(false, true);
-          errorMessage = e.getMessage();
-          // Restore interruption status of the corresponding thread
-          Thread.currentThread().interrupt();
-        }
-      }
-
-      addedPartitions.putAll(addedParts);
-      if (failureOccurred.get()) {
-        throw new MetaException(errorMessage);
-      }
-
-      return newParts;
-    }
-
-    @Override
-    public AddPartitionsResult add_partitions_req(AddPartitionsRequest request)
-        throws TException {
-      AddPartitionsResult result = new AddPartitionsResult();
-      if (request.getParts().isEmpty()) {
-        return result;
-      }
-      try {
-        if (!request.isSetCatName()) {
-          request.setCatName(getDefaultCatalog(conf));
-        }
-        // Make sure all of the partitions have the catalog set as well
-        request.getParts().forEach(p -> {
-          if (!p.isSetCatName()) {
-            p.setCatName(getDefaultCatalog(conf));
-          }
-        });
-        List<Partition> parts = add_partitions_core(getMS(), request.getCatName(), request.getDbName(),
-            request.getTblName(), request.getParts(), request.isIfNotExists());
-        if (request.isNeedResult()) {
-          result.setPartitions(parts);
-        }
-      } catch (TException te) {
-        throw te;
-      } catch (Exception e) {
-        throw newMetaException(e);
-      }
-      return result;
-    }
-
-    @Override
-    public int add_partitions(final List<Partition> parts) throws MetaException,
-        InvalidObjectException, AlreadyExistsException {
-      startFunction("add_partition");
-      if (parts == null) {
-        throw new MetaException("Partition list cannot be null.");
-      }
-      if (parts.isEmpty()) {
-        return 0;
-      }
-
-      Integer ret = null;
-      Exception ex = null;
-      try {
-        // Old API assumed all partitions belong to the same table; keep the same assumption
-        if (!parts.get(0).isSetCatName()) {
-          String defaultCat = getDefaultCatalog(conf);
-          for (Partition p : parts) {
-            p.setCatName(defaultCat);
-          }
-        }
-        ret = add_partitions_core(getMS(), parts.get(0).getCatName(), parts.get(0).getDbName(),
-            parts.get(0).getTableName(), parts, false).size();
-        assert ret == parts.size();
-      } catch (MetaException | InvalidObjectException | AlreadyExistsException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        String tableName = parts.get(0).getTableName();
-        endFunction("add_partition", ret != null, ex, tableName);
-      }
-      return ret;
-    }
-
-    @Override
-    public int add_partitions_pspec(final List<PartitionSpec> partSpecs)
-        throws TException {
-      logAndAudit("add_partitions_pspec");
-
-      if (partSpecs.isEmpty()) {
-        return 0;
-      }
-
-      String dbName = partSpecs.get(0).getDbName();
-      String tableName = partSpecs.get(0).getTableName();
-      // If the catalog name isn't set, we need to go through and set it.
-      String catName;
-      if (!partSpecs.get(0).isSetCatName()) {
-        catName = getDefaultCatalog(conf);
-        partSpecs.forEach(ps -> ps.setCatName(catName));
-      } else {
-        catName = partSpecs.get(0).getCatName();
-      }
-
-      return add_partitions_pspec_core(getMS(), catName, dbName, tableName, partSpecs, false);
-    }
-
-    private int add_partitions_pspec_core(RawStore ms, String catName, String dbName,
-                                          String tblName, List<PartitionSpec> partSpecs,
-                                          boolean ifNotExists)
-        throws TException {
-      boolean success = false;
-      if (dbName == null || tblName == null) {
-        throw new MetaException("The database and table name cannot be null.");
-      }
-      // Ensures that the list doesn't have dups, and keeps track of directories we have created.
-      final Map<PartValEqWrapperLite, Boolean> addedPartitions = new ConcurrentHashMap<>();
-      PartitionSpecProxy partitionSpecProxy = PartitionSpecProxy.Factory.get(partSpecs);
-      final PartitionSpecProxy.PartitionIterator partitionIterator = partitionSpecProxy
-          .getPartitionIterator();
-      Table tbl = null;
-      Map<String, String> transactionalListenerResponses = Collections.emptyMap();
-      Database db = null;
-      try {
-        ms.openTransaction();
-        try {
-          db = ms.getDatabase(catName, dbName);
-        } catch (NoSuchObjectException notExists) {
-          throw new InvalidObjectException("Unable to add partitions because "
-              + "database or table " + dbName + "." + tblName + " does not exist");
-        }
-        if (db.getType() == DatabaseType.REMOTE)
-          throw new MetaException("Operation add_partitions_pspec not supported on tables in REMOTE database");
-        tbl = ms.getTable(catName, dbName, tblName, null);
-        if (tbl == null) {
-          throw new InvalidObjectException("Unable to add partitions because "
-              + "database or table " + dbName + "." + tblName + " does not exist");
-        }
-        firePreEvent(new PreAddPartitionEvent(tbl, partitionSpecProxy, this));
-        Set<PartValEqWrapperLite> partsToAdd = new HashSet<>(partitionSpecProxy.size());
-        List<Partition> partitionsToAdd = new ArrayList<>(partitionSpecProxy.size());
-        List<FieldSchema> partitionKeys = tbl.getPartitionKeys();
-        while (partitionIterator.hasNext()) {
-          // Iterate through the partitions and validate them. If one of the partitions is
-          // incorrect, an exception will be thrown before the threads which create the partition
-          // folders are submitted. This way we can be sure that no partition or partition folder
-          // will be created if the list contains an invalid partition.
-          final Partition part = partitionIterator.getCurrent();
-          if (validatePartition(part, catName, tblName, dbName, partsToAdd, ms, ifNotExists,
-              partitionKeys)) {
-            partitionsToAdd.add(part);
-          }
-          partitionIterator.next();
-        }
-
-        createPartitionFolders(partitionsToAdd, tbl, addedPartitions);
-
-        ms.addPartitions(catName, dbName, tblName, partitionSpecProxy, ifNotExists);
-
-        if (!transactionalListeners.isEmpty()) {
-          transactionalListenerResponses =
-              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                    EventType.ADD_PARTITION,
-                                                    new AddPartitionEvent(tbl, partitionSpecProxy, true, this));
-        }
-
-        success = ms.commitTransaction();
-        return addedPartitions.size();
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-          cleanupPartitionFolders(addedPartitions, db);
-        }
-
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                EventType.ADD_PARTITION,
-                                                new AddPartitionEvent(tbl, partitionSpecProxy, true, this),
-                                                null,
-                                                transactionalListenerResponses, ms);
-        }
-      }
-    }
-
-    private boolean startAddPartition(
-        RawStore ms, Partition part, List<FieldSchema> partitionKeys, boolean ifNotExists)
-        throws TException {
-      MetaStoreServerUtils.validatePartitionNameCharacters(part.getValues(),
-          partitionValidationPattern);
-      boolean doesExist = ms.doesPartitionExist(part.getCatName(),
-          part.getDbName(), part.getTableName(), partitionKeys, part.getValues());
-      if (doesExist && !ifNotExists) {
-        throw new AlreadyExistsException("Partition already exists: " + part);
-      }
-      return !doesExist;
-    }
-
-    /**
-     * Handles the location for a partition being created.
-     * @param tbl Table.
-     * @param part Partition.
-     * @return Whether the partition SD location is set to a newly created directory.
-     */
-    private boolean createLocationForAddedPartition(
-        final Table tbl, final Partition part) throws MetaException {
-      Path partLocation = null;
-      String partLocationStr = null;
-      if (part.getSd() != null) {
-        partLocationStr = part.getSd().getLocation();
-      }
-
-      if (partLocationStr == null || partLocationStr.isEmpty()) {
-        // set default location if not specified and this is
-        // a physical table partition (not a view)
-        if (tbl.getSd().getLocation() != null) {
-          partLocation = new Path(tbl.getSd().getLocation(), Warehouse
-              .makePartName(tbl.getPartitionKeys(), part.getValues()));
-        }
-      } else {
-        if (tbl.getSd().getLocation() == null) {
-          throw new MetaException("Cannot specify location for a view partition");
-        }
-        partLocation = wh.getDnsPath(new Path(partLocationStr));
-      }
-
-      boolean result = false;
-      if (partLocation != null) {
-        part.getSd().setLocation(partLocation.toString());
-
-        // Check to see if the directory already exists before calling
-        // mkdirs() because if the file system is read-only, mkdirs will
-        // throw an exception even if the directory already exists.
-        if (!wh.isDir(partLocation)) {
-          if (!wh.mkdirs(partLocation)) {
-            throw new MetaException(partLocation
-                + " is not a directory or unable to create one");
-          }
-          result = true;
-        }
-      }
-      return result;
-    }
-
-    /**
-     * Verify if update stats while altering partition(s)
-     * For the following three cases HMS will not update partition stats
-     * 1) Table property 'DO_NOT_UPDATE_STATS' = True
-     * 2) HMS configuration property 'STATS_AUTO_GATHER' = False
-     * 3) Is View
-     */
-    private boolean canUpdateStats(Table tbl) {
-        Map<String,String> tblParams = tbl.getParameters();
-        boolean updateStatsTbl = true;
-        if ((tblParams != null) && tblParams.containsKey(StatsSetupConst.DO_NOT_UPDATE_STATS)) {
-            updateStatsTbl = !Boolean.valueOf(tblParams.get(StatsSetupConst.DO_NOT_UPDATE_STATS));
-        }
-        if (!MetastoreConf.getBoolVar(conf, ConfVars.STATS_AUTO_GATHER) ||
-            MetaStoreUtils.isView(tbl) ||
-            !updateStatsTbl) {
-          return false;
-        }
-        return true;
-    }
-
-    private void initializeAddedPartition(final Table tbl, final Partition part, boolean madeDir,
-                                          EnvironmentContext environmentContext) throws MetaException {
-      initializeAddedPartition(tbl,
-              new PartitionSpecProxy.SimplePartitionWrapperIterator(part), madeDir, environmentContext);
-    }
-
-    private void initializeAddedPartition(
-        final Table tbl, final PartitionSpecProxy.PartitionIterator part, boolean madeDir,
-        EnvironmentContext environmentContext) throws MetaException {
-      if (canUpdateStats(tbl)) {
-        MetaStoreServerUtils.updatePartitionStatsFast(part, tbl, wh, madeDir,
-                false, environmentContext, true);
-      }
-
-      // set create time
-      long time = System.currentTimeMillis() / 1000;
-      part.setCreateTime((int) time);
-      if (part.getParameters() == null ||
-          part.getParameters().get(hive_metastoreConstants.DDL_TIME) == null) {
-        part.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(time));
-      }
-    }
-
-    private void initializePartitionParameters(final Table tbl, final Partition part)
-        throws MetaException {
-      initializePartitionParameters(tbl,
-          new PartitionSpecProxy.SimplePartitionWrapperIterator(part));
-    }
-
-    private void initializePartitionParameters(final Table tbl,
-        final PartitionSpecProxy.PartitionIterator part) throws MetaException {
-
-      // Inherit table properties into partition properties.
-      Map<String, String> tblParams = tbl.getParameters();
-      String inheritProps = MetastoreConf.getVar(conf, ConfVars.PART_INHERIT_TBL_PROPS).trim();
-      // Default value is empty string in which case no properties will be inherited.
-      // * implies all properties needs to be inherited
-      Set<String> inheritKeys = new HashSet<>(Arrays.asList(inheritProps.split(",")));
-      if (inheritKeys.contains("*")) {
-        inheritKeys = tblParams.keySet();
-      }
-
-      for (String key : inheritKeys) {
-        String paramVal = tblParams.get(key);
-        if (null != paramVal) { // add the property only if it exists in table properties
-          part.putToParameters(key, paramVal);
-        }
-      }
-    }
-
-    private Partition add_partition_core(final RawStore ms,
-        final Partition part, final EnvironmentContext envContext)
-        throws TException {
-      boolean success = false;
-      Table tbl = null;
-      Map<String, String> transactionalListenerResponses = Collections.emptyMap();
-      if (!part.isSetCatName()) {
-        part.setCatName(getDefaultCatalog(conf));
-      }
-
-      throwUnsupportedExceptionIfRemoteDB(part.getDbName(), "add_partition");
-      try {
-        ms.openTransaction();
-        tbl = ms.getTable(part.getCatName(), part.getDbName(), part.getTableName(), null);
-        if (tbl == null) {
-          throw new InvalidObjectException(
-              "Unable to add partition because table or database do not exist");
-        }
-
-        firePreEvent(new PreAddPartitionEvent(tbl, part, this));
-
-        if (part.getValues() == null || part.getValues().isEmpty()) {
-          throw new MetaException("The partition values cannot be null or empty.");
-        }
-        boolean shouldAdd = startAddPartition(ms, part, tbl.getPartitionKeys(), false);
-        assert shouldAdd; // start would throw if it already existed here
-        boolean madeDir = createLocationForAddedPartition(tbl, part);
-        try {
-          initializeAddedPartition(tbl, part, madeDir, envContext);
-          initializePartitionParameters(tbl, part);
-          success = ms.addPartition(part);
-        } finally {
-          if (!success && madeDir) {
-            wh.deleteDir(new Path(part.getSd().getLocation()), true, false,
-                    ReplChangeManager.shouldEnableCm(ms.getDatabase(part.getCatName(), part.getDbName()), tbl));
-          }
-        }
-
-        // Setting success to false to make sure that if the listener fails, rollback happens.
-        success = false;
-
-        if (!transactionalListeners.isEmpty()) {
-          transactionalListenerResponses =
-              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                    EventType.ADD_PARTITION,
-                                                    new AddPartitionEvent(tbl, Arrays.asList(part), true, this),
-                                                    envContext);
-
-        }
-
-        // we proceed only if we'd actually succeeded anyway, otherwise,
-        // we'd have thrown an exception
-        success = ms.commitTransaction();
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                EventType.ADD_PARTITION,
-                                                new AddPartitionEvent(tbl, Arrays.asList(part), success, this),
-                                                envContext,
-                                                transactionalListenerResponses, ms);
-
-        }
-      }
-      return part;
-    }
-
-    @Override
-    public Partition add_partition(final Partition part)
-        throws InvalidObjectException, AlreadyExistsException, MetaException {
-      return add_partition_with_environment_context(part, null);
-    }
-
-    @Override
-    public Partition add_partition_with_environment_context(
-        final Partition part, EnvironmentContext envContext)
-        throws InvalidObjectException, AlreadyExistsException,
-        MetaException {
-      if (part == null) {
-        throw new MetaException("Partition cannot be null.");
-      }
-      startTableFunction("add_partition",
-          part.getCatName(), part.getDbName(), part.getTableName());
-      Partition ret = null;
-      Exception ex = null;
-      try {
-        ret = add_partition_core(getMS(), part, envContext);
-      } catch (MetaException | InvalidObjectException | AlreadyExistsException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("add_partition", ret != null, ex, part != null ?  part.getTableName(): null);
-      }
-      return ret;
-    }
-
-    @Override
-    public Partition exchange_partition(Map<String, String> partitionSpecs,
-        String sourceDbName, String sourceTableName, String destDbName,
-        String destTableName) throws TException {
-      exchange_partitions(partitionSpecs, sourceDbName, sourceTableName, destDbName, destTableName);
-      // Wouldn't it make more sense to return the first element of the list returned by the
-      // previous call?
-      return new Partition();
-    }
-
-    @Override
-    public List<Partition> exchange_partitions(Map<String, String> partitionSpecs,
-        String sourceDbName, String sourceTableName, String destDbName,
-        String destTableName) throws TException {
-      String[] parsedDestDbName = parseDbName(destDbName, conf);
-      String[] parsedSourceDbName = parseDbName(sourceDbName, conf);
-      // No need to check catalog for null as parseDbName() will never return null for the catalog.
-      if (partitionSpecs == null || parsedSourceDbName[DB_NAME] == null || sourceTableName == null
-          || parsedDestDbName[DB_NAME] == null || destTableName == null) {
-        throw new MetaException("The DB and table name for the source and destination tables,"
-            + " and the partition specs must not be null.");
-      }
-      if (!parsedDestDbName[CAT_NAME].equals(parsedSourceDbName[CAT_NAME])) {
-        throw new MetaException("You cannot move a partition across catalogs");
-      }
-
-      boolean success = false;
-      boolean pathCreated = false;
-      RawStore ms = getMS();
-      ms.openTransaction();
-
-      Table destinationTable =
-          ms.getTable(
-              parsedDestDbName[CAT_NAME], parsedDestDbName[DB_NAME], destTableName, null);
-      if (destinationTable == null) {
-        throw new MetaException( "The destination table " +
-            TableName.getQualified(parsedDestDbName[CAT_NAME],
-                parsedDestDbName[DB_NAME], destTableName) + " not found");
-      }
-      Table sourceTable =
-          ms.getTable(
-              parsedSourceDbName[CAT_NAME], parsedSourceDbName[DB_NAME], sourceTableName, null);
-      if (sourceTable == null) {
-        throw new MetaException("The source table " +
-            TableName.getQualified(parsedSourceDbName[CAT_NAME],
-                parsedSourceDbName[DB_NAME], sourceTableName) + " not found");
-      }
-
-      List<String> partVals = MetaStoreUtils.getPvals(sourceTable.getPartitionKeys(),
-          partitionSpecs);
-      List<String> partValsPresent = new ArrayList<> ();
-      List<FieldSchema> partitionKeysPresent = new ArrayList<> ();
-      int i = 0;
-      for (FieldSchema fs: sourceTable.getPartitionKeys()) {
-        String partVal = partVals.get(i);
-        if (partVal != null && !partVal.equals("")) {
-          partValsPresent.add(partVal);
-          partitionKeysPresent.add(fs);
-        }
-        i++;
-      }
-      // Passed the unparsed DB name here, as get_partitions_ps expects to parse it
-      List<Partition> partitionsToExchange = get_partitions_ps(sourceDbName, sourceTableName,
-          partVals, (short)-1);
-      if (partitionsToExchange == null || partitionsToExchange.isEmpty()) {
-        throw new MetaException("No partition is found with the values " + partitionSpecs
-            + " for the table " + sourceTableName);
-      }
-      boolean sameColumns = MetaStoreUtils.compareFieldColumns(
-          sourceTable.getSd().getCols(), destinationTable.getSd().getCols());
-      boolean samePartitions = MetaStoreUtils.compareFieldColumns(
-          sourceTable.getPartitionKeys(), destinationTable.getPartitionKeys());
-      if (!sameColumns || !samePartitions) {
-        throw new MetaException("The tables have different schemas." +
-            " Their partitions cannot be exchanged.");
-      }
-      Path sourcePath = new Path(sourceTable.getSd().getLocation(),
-          Warehouse.makePartName(partitionKeysPresent, partValsPresent));
-      Path destPath = new Path(destinationTable.getSd().getLocation(),
-          Warehouse.makePartName(partitionKeysPresent, partValsPresent));
-      List<Partition> destPartitions = new ArrayList<>();
-
-      Map<String, String> transactionalListenerResponsesForAddPartition = Collections.emptyMap();
-      List<Map<String, String>> transactionalListenerResponsesForDropPartition =
-          Lists.newArrayListWithCapacity(partitionsToExchange.size());
-
-      // Check if any of the partitions already exists in destTable.
-      List<String> destPartitionNames = ms.listPartitionNames(parsedDestDbName[CAT_NAME],
-          parsedDestDbName[DB_NAME], destTableName, (short) -1);
-      if (destPartitionNames != null && !destPartitionNames.isEmpty()) {
-        for (Partition partition : partitionsToExchange) {
-          String partToExchangeName =
-              Warehouse.makePartName(destinationTable.getPartitionKeys(), partition.getValues());
-          if (destPartitionNames.contains(partToExchangeName)) {
-            throw new MetaException("The partition " + partToExchangeName
-                + " already exists in the table " + destTableName);
-          }
-        }
-      }
-
-      Database srcDb = ms.getDatabase(parsedSourceDbName[CAT_NAME], parsedSourceDbName[DB_NAME]);
-      Database destDb = ms.getDatabase(parsedDestDbName[CAT_NAME], parsedDestDbName[DB_NAME]);
-      if (!isRenameAllowed(srcDb, destDb)) {
-        throw new MetaException("Exchange partition not allowed for " +
-                TableName.getQualified(parsedSourceDbName[CAT_NAME],
-                parsedSourceDbName[DB_NAME], sourceTableName) + " Dest db : " + destDbName);
-      }
-      try {
-        for (Partition partition: partitionsToExchange) {
-          Partition destPartition = new Partition(partition);
-          destPartition.setDbName(parsedDestDbName[DB_NAME]);
-          destPartition.setTableName(destinationTable.getTableName());
-          Path destPartitionPath = new Path(destinationTable.getSd().getLocation(),
-              Warehouse.makePartName(destinationTable.getPartitionKeys(), partition.getValues()));
-          destPartition.getSd().setLocation(destPartitionPath.toString());
-          ms.addPartition(destPartition);
-          destPartitions.add(destPartition);
-          ms.dropPartition(parsedSourceDbName[CAT_NAME], partition.getDbName(), sourceTable.getTableName(),
-            partition.getValues());
-        }
-        Path destParentPath = destPath.getParent();
-        if (!wh.isDir(destParentPath)) {
-          if (!wh.mkdirs(destParentPath)) {
-              throw new MetaException("Unable to create path " + destParentPath);
-          }
-        }
-        /*
-         * TODO: Use the hard link feature of hdfs
-         * once https://issues.apache.org/jira/browse/HDFS-3370 is done
-         */
-        pathCreated = wh.renameDir(sourcePath, destPath, false);
-
-        // Setting success to false to make sure that if the listener fails, rollback happens.
-        success = false;
-
-        if (!transactionalListeners.isEmpty()) {
-          transactionalListenerResponsesForAddPartition =
-              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                    EventType.ADD_PARTITION,
-                                                    new AddPartitionEvent(destinationTable, destPartitions, true, this));
-
-          for (Partition partition : partitionsToExchange) {
-            DropPartitionEvent dropPartitionEvent =
-                new DropPartitionEvent(sourceTable, partition, true, true, this);
-            transactionalListenerResponsesForDropPartition.add(
-                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                      EventType.DROP_PARTITION,
-                                                      dropPartitionEvent));
-          }
-        }
-
-        success = ms.commitTransaction();
-        return destPartitions;
-      } finally {
-        if (!success || !pathCreated) {
-          ms.rollbackTransaction();
-          if (pathCreated) {
-            wh.renameDir(destPath, sourcePath, false);
-          }
-        }
-
-        if (!listeners.isEmpty()) {
-          AddPartitionEvent addPartitionEvent = new AddPartitionEvent(destinationTable, destPartitions, success, this);
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                EventType.ADD_PARTITION,
-                                                addPartitionEvent,
-                                                null,
-                                                transactionalListenerResponsesForAddPartition, ms);
-
-          i = 0;
-          for (Partition partition : partitionsToExchange) {
-            DropPartitionEvent dropPartitionEvent =
-                new DropPartitionEvent(sourceTable, partition, success, true, this);
-            Map<String, String> parameters =
-                (transactionalListenerResponsesForDropPartition.size() > i)
-                    ? transactionalListenerResponsesForDropPartition.get(i)
-                    : null;
-
-            MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                  EventType.DROP_PARTITION,
-                                                  dropPartitionEvent,
-                                                  null,
-                                                  parameters, ms);
-            i++;
-          }
-        }
-      }
-    }
-
-    private boolean drop_partition_common(RawStore ms, String catName, String db_name,
-                                          String tbl_name, List<String> part_vals,
-                                          final boolean deleteData, final EnvironmentContext envContext)
-        throws MetaException, NoSuchObjectException, IOException, InvalidObjectException,
-      InvalidInputException {
-      boolean success = false;
-      Path partPath = null;
-      Table tbl = null;
-      Partition part = null;
-      boolean isArchived = false;
-      Path archiveParentDir = null;
-      boolean mustPurge = false;
-      boolean tableDataShouldBeDeleted = false;
-      boolean needsCm = false;
-      Map<String, String> transactionalListenerResponses = Collections.emptyMap();
-
-      if (db_name == null) {
-        throw new MetaException("The DB name cannot be null.");
-      }
-      if (tbl_name == null) {
-        throw new MetaException("The table name cannot be null.");
-      }
-      if (part_vals == null) {
-        throw new MetaException("The partition values cannot be null.");
-      }
-
-      try {
-        ms.openTransaction();
-        part = ms.getPartition(catName, db_name, tbl_name, part_vals);
-        tbl = get_table_core(catName, db_name, tbl_name, null);
-        tableDataShouldBeDeleted = checkTableDataShouldBeDeleted(tbl, deleteData);
-        firePreEvent(new PreDropPartitionEvent(tbl, part, deleteData, this));
-        mustPurge = isMustPurge(envContext, tbl);
-
-        if (part == null) {
-          throw new NoSuchObjectException("Partition doesn't exist. "
-              + part_vals);
-        }
-
-        isArchived = MetaStoreUtils.isArchived(part);
-        if (isArchived) {
-          archiveParentDir = MetaStoreUtils.getOriginalLocation(part);
-          verifyIsWritablePath(archiveParentDir);
-        }
-
-        if ((part.getSd() != null) && (part.getSd().getLocation() != null)) {
-          partPath = new Path(part.getSd().getLocation());
-          verifyIsWritablePath(partPath);
-        }
-
-        if (!ms.dropPartition(catName, db_name, tbl_name, part_vals)) {
-          throw new MetaException("Unable to drop partition");
-        } else {
-          if (!transactionalListeners.isEmpty()) {
-
-            transactionalListenerResponses =
-                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                      EventType.DROP_PARTITION,
-                                                      new DropPartitionEvent(tbl, part, true, deleteData, this),
-                                                      envContext);
-          }
-          needsCm = ReplChangeManager.shouldEnableCm(ms.getDatabase(catName, db_name), tbl);
-          success = ms.commitTransaction();
-        }
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        } else if (deleteData && ((partPath != null) || (archiveParentDir != null))) {
-          if (tableDataShouldBeDeleted) {
-            if (mustPurge) {
-              LOG.info("dropPartition() will purge " + partPath + " directly, skipping trash.");
-            }
-            else {
-              LOG.info("dropPartition() will move " + partPath + " to trash-directory.");
-            }
-            // Archived partitions have har:/to_har_file as their location.
-            // The original directory was saved in params
-
-            if (isArchived) {
-              assert (archiveParentDir != null);
-              wh.deleteDir(archiveParentDir, true, mustPurge, needsCm);
-            } else {
-              assert (partPath != null);
-              wh.deleteDir(partPath, true, mustPurge, needsCm);
-              deleteParentRecursive(partPath.getParent(), part_vals.size() - 1, mustPurge, needsCm);
-            }
-            // ok even if the data is not deleted
-          }
-        }
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                EventType.DROP_PARTITION,
-                                                new DropPartitionEvent(tbl, part, success, deleteData, this),
-                                                envContext,
-                                                transactionalListenerResponses, ms);
-        }
-      }
-      return true;
-    }
-
-    private static boolean isMustPurge(EnvironmentContext envContext, Table tbl) {
-      // Data needs deletion. Check if trash may be skipped.
-      // Trash may be skipped iff:
-      //  1. deleteData == true, obviously.
-      //  2. tbl is external.
-      //  3. Either
-      //    3.1. User has specified PURGE from the commandline, and if not,
-      //    3.2. User has set the table to auto-purge.
-      return ((envContext != null) && Boolean.parseBoolean(envContext.getProperties().get("ifPurge")))
-          || MetaStoreUtils.isSkipTrash(tbl.getParameters());
-    }
-
-    private void throwUnsupportedExceptionIfRemoteDB(String dbName, String operationName) throws MetaException {
-      if (isDatabaseRemote(dbName)) {
-        throw new MetaException("Operation " + operationName + " not supported for REMOTE database " + dbName);
-      }
-    }
-
-    private boolean isDatabaseRemote(String name) {
-      try {
-        String[] parsedDbName = parseDbName(name, conf);
-        Database db = get_database_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME]);
-        if (db != null && db.getType() == DatabaseType.REMOTE) {
-          return true;
-        }
-      } catch (Exception e) {
-        return false;
-      }
-      return false;
-    }
-
-    private void deleteParentRecursive(Path parent, int depth, boolean mustPurge, boolean needRecycle)
-            throws IOException, MetaException {
-      if (depth > 0 && parent != null && wh.isWritable(parent)) {
-        if (wh.isDir(parent) && wh.isEmptyDir(parent)) {
-          wh.deleteDir(parent, true, mustPurge, needRecycle);
-        }
-        deleteParentRecursive(parent.getParent(), depth - 1, mustPurge, needRecycle);
-      }
-    }
-
-    @Override
-    public boolean drop_partition(final String db_name, final String tbl_name,
-        final List<String> part_vals, final boolean deleteData)
-        throws TException {
-      return drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData,
-          null);
-    }
-
-    private static class PathAndPartValSize {
-      PathAndPartValSize(Path path, int partValSize) {
-        this.path = path;
-        this.partValSize = partValSize;
-      }
-      public Path path;
-      int partValSize;
-    }
-
-    @Override
-    public DropPartitionsResult drop_partitions_req(
-        DropPartitionsRequest request) throws TException {
-      RawStore ms = getMS();
-      String dbName = request.getDbName(), tblName = request.getTblName();
-      String catName = request.isSetCatName() ? request.getCatName() : getDefaultCatalog(conf);
-      boolean ifExists = request.isSetIfExists() && request.isIfExists();
-      boolean deleteData = request.isSetDeleteData() && request.isDeleteData();
-      boolean ignoreProtection = request.isSetIgnoreProtection() && request.isIgnoreProtection();
-      boolean needResult = !request.isSetNeedResult() || request.isNeedResult();
-      List<PathAndPartValSize> dirsToDelete = new ArrayList<>();
-      List<Path> archToDelete = new ArrayList<>();
-      EnvironmentContext envContext = request.isSetEnvironmentContext()
-          ? request.getEnvironmentContext() : null;
-
-      boolean success = false;
-      ms.openTransaction();
-      Table tbl = null;
-      List<Partition> parts = null;
-      boolean mustPurge = false;
-      List<Map<String, String>> transactionalListenerResponses = Lists.newArrayList();
-      boolean needsCm = ReplChangeManager.shouldEnableCm(ms.getDatabase(catName, dbName),
-              ms.getTable(catName, dbName, tblName));
-
-      try {
-        // We need Partition-s for firing events and for result; DN needs MPartition-s to drop.
-        // Great... Maybe we could bypass fetching MPartitions by issuing direct SQL deletes.
-        tbl = get_table_core(catName, dbName, tblName);
-        isExternal(tbl);
-        mustPurge = isMustPurge(envContext, tbl);
-        int minCount = 0;
-        RequestPartsSpec spec = request.getParts();
-        List<String> partNames = null;
-        if (spec.isSetExprs()) {
-          // Dropping by expressions.
-          parts = new ArrayList<>(spec.getExprs().size());
-          for (DropPartitionsExpr expr : spec.getExprs()) {
-            ++minCount; // At least one partition per expression, if not ifExists
-            List<Partition> result = new ArrayList<>();
-            boolean hasUnknown = ms.getPartitionsByExpr(
-                catName, dbName, tblName, expr.getExpr(), null, (short)-1, result);
-            if (hasUnknown) {
-              // Expr is built by DDLSA, it should only contain part cols and simple ops
-              throw new MetaException("Unexpected unknown partitions to drop");
-            }
-            // this is to prevent dropping archived partition which is archived in a
-            // different level the drop command specified.
-            if (!ignoreProtection && expr.isSetPartArchiveLevel()) {
-              for (Partition part : parts) {
-                if (MetaStoreUtils.isArchived(part)
-                    && MetaStoreUtils.getArchivingLevel(part) < expr.getPartArchiveLevel()) {
-                  throw new MetaException("Cannot drop a subset of partitions "
-                      + " in an archive, partition " + part);
-                }
-              }
-            }
-            parts.addAll(result);
-          }
-        } else if (spec.isSetNames()) {
-          partNames = spec.getNames();
-          minCount = partNames.size();
-          parts = ms.getPartitionsByNames(catName, dbName, tblName, partNames);
-        } else {
-          throw new MetaException("Partition spec is not set");
-        }
-
-        if ((parts.size() < minCount) && !ifExists) {
-          throw new NoSuchObjectException("Some partitions to drop are missing");
-        }
-
-        List<String> colNames = null;
-        if (partNames == null) {
-          partNames = new ArrayList<>(parts.size());
-          colNames = new ArrayList<>(tbl.getPartitionKeys().size());
-          for (FieldSchema col : tbl.getPartitionKeys()) {
-            colNames.add(col.getName());
-          }
-        }
-
-        for (Partition part : parts) {
-
-          // TODO - we need to speed this up for the normal path where all partitions are under
-          // the table and we don't have to stat every partition
-
-          firePreEvent(new PreDropPartitionEvent(tbl, part, deleteData, this));
-          if (colNames != null) {
-            partNames.add(FileUtils.makePartName(colNames, part.getValues()));
-          }
-          // Preserve the old behavior of failing when we cannot write, even w/o deleteData,
-          // and even if the table is external. That might not make any sense.
-          if (MetaStoreUtils.isArchived(part)) {
-            Path archiveParentDir = MetaStoreUtils.getOriginalLocation(part);
-            verifyIsWritablePath(archiveParentDir);
-            archToDelete.add(archiveParentDir);
-          }
-          if ((part.getSd() != null) && (part.getSd().getLocation() != null)) {
-            Path partPath = new Path(part.getSd().getLocation());
-            verifyIsWritablePath(partPath);
-            dirsToDelete.add(new PathAndPartValSize(partPath, part.getValues().size()));
-          }
-        }
-
-        ms.dropPartitions(catName, dbName, tblName, partNames);
-        if (parts != null && !transactionalListeners.isEmpty()) {
-          for (Partition part : parts) {
-            transactionalListenerResponses.add(
-                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                      EventType.DROP_PARTITION,
-                                                      new DropPartitionEvent(tbl, part, true, deleteData, this),
-                                                      envContext));
-          }
-        }
-
-        success = ms.commitTransaction();
-        DropPartitionsResult result = new DropPartitionsResult();
-        if (needResult) {
-          result.setPartitions(parts);
-        }
-
-        return result;
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        } else if (checkTableDataShouldBeDeleted(tbl, deleteData)) {
-          LOG.info( mustPurge?
-                      "dropPartition() will purge partition-directories directly, skipping trash."
-                    :  "dropPartition() will move partition-directories to trash-directory.");
-          // Archived partitions have har:/to_har_file as their location.
-          // The original directory was saved in params
-          for (Path path : archToDelete) {
-            wh.deleteDir(path, true, mustPurge, needsCm);
-          }
-          for (PathAndPartValSize p : dirsToDelete) {
-            wh.deleteDir(p.path, true, mustPurge, needsCm);
-            try {
-              deleteParentRecursive(p.path.getParent(), p.partValSize - 1, mustPurge, needsCm);
-            } catch (IOException ex) {
-              LOG.warn("Error from deleteParentRecursive", ex);
-              throw new MetaException("Failed to delete parent: " + ex.getMessage());
-            }
-          }
-        }
-        if (parts != null) {
-          int i = 0;
-          if (parts != null && !listeners.isEmpty()) {
-            for (Partition part : parts) {
-              Map<String, String> parameters =
-                  (!transactionalListenerResponses.isEmpty()) ? transactionalListenerResponses.get(i) : null;
-
-              MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                    EventType.DROP_PARTITION,
-                                                    new DropPartitionEvent(tbl, part, success, deleteData, this),
-                                                    envContext,
-                                                    parameters, ms);
-
-              i++;
-            }
-          }
-        }
-      }
-    }
-
-    private void verifyIsWritablePath(Path dir) throws MetaException {
-      try {
-        if (!wh.isWritable(dir.getParent())) {
-          throw new MetaException("Table partition not deleted since " + dir.getParent()
-              + " is not writable by " + SecurityUtils.getUser());
-        }
-      } catch (IOException ex) {
-        LOG.warn("Error from isWritable", ex);
-        throw new MetaException("Table partition not deleted since " + dir.getParent()
-            + " access cannot be checked: " + ex.getMessage());
-      }
-    }
-
-    @Override
-    public boolean drop_partition_with_environment_context(final String db_name,
-        final String tbl_name, final List<String> part_vals, final boolean deleteData,
-        final EnvironmentContext envContext)
-        throws TException {
-      String[] parsedDbName = parseDbName(db_name, conf);
-      startPartitionFunction("drop_partition", parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-          tbl_name, part_vals);
-      LOG.info("Partition values:" + part_vals);
-
-      boolean ret = false;
-      Exception ex = null;
-      try {
-        ret = drop_partition_common(getMS(), parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-            tbl_name, part_vals, deleteData, envContext);
-      } catch (IOException e) {
-        ex = e;
-        throw new MetaException(e.getMessage());
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("drop_partition", ret, ex, tbl_name);
-      }
-      return ret;
-
-    }
-
-    /**
-     * Use {@link #get_partition_req(GetPartitionRequest)} ()} instead.
-     *
-     */
-    @Override
-    @Deprecated
-    public Partition get_partition(final String db_name, final String tbl_name,
-        final List<String> part_vals) throws MetaException, NoSuchObjectException {
-      String[] parsedDbName = parseDbName(db_name, conf);
-      startPartitionFunction("get_partition", parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-          tbl_name, part_vals);
-
-      Partition ret = null;
-      Exception ex = null;
-      try {
-        authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-        fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-        ret = getMS().getPartition(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, part_vals);
-        ret = FilterUtils.filterPartitionIfEnabled(isServerFilterEnabled, filterHook, ret);
-      } catch (Exception e) {
-        ex = e;
-        throwMetaException(e);
-      } finally {
-        endFunction("get_partition", ret != null, ex, tbl_name);
-      }
-      return ret;
-    }
-
-    @Override
-    public GetPartitionResponse get_partition_req(GetPartitionRequest req)
-        throws MetaException, NoSuchObjectException, TException {
-      // TODO Move the logic from get_partition to here, as that method is getting deprecated
-      String dbName = MetaStoreUtils.prependCatalogToDbName(req.getCatName(), req.getDbName(), conf);
-      Partition p = get_partition(dbName, req.getTblName(), req.getPartVals());
-      GetPartitionResponse res = new GetPartitionResponse();
-      res.setPartition(p);
-      return res;
-    }
-
-    /**
-     * Fire a pre-event for read table operation, if there are any
-     * pre-event listeners registered
-     */
-    private void fireReadTablePreEvent(String catName, String dbName, String tblName)
-        throws MetaException, NoSuchObjectException {
-      if(preListeners.size() > 0) {
-        Supplier<Table> tableSupplier = Suppliers.memoize(new Supplier<Table>() {
-          @Override public Table get() {
-            try {
-              Table t = getMS().getTable(catName, dbName, tblName, null);
-              if (t == null) {
-                throw new NoSuchObjectException(TableName.getQualified(catName, dbName, tblName)
-                    + " table not found");
-              }
-              return t;
-            } catch(MetaException | NoSuchObjectException e) {
-              throw new RuntimeException(e);
-            }
-          }
-        });
-        firePreEvent(new PreReadTableEvent(tableSupplier, this));
-      }
-    }
-
-    @Override
-    @Deprecated
-    public Partition get_partition_with_auth(final String db_name,
-        final String tbl_name, final List<String> part_vals,
-        final String user_name, final List<String> group_names)
-        throws TException {
-      String[] parsedDbName = parseDbName(db_name, conf);
-      startPartitionFunction("get_partition_with_auth", parsedDbName[CAT_NAME],
-          parsedDbName[DB_NAME], tbl_name, part_vals);
-      fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-      Partition ret = null;
-      Exception ex = null;
-      try {
-        authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-
-        ret = getMS().getPartitionWithAuth(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-            tbl_name, part_vals, user_name, group_names);
-        ret = FilterUtils.filterPartitionIfEnabled(isServerFilterEnabled, filterHook, ret);
-      } catch (InvalidObjectException e) {
-        ex = e;
-        throw new NoSuchObjectException(e.getMessage());
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_partition_with_auth", ret != null, ex, tbl_name);
-      }
-      return ret;
-    }
-
-    /**
-     * Use {@link #get_partitions_req(PartitionsRequest)} ()} instead.
-     *
-     */
-    @Override
-    @Deprecated
-    public List<Partition> get_partitions(final String db_name, final String tbl_name,
-        final short max_parts) throws NoSuchObjectException, MetaException {
-      String[] parsedDbName = parseDbName(db_name, conf);
-      startTableFunction("get_partitions", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-      fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-      List<Partition> ret = null;
-      Exception ex = null;
-      try {
-        checkLimitNumberOfPartitionsByFilter(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-            tbl_name, NO_FILTER_STRING, max_parts);
-
-        authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-
-        ret = getMS().getPartitions(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name,
-            max_parts);
-        ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret);
-      } catch (Exception e) {
-        ex = e;
-        throwMetaException(e);
-      } finally {
-        endFunction("get_partitions", ret != null, ex, tbl_name);
-      }
-      return ret;
-
-    }
-
-    @Override
-    public PartitionsResponse get_partitions_req(PartitionsRequest req)
-        throws NoSuchObjectException, MetaException, TException {
-      String dbName = MetaStoreUtils.prependCatalogToDbName(req.getCatName(), req.getDbName(), conf);
-      List<Partition> partitions = get_partitions(dbName, req.getTblName(), req.getMaxParts());
-      PartitionsResponse res = new PartitionsResponse();
-      res.setPartitions(partitions);
-      return res;
-    }
-
-    @Override
-    @Deprecated
-    public List<Partition> get_partitions_with_auth(final String dbName,
-        final String tblName, final short maxParts, final String userName,
-        final List<String> groupNames) throws TException {
-      String[] parsedDbName = parseDbName(dbName, conf);
-      startTableFunction("get_partitions_with_auth", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName);
-
-      List<Partition> ret = null;
-      Exception ex = null;
-      try {
-        checkLimitNumberOfPartitionsByFilter(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-            tblName, NO_FILTER_STRING, maxParts);
-
-        authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName);
-
-        ret = getMS().getPartitionsWithAuth(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName,
-            maxParts, userName, groupNames);
-        ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret);
-      } catch (InvalidObjectException e) {
-        ex = e;
-        throw new NoSuchObjectException(e.getMessage());
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_partitions_with_auth", ret != null, ex, tblName);
-      }
-      return ret;
-
-    }
-
-    private void checkLimitNumberOfPartitionsByFilter(String catName, String dbName,
-                                                      String tblName, String filterString,
-                                                      int maxParts) throws TException {
-      if (isPartitionLimitEnabled()) {
-        checkLimitNumberOfPartitions(tblName, get_num_partitions_by_filter(prependCatalogToDbName(
-            catName, dbName, conf), tblName, filterString), maxParts);
-      }
-    }
-
-    private void checkLimitNumberOfPartitionsByExpr(String catName, String dbName, String tblName,
-                                                    byte[] filterExpr, int maxParts)
-        throws TException {
-      if (isPartitionLimitEnabled()) {
-        checkLimitNumberOfPartitions(tblName, get_num_partitions_by_expr(catName, dbName, tblName,
-            filterExpr), maxParts);
-      }
-    }
-
-    private boolean isPartitionLimitEnabled() {
-      int partitionLimit = MetastoreConf.getIntVar(conf, ConfVars.LIMIT_PARTITION_REQUEST);
-      return partitionLimit > -1;
-    }
-
-    private void checkLimitNumberOfPartitions(String tblName, int numPartitions, int maxToFetch) throws MetaException {
-      if (isPartitionLimitEnabled()) {
-        int partitionLimit = MetastoreConf.getIntVar(conf, ConfVars.LIMIT_PARTITION_REQUEST);
-        int partitionRequest = (maxToFetch < 0) ? numPartitions : maxToFetch;
-        if (partitionRequest > partitionLimit) {
-          String configName = ConfVars.LIMIT_PARTITION_REQUEST.toString();
-          throw new MetaException(String.format(PARTITION_NUMBER_EXCEED_LIMIT_MSG, partitionRequest,
-              tblName, partitionLimit, configName));
-        }
-      }
-    }
-
-    @Override
-    @Deprecated
-    public List<PartitionSpec> get_partitions_pspec(final String db_name, final String tbl_name, final int max_parts)
-      throws NoSuchObjectException, MetaException  {
-
-      String[] parsedDbName = parseDbName(db_name, conf);
-      String tableName = tbl_name.toLowerCase();
-
-      startTableFunction("get_partitions_pspec", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName);
-
-      List<PartitionSpec> partitionSpecs = null;
-      try {
-        Table table = get_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName);
-        // get_partitions will parse out the catalog and db names itself
-        List<Partition> partitions = get_partitions(db_name, tableName, (short) max_parts);
-
-        if (is_partition_spec_grouping_enabled(table)) {
-          partitionSpecs = MetaStoreServerUtils
-              .getPartitionspecsGroupedByStorageDescriptor(table, partitions);
-        }
-        else {
-          PartitionSpec pSpec = new PartitionSpec();
-          pSpec.setPartitionList(new PartitionListComposingSpec(partitions));
-          pSpec.setCatName(parsedDbName[CAT_NAME]);
-          pSpec.setDbName(parsedDbName[DB_NAME]);
-          pSpec.setTableName(tableName);
-          pSpec.setRootPath(table.getSd().getLocation());
-          partitionSpecs = Arrays.asList(pSpec);
-        }
-
-        return partitionSpecs;
-      }
-      finally {
-        endFunction("get_partitions_pspec", partitionSpecs != null && !partitionSpecs.isEmpty(), null, tbl_name);
-      }
-    }
-
-    @Override
-    public GetPartitionsResponse get_partitions_with_specs(GetPartitionsRequest request)
-        throws MetaException, TException {
-      String catName = null;
-      if (request.isSetCatName()) {
-        catName = request.getCatName();
-      }
-      String[] parsedDbName = parseDbName(request.getDbName(), conf);
-      String tableName = request.getTblName();
-      if (catName == null) {
-        // if catName is not provided in the request use the catName parsed from the dbName
-        catName = parsedDbName[CAT_NAME];
-      }
-      startTableFunction("get_partitions_with_specs", catName, parsedDbName[DB_NAME],
-          tableName);
-      GetPartitionsResponse response = null;
-      Exception ex = null;
-      try {
-        Table table = get_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName);
-        List<Partition> partitions = getMS()
-            .getPartitionSpecsByFilterAndProjection(table, request.getProjectionSpec(),
-                request.getFilterSpec());
-        List<String> processorCapabilities = request.getProcessorCapabilities();
-        String processorId = request.getProcessorIdentifier();
-        if (processorCapabilities == null || processorCapabilities.size() == 0 ||
-              processorCapabilities.contains("MANAGERAWMETADATA")) {
-          LOG.info("Skipping translation for processor with " + processorId);
-        } else {
-          if (transformer != null) {
-            partitions = transformer.transformPartitions(partitions, table, processorCapabilities, processorId);
-          }
-        }
-        List<PartitionSpec> partitionSpecs =
-            MetaStoreServerUtils.getPartitionspecsGroupedByStorageDescriptor(table, partitions);
-        response = new GetPartitionsResponse();
-        response.setPartitionSpec(partitionSpecs);
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_partitions_with_specs", response != null, ex, tableName);
-      }
-      return response;
-    }
-
-    private static boolean is_partition_spec_grouping_enabled(Table table) {
-
-      Map<String, String> parameters = table.getParameters();
-      return parameters.containsKey("hive.hcatalog.partition.spec.grouping.enabled")
-          && parameters.get("hive.hcatalog.partition.spec.grouping.enabled").equalsIgnoreCase("true");
-    }
-
-    @Override
-    @Deprecated
-    public List<String> get_partition_names(final String db_name, final String tbl_name,
-        final short max_parts) throws NoSuchObjectException, MetaException {
-      String[] parsedDbName = parseDbName(db_name, conf);
-      startTableFunction("get_partition_names", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-      fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-      List<String> ret = null;
-      Exception ex = null;
-      try {
-        authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-        ret = getMS().listPartitionNames(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name,
-            max_parts);
-        ret = FilterUtils.filterPartitionNamesIfEnabled(isServerFilterEnabled,
-            filterHook, parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, ret);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_partition_names", ret != null, ex, tbl_name);
-      }
-      return ret;
-    }
-
-    @Override
-    public PartitionValuesResponse get_partition_values(PartitionValuesRequest request)
-        throws MetaException {
-      String catName = request.isSetCatName() ? request.getCatName() : getDefaultCatalog(conf);
-      String dbName = request.getDbName();
-      String tblName = request.getTblName();
-
-      try {
-        authorizeTableForPartitionMetadata(catName, dbName, tblName);
-
-        // This is serious black magic, as the following 2 lines do nothing AFAICT but without them
-        // the subsequent call to listPartitionValues fails.
-        List<FieldSchema> partCols = new ArrayList<FieldSchema>();
-        partCols.add(request.getPartitionKeys().get(0));
-        return getMS().listPartitionValues(catName, dbName, tblName, request.getPartitionKeys(),
-            request.isApplyDistinct(), request.getFilter(), request.isAscending(),
-            request.getPartitionOrder(), request.getMaxParts());
-      } catch (NoSuchObjectException e) {
-        LOG.error(String.format("Unable to get partition for %s.%s.%s", catName, dbName, tblName), e);
-        throw new MetaException(e.getMessage());
-      }
-    }
-
-    @Deprecated
-    @Override
-    public void alter_partition(final String db_name, final String tbl_name,
-        final Partition new_part)
-        throws TException {
-      rename_partition(db_name, tbl_name, null, new_part);
-    }
-
-    @Deprecated
-    @Override
-    public void alter_partition_with_environment_context(final String dbName,
-        final String tableName, final Partition newPartition,
-        final EnvironmentContext envContext)
-        throws TException {
-      String[] parsedDbName = parseDbName(dbName, conf);
-      // TODO: this method name is confusing, it actually does full alter (sortof)
-      rename_partition(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, null, newPartition,
-          envContext, null);
-    }
-
-    @Deprecated
-    @Override
-    public void rename_partition(final String db_name, final String tbl_name,
-        final List<String> part_vals, final Partition new_part)
-        throws TException {
-      // Call rename_partition without an environment context.
-      String[] parsedDbName = parseDbName(db_name, conf);
-      rename_partition(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, part_vals, new_part,
-          null, null);
-    }
-
-    @Override
-    public RenamePartitionResponse rename_partition_req(
-        RenamePartitionRequest req) throws InvalidOperationException ,MetaException ,TException {
-      rename_partition(req.getCatName(), req.getDbName(), req.getTableName(), req.getPartVals(),
-          req.getNewPart(), null, req.getValidWriteIdList());
-      return new RenamePartitionResponse();
-    };
-
-    private void rename_partition(String catName, String db_name, String tbl_name,
-        List<String> part_vals, Partition new_part, EnvironmentContext envContext,
-        String validWriteIds) throws TException {
-      startTableFunction("alter_partition", catName, db_name, tbl_name);
-
-      if (LOG.isInfoEnabled()) {
-        LOG.info("New partition values:" + new_part.getValues());
-        if (part_vals != null && part_vals.size() > 0) {
-          LOG.info("Old Partition values:" + part_vals);
-        }
-      }
-
-      // Adds the missing scheme/authority for the new partition location
-      if (new_part.getSd() != null) {
-        String newLocation = new_part.getSd().getLocation();
-        if (org.apache.commons.lang3.StringUtils.isNotEmpty(newLocation)) {
-          Path tblPath = wh.getDnsPath(new Path(newLocation));
-          new_part.getSd().setLocation(tblPath.toString());
-        }
-      }
-
-      // Make sure the new partition has the catalog value set
-      if (!new_part.isSetCatName()) {
-        new_part.setCatName(catName);
-      }
-
-      Partition oldPart = null;
-      Exception ex = null;
-      try {
-        firePreEvent(new PreAlterPartitionEvent(db_name, tbl_name, part_vals, new_part, this));
-        if (part_vals != null && !part_vals.isEmpty()) {
-          MetaStoreServerUtils.validatePartitionNameCharacters(new_part.getValues(),
-              partitionValidationPattern);
-        }
-
-        oldPart = alterHandler.alterPartition(getMS(), wh, catName, db_name, tbl_name,
-            part_vals, new_part, envContext, this, validWriteIds);
-
-        // Only fetch the table if we actually have a listener
-        Table table = null;
-        if (!listeners.isEmpty()) {
-          if (table == null) {
-            table = getMS().getTable(catName, db_name, tbl_name, null);
-          }
-
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                EventType.ALTER_PARTITION,
-                                                new AlterPartitionEvent(oldPart, new_part, table, false,
-                                                        true, new_part.getWriteId(), this),
-                                                envContext);
-        }
-      } catch (InvalidObjectException e) {
-        ex = e;
-        throw new InvalidOperationException(e.getMessage());
-      } catch (AlreadyExistsException e) {
-        ex = e;
-        throw new InvalidOperationException(e.getMessage());
-      } catch (MetaException | InvalidOperationException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("alter_partition", oldPart != null, ex, tbl_name);
-      }
-    }
-
-    @Override
-    public void alter_partitions(final String db_name, final String tbl_name,
-        final List<Partition> new_parts)
-        throws TException {
-      String[] o = parseDbName(db_name, conf);
-      alter_partitions_with_environment_context(o[0], o[1],
-          tbl_name, new_parts, null, null, -1);
-    }
-
-    @Override
-    public AlterPartitionsResponse alter_partitions_req(AlterPartitionsRequest req) throws TException {
-      alter_partitions_with_environment_context(req.getCatName(),
-          req.getDbName(), req.getTableName(), req.getPartitions(), req.getEnvironmentContext(),
-          req.isSetValidWriteIdList() ? req.getValidWriteIdList() : null,
-          req.isSetWriteId() ? req.getWriteId() : -1);
-      return new AlterPartitionsResponse();
-    }
-
-    // The old API we are keeping for backward compat. Not used within Hive.
-    @Deprecated
-    @Override
-    public void alter_partitions_with_environment_context(final String db_name, final String tbl_name,
-        final List<Partition> new_parts, EnvironmentContext environmentContext)
-            throws TException {
-      String[] o = parseDbName(db_name, conf);
-      alter_partitions_with_environment_context(o[0], o[1], tbl_name, new_parts, environmentContext,
-          null, -1);
-    }
-
-    private void alter_partitions_with_environment_context(String catName, String db_name, final String tbl_name,
-        final List<Partition> new_parts, EnvironmentContext environmentContext,
-        String writeIdList, long writeId)
-        throws TException {
-      if (environmentContext == null) {
-        environmentContext = new EnvironmentContext();
-      }
-      if (catName == null) {
-        catName = MetaStoreUtils.getDefaultCatalog(conf);
-      }
-
-      startTableFunction("alter_partitions", catName, db_name, tbl_name);
-
-      if (LOG.isInfoEnabled()) {
-        for (Partition tmpPart : new_parts) {
-          LOG.info("New partition values:" + tmpPart.getValues());
-        }
-      }
-      // all partitions are altered atomically
-      // all prehooks are fired together followed by all post hooks
-      List<Partition> oldParts = null;
-      Exception ex = null;
-      try {
-        for (Partition tmpPart : new_parts) {
-          // Make sure the catalog name is set in the new partition
-          if (!tmpPart.isSetCatName()) {
-            tmpPart.setCatName(getDefaultCatalog(conf));
-          }
-          firePreEvent(new PreAlterPartitionEvent(db_name, tbl_name, null, tmpPart, this));
-        }
-        oldParts = alterHandler.alterPartitions(getMS(), wh,
-            catName, db_name, tbl_name, new_parts, environmentContext, writeIdList, writeId, this);
-        Iterator<Partition> olditr = oldParts.iterator();
-        // Only fetch the table if we have a listener that needs it.
-        Table table = null;
-        for (Partition tmpPart : new_parts) {
-          Partition oldTmpPart;
-          if (olditr.hasNext()) {
-            oldTmpPart = olditr.next();
-          }
-          else {
-            throw new InvalidOperationException("failed to alterpartitions");
-          }
-
-          if (table == null) {
-            table = getMS().getTable(catName, db_name, tbl_name,  null);
-          }
-
-          if (!listeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                  EventType.ALTER_PARTITION,
-                                                  new AlterPartitionEvent(oldTmpPart, tmpPart, table, false,
-                                                   true, writeId, this));
-          }
-        }
-      } catch (InvalidObjectException e) {
-        ex = e;
-        throw new InvalidOperationException(e.getMessage());
-      } catch (AlreadyExistsException e) {
-        ex = e;
-        throw new InvalidOperationException(e.getMessage());
-      } catch (MetaException | InvalidOperationException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("alter_partition", oldParts != null, ex, tbl_name);
-      }
-    }
-
-    @Override
-    public String getVersion() throws TException {
-      String version = MetastoreVersionInfo.getVersion();
-      endFunction(startFunction("getVersion"), version != null, null);
-      return version;
-    }
-
-    @Override
-    public void alter_table(final String dbname, final String name,
-        final Table newTable)
-        throws InvalidOperationException, MetaException {
-      // Do not set an environment context.
-      String[] parsedDbName = parseDbName(dbname, conf);
-      alter_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], name, newTable,
-          null, null, null, null);
-    }
-
-    @Override
-    public void alter_table_with_cascade(final String dbname, final String name,
-        final Table newTable, final boolean cascade)
-        throws InvalidOperationException, MetaException {
-      EnvironmentContext envContext = null;
-      if (cascade) {
-        envContext = new EnvironmentContext();
-        envContext.putToProperties(StatsSetupConst.CASCADE, StatsSetupConst.TRUE);
-      }
-      String[] parsedDbName = parseDbName(dbname, conf);
-      alter_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], name, newTable,
-          envContext, null, null, null);
-    }
-
-    @Override
-    public AlterTableResponse alter_table_req(AlterTableRequest req)
-        throws InvalidOperationException, MetaException, TException {
-      alter_table_core(req.getCatName(), req.getDbName(), req.getTableName(),
-          req.getTable(), req.getEnvironmentContext(), req.getValidWriteIdList(),
-          req.getProcessorCapabilities(), req.getProcessorIdentifier());
-      return new AlterTableResponse();
-    }
-
-    @Override
-    public void alter_table_with_environment_context(final String dbname,
-        final String name, final Table newTable,
-        final EnvironmentContext envContext)
-        throws InvalidOperationException, MetaException {
-      String[] parsedDbName = parseDbName(dbname, conf);
-      alter_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-          name, newTable, envContext, null, null, null);
-    }
-
-    private void alter_table_core(String catName, String dbname, String name, Table newTable,
-        EnvironmentContext envContext, String validWriteIdList, List<String> processorCapabilities, String processorId)
-        throws InvalidOperationException, MetaException {
-      startFunction("alter_table", ": " + TableName.getQualified(catName, dbname, name)
-          + " newtbl=" + newTable.getTableName());
-      if (envContext == null) {
-        envContext = new EnvironmentContext();
-      }
-      if (catName == null) {
-        catName = MetaStoreUtils.getDefaultCatalog(conf);
-      }
-
-      // Update the time if it hasn't been specified.
-      if (newTable.getParameters() == null ||
-          newTable.getParameters().get(hive_metastoreConstants.DDL_TIME) == null) {
-        newTable.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System
-            .currentTimeMillis() / 1000));
-      }
-
-      // Adds the missing scheme/authority for the new table location
-      if (newTable.getSd() != null) {
-        String newLocation = newTable.getSd().getLocation();
-        if (org.apache.commons.lang3.StringUtils.isNotEmpty(newLocation)) {
-          Path tblPath = wh.getDnsPath(new Path(newLocation));
-          newTable.getSd().setLocation(tblPath.toString());
-        }
-      }
-      // Set the catalog name if it hasn't been set in the new table
-      if (!newTable.isSetCatName()) {
-        newTable.setCatName(catName);
-      }
-
-      boolean success = false;
-      Exception ex = null;
-      try {
-        Table oldt = get_table_core(catName, dbname, name, null);
-        if (transformer != null && !isInTest) {
-          newTable = transformer.transformAlterTable(newTable, processorCapabilities, processorId);
-        }
-        firePreEvent(new PreAlterTableEvent(oldt, newTable, this));
-        alterHandler.alterTable(getMS(), wh, catName, dbname, name, newTable,
-                envContext, this, validWriteIdList);
-        success = true;
-      } catch (NoSuchObjectException e) {
-        // thrown when the table to be altered does not exist
-        ex = e;
-        throw new InvalidOperationException(e.getMessage());
-      } catch (MetaException | InvalidOperationException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("alter_table", success, ex, name);
-      }
-    }
-
-    @Override
-    public List<String> get_tables(final String dbname, final String pattern)
-        throws MetaException {
-      startFunction("get_tables", ": db=" + dbname + " pat=" + pattern);
-
-      List<String> ret = null;
-      Exception ex = null;
-      Database db = null;
-      String[] parsedDbName = parseDbName(dbname, conf);
-      try {
-        db = get_database_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME]);
-        if (db != null && db.getType() != null) {
-          if (db.getType().equals(DatabaseType.REMOTE)) {
-            return DataConnectorProviderFactory.getDataConnectorProvider(db).getTableNames();
-          }
-        }
-      } catch (Exception e) { /* appears we return empty set instead of throwing an exception */ }
-
-      try {
-        ret = getMS().getTables(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], pattern);
-        if(ret !=  null && !ret.isEmpty()) {
-          List<Table> tableInfo = new ArrayList<>();
-          tableInfo = getMS().getTableObjectsByName(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], ret);
-          tableInfo = FilterUtils.filterTablesIfEnabled(isServerFilterEnabled, filterHook, tableInfo);// tableInfo object has the owner information of the table which is being passed to FilterUtils.
-          ret = new ArrayList<>();
-          for (Table tbl : tableInfo) {
-            ret.add(tbl.getTableName());
-          }
-        }
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_tables", ret != null, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public List<String> get_tables_by_type(final String dbname, final String pattern, final String tableType)
-        throws MetaException {
-      startFunction("get_tables_by_type", ": db=" + dbname + " pat=" + pattern + ",type=" + tableType);
-
-      List<String> ret = null;
-      Exception ex = null;
-      String[] parsedDbName = parseDbName(dbname, conf);
-      try {
-        ret = getTablesByTypeCore(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], pattern, tableType);
-        ret = FilterUtils.filterTableNamesIfEnabled(isServerFilterEnabled, filterHook,
-            parsedDbName[CAT_NAME], parsedDbName[DB_NAME], ret);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_tables_by_type", ret != null, ex);
-      }
-      return ret;
-    }
-
-    private List<String> getTablesByTypeCore(final String catName, final String dbname,
-        final String pattern, final String tableType) throws MetaException {
-      startFunction("getTablesByTypeCore", ": catName=" + catName +
-          ": db=" + dbname + " pat=" + pattern + ",type=" + tableType);
-
-      List<String> ret = null;
-      Exception ex = null;
-      Database db = null;
-      try {
-        db = get_database_core(catName, dbname);
-        if (db != null) {
-          if (db.getType().equals(DatabaseType.REMOTE)) {
-            return DataConnectorProviderFactory.getDataConnectorProvider(db).getTableNames();
-          }
-        }
-      } catch (Exception e) { /* ignore */ }
-
-      try {
-        ret = getMS().getTables(catName, dbname, pattern, TableType.valueOf(tableType), -1);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("getTablesByTypeCore", ret != null, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public List<Table> get_all_materialized_view_objects_for_rewriting()
-        throws MetaException {
-      startFunction("get_all_materialized_view_objects_for_rewriting");
-
-      List<Table> ret = null;
-      Exception ex = null;
-      try {
-        ret = getMS().getAllMaterializedViewObjectsForRewriting(DEFAULT_CATALOG_NAME);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_all_materialized_view_objects_for_rewriting", ret != null, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public List<String> get_materialized_views_for_rewriting(final String dbname)
-        throws MetaException {
-      startFunction("get_materialized_views_for_rewriting", ": db=" + dbname);
-
-      List<String> ret = null;
-      Exception ex = null;
-      String[] parsedDbName = parseDbName(dbname, conf);
-      try {
-        ret = getMS().getMaterializedViewsForRewriting(parsedDbName[CAT_NAME], parsedDbName[DB_NAME]);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_materialized_views_for_rewriting", ret != null, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public List<String> get_all_tables(final String dbname) throws MetaException {
-      startFunction("get_all_tables", ": db=" + dbname);
-
-      List<String> ret = null;
-      Exception ex = null;
-      String[] parsedDbName = parseDbName(dbname, conf);
-      try {
-        ret = getMS().getAllTables(parsedDbName[CAT_NAME], parsedDbName[DB_NAME]);
-        ret = FilterUtils.filterTableNamesIfEnabled(isServerFilterEnabled, filterHook,
-            parsedDbName[CAT_NAME], parsedDbName[DB_NAME], ret);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_all_tables", ret != null, ex);
-      }
-      return ret;
-    }
-
-    /**
-     * Use {@link #get_fields_req(GetFieldsRequest)} ()} instead.
-     *
-     */
-    @Override
-    @Deprecated
-    public List<FieldSchema> get_fields(String db, String tableName)
-        throws MetaException, UnknownTableException, UnknownDBException {
-      return get_fields_with_environment_context(db, tableName, null);
-    }
-
-    @Override
-    @Deprecated
-    public List<FieldSchema> get_fields_with_environment_context(String db, String tableName,
-        final EnvironmentContext envContext)
-        throws MetaException, UnknownTableException, UnknownDBException {
-      startFunction("get_fields_with_environment_context", ": db=" + db + "tbl=" + tableName);
-      String[] names = tableName.split("\\.");
-      String base_table_name = names[0];
-      String[] parsedDbName = parseDbName(db, conf);
-
-      Table tbl;
-      List<FieldSchema> ret = null;
-      Exception ex = null;
-      ClassLoader orgHiveLoader = null;
-      try {
-        try {
-          tbl = get_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], base_table_name);
-          firePreEvent(new PreReadTableEvent(tbl, this));
-        } catch (NoSuchObjectException e) {
-          throw new UnknownTableException(e.getMessage());
-        }
-        if (null == tbl.getSd().getSerdeInfo().getSerializationLib() ||
-          MetastoreConf.getStringCollection(conf,
-              ConfVars.SERDES_USING_METASTORE_FOR_SCHEMA).contains(
-                  tbl.getSd().getSerdeInfo().getSerializationLib())) {
-          ret = tbl.getSd().getCols();
-        } else {
-          StorageSchemaReader schemaReader = getStorageSchemaReader();
-          ret = schemaReader.readSchema(tbl, envContext, getConf());
-        }
-      } catch (UnknownTableException | MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        if (orgHiveLoader != null) {
-          conf.setClassLoader(orgHiveLoader);
-        }
-        endFunction("get_fields_with_environment_context", ret != null, ex, tableName);
-      }
-
-      return ret;
-    }
-
-    @Override
-    public GetFieldsResponse get_fields_req(GetFieldsRequest req)
-        throws MetaException, UnknownTableException, UnknownDBException, TException {
-      String dbName = MetaStoreUtils.prependCatalogToDbName(req.getCatName(), req.getDbName(), conf);
-      List<FieldSchema> fields = get_fields_with_environment_context(
-          dbName, req.getTblName(), req.getEnvContext());
-      GetFieldsResponse res = new GetFieldsResponse();
-      res.setFields(fields);
-      return res;
-    }
-
-    private StorageSchemaReader getStorageSchemaReader() throws MetaException {
-      if (storageSchemaReader == null) {
-        String className =
-            MetastoreConf.getVar(conf, MetastoreConf.ConfVars.STORAGE_SCHEMA_READER_IMPL);
-        Class<? extends StorageSchemaReader> readerClass =
-            JavaUtils.getClass(className, StorageSchemaReader.class);
-        try {
-          storageSchemaReader = readerClass.newInstance();
-        } catch (InstantiationException|IllegalAccessException e) {
-          LOG.error("Unable to instantiate class " + className, e);
-          throw new MetaException(e.getMessage());
-        }
-      }
-      return storageSchemaReader;
-    }
-
-    /**
-     * Use {@link #get_schema_req(GetSchemaRequest)} ()} instead.
-     *
-     */
-    @Override
-    @Deprecated
-    public List<FieldSchema> get_schema(String db, String tableName)
-        throws MetaException, UnknownTableException, UnknownDBException {
-      return get_schema_with_environment_context(db,tableName, null);
-    }
-
-    /**
-     * Return the schema of the table. This function includes partition columns
-     * in addition to the regular columns.
-     *
-     * @param db
-     *          Name of the database
-     * @param tableName
-     *          Name of the table
-     * @param envContext
-     *          Store session based properties
-     * @return List of columns, each column is a FieldSchema structure
-     * @throws MetaException
-     * @throws UnknownTableException
-     * @throws UnknownDBException
-     */
-    @Override
-    @Deprecated
-    public List<FieldSchema> get_schema_with_environment_context(String db, String tableName,
-          final EnvironmentContext envContext)
-        throws MetaException, UnknownTableException, UnknownDBException {
-      startFunction("get_schema_with_environment_context", ": db=" + db + "tbl=" + tableName);
-      boolean success = false;
-      Exception ex = null;
-      try {
-        String[] names = tableName.split("\\.");
-        String base_table_name = names[0];
-        String[] parsedDbName = parseDbName(db, conf);
-
-        Table tbl;
-        try {
-          tbl = get_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], base_table_name);
-        } catch (NoSuchObjectException e) {
-          throw new UnknownTableException(e.getMessage());
-        }
-        // Pass unparsed db name here
-        List<FieldSchema> fieldSchemas = get_fields_with_environment_context(db, base_table_name,
-            envContext);
-
-        if (tbl == null || fieldSchemas == null) {
-          throw new UnknownTableException(tableName + " doesn't exist");
-        }
-
-        if (tbl.getPartitionKeys() != null) {
-          // Combine the column field schemas and the partition keys to create the
-          // whole schema
-          fieldSchemas.addAll(tbl.getPartitionKeys());
-        }
-        success = true;
-        return fieldSchemas;
-      } catch (UnknownDBException | UnknownTableException | MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        MetaException me = new MetaException(e.toString());
-        me.initCause(e);
-        throw me;
-      } finally {
-        endFunction("get_schema_with_environment_context", success, ex, tableName);
-      }
-    }
-
-    @Override
-    public GetSchemaResponse get_schema_req(GetSchemaRequest req)
-        throws MetaException, UnknownTableException, UnknownDBException, TException {
-      String dbName = MetaStoreUtils.prependCatalogToDbName(req.getCatName(), req.getDbName(), conf);
-      List<FieldSchema> fields = get_schema_with_environment_context(
-          dbName, req.getTblName(), req.getEnvContext());
-      GetSchemaResponse res = new GetSchemaResponse();
-      res.setFields(fields);
-      return res;
-    }
-
-    @Override
-    public String getCpuProfile(int profileDurationInSec) throws TException {
-      return "";
-    }
-
-    /**
-     * Returns the value of the given configuration variable name. If the
-     * configuration variable with the given name doesn't exist, or if there
-     * were an exception thrown while retrieving the variable, or if name is
-     * null, defaultValue is returned.
-     */
-    @Override
-    public String get_config_value(String name, String defaultValue)
-        throws TException {
-      startFunction("get_config_value", ": name=" + name + " defaultValue="
-          + defaultValue);
-      boolean success = false;
-      Exception ex = null;
-      try {
-        if (name == null) {
-          success = true;
-          return defaultValue;
-        }
-        // Allow only keys that start with hive.*, hdfs.*, mapred.* for security
-        // i.e. don't allow access to db password
-        if (!Pattern.matches("(hive|hdfs|mapred|metastore).*", name)) {
-          throw new ConfigValSecurityException("For security reasons, the "
-              + "config key " + name + " cannot be accessed");
-        }
-
-        String toReturn = defaultValue;
-        try {
-          toReturn = MetastoreConf.get(conf, name);
-          if (toReturn == null) {
-            toReturn = defaultValue;
-          }
-        } catch (RuntimeException e) {
-          LOG.error(threadLocalId.get().toString() + ": "
-              + "RuntimeException thrown in get_config_value - msg: "
-              + e.getMessage() + " cause: " + e.getCause());
-        }
-        success = true;
-        return toReturn;
-      } catch (ConfigValSecurityException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw new TException(e);
-      } finally {
-        endFunction("get_config_value", success, ex);
-      }
-    }
-
-    private List<String> getPartValsFromName(Table t, String partName)
-        throws MetaException, InvalidObjectException {
-      Preconditions.checkArgument(t != null, "Table can not be null");
-      // Unescape the partition name
-      LinkedHashMap<String, String> hm = Warehouse.makeSpecFromName(partName);
-
-      List<String> partVals = new ArrayList<>();
-      for (FieldSchema field : t.getPartitionKeys()) {
-        String key = field.getName();
-        String val = hm.get(key);
-        if (val == null) {
-          throw new InvalidObjectException("incomplete partition name - missing " + key);
-        }
-        partVals.add(val);
-      }
-      return partVals;
-    }
-
-    private List<String> getPartValsFromName(RawStore ms, String catName, String dbName,
-                                             String tblName, String partName)
-        throws MetaException, InvalidObjectException {
-      Table t = ms.getTable(catName, dbName, tblName,  null);
-      if (t == null) {
-        throw new InvalidObjectException(dbName + "." + tblName
-            + " table not found");
-      }
-      return getPartValsFromName(t, partName);
-    }
-
-    private Partition get_partition_by_name_core(final RawStore ms, final String catName,
-                                                 final String db_name, final String tbl_name,
-                                                 final String part_name) throws TException {
-      fireReadTablePreEvent(catName, db_name, tbl_name);
-      List<String> partVals;
-      try {
-        partVals = getPartValsFromName(ms, catName, db_name, tbl_name, part_name);
-      } catch (InvalidObjectException e) {
-        throw new NoSuchObjectException(e.getMessage());
-      }
-      Partition p = ms.getPartition(catName, db_name, tbl_name, partVals);
-      p = FilterUtils.filterPartitionIfEnabled(isServerFilterEnabled, filterHook, p);
-
-      if (p == null) {
-        throw new NoSuchObjectException(TableName.getQualified(catName, db_name, tbl_name)
-            + " partition (" + part_name + ") not found");
-      }
-      return p;
-    }
-
-    @Override
-    @Deprecated
-    public Partition get_partition_by_name(final String db_name, final String tbl_name,
-        final String part_name) throws TException {
-
-      String[] parsedDbName = parseDbName(db_name, conf);
-      startFunction("get_partition_by_name", ": tbl=" +
-          TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name)
-          + " part=" + part_name);
-      Partition ret = null;
-      Exception ex = null;
-      try {
-        ret = get_partition_by_name_core(getMS(), parsedDbName[CAT_NAME],
-            parsedDbName[DB_NAME], tbl_name, part_name);
-        ret = FilterUtils.filterPartitionIfEnabled(isServerFilterEnabled, filterHook, ret);
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_partition_by_name", ret != null, ex, tbl_name);
-      }
-      return ret;
-    }
-
-    @Override
-    public Partition append_partition_by_name(final String db_name, final String tbl_name,
-        final String part_name) throws TException {
-      return append_partition_by_name_with_environment_context(db_name, tbl_name, part_name, null);
-    }
-
-    @Override
-    public Partition append_partition_by_name_with_environment_context(final String db_name,
-        final String tbl_name, final String part_name, final EnvironmentContext env_context)
-        throws TException {
-      String[] parsedDbName = parseDbName(db_name, conf);
-      startFunction("append_partition_by_name", ": tbl="
-          + TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-          tbl_name) + " part=" + part_name);
-
-      Partition ret = null;
-      Exception ex = null;
-      try {
-        RawStore ms = getMS();
-        List<String> partVals = getPartValsFromName(ms, parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, part_name);
-        ret = append_partition_common(ms, parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, partVals, env_context);
-      } catch (InvalidObjectException | AlreadyExistsException | MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("append_partition_by_name", ret != null, ex, tbl_name);
-      }
-      return ret;
-    }
-
-    private boolean drop_partition_by_name_core(final RawStore ms, final String catName,
-                                                final String db_name, final String tbl_name,
-                                                final String part_name, final boolean deleteData,
-                                                final EnvironmentContext envContext)
-        throws TException, IOException {
-
-      List<String> partVals;
-      try {
-        partVals = getPartValsFromName(ms, catName, db_name, tbl_name, part_name);
-      } catch (InvalidObjectException e) {
-        throw new NoSuchObjectException(e.getMessage());
-      }
-
-      return drop_partition_common(ms, catName, db_name, tbl_name, partVals, deleteData, envContext);
-    }
-
-    @Override
-    public boolean drop_partition_by_name(final String db_name, final String tbl_name,
-        final String part_name, final boolean deleteData) throws TException {
-      return drop_partition_by_name_with_environment_context(db_name, tbl_name, part_name,
-          deleteData, null);
-    }
-
-    @Override
-    public boolean drop_partition_by_name_with_environment_context(final String db_name,
-        final String tbl_name, final String part_name, final boolean deleteData,
-        final EnvironmentContext envContext) throws TException {
-      String[] parsedDbName = parseDbName(db_name, conf);
-      startFunction("drop_partition_by_name", ": tbl=" +
-          TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name)
-          + " part=" + part_name);
-
-      boolean ret = false;
-      Exception ex = null;
-      try {
-        ret = drop_partition_by_name_core(getMS(), parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-            tbl_name, part_name, deleteData, envContext);
-      } catch (IOException e) {
-        ex = e;
-        throw new MetaException(e.getMessage());
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("drop_partition_by_name", ret, ex, tbl_name);
-      }
-
-      return ret;
-    }
-
-    @Override
-    @Deprecated
-    public List<Partition> get_partitions_ps(final String db_name,
-        final String tbl_name, final List<String> part_vals,
-        final short max_parts) throws TException {
-      String[] parsedDbName = parseDbName(db_name, conf);
-      startPartitionFunction("get_partitions_ps", parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-          tbl_name, part_vals);
-
-      List<Partition> ret = null;
-      Exception ex = null;
-      try {
-        authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-        // Don't send the parsedDbName, as this method will parse itself.
-        ret = get_partitions_ps_with_auth(db_name, tbl_name, part_vals,
-            max_parts, null, null);
-        ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret);
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_partitions_ps", ret != null, ex, tbl_name);
-      }
-
-      return ret;
-    }
-
-    /**
-     * Use {@link #get_partitions_ps_with_auth_req(GetPartitionsPsWithAuthRequest)} ()} instead.
-     *
-     */
-    @Override
-    @Deprecated
-    public List<Partition> get_partitions_ps_with_auth(final String db_name,
-        final String tbl_name, final List<String> part_vals,
-        final short max_parts, final String userName,
-        final List<String> groupNames) throws TException {
-      String[] parsedDbName = parseDbName(db_name, conf);
-      startPartitionFunction("get_partitions_ps_with_auth", parsedDbName[CAT_NAME],
-          parsedDbName[DB_NAME], tbl_name, part_vals);
-      fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-      List<Partition> ret = null;
-      Exception ex = null;
-      try {
-        authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-        ret = getMS().listPartitionsPsWithAuth(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-            tbl_name, part_vals, max_parts, userName, groupNames);
-        ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret);
-      } catch (InvalidObjectException e) {
-        ex = e;
-        throw new MetaException(e.getMessage());
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_partitions_ps_with_auth", ret != null, ex, tbl_name);
-      }
-      return ret;
-    }
-
-    @Override
-    public GetPartitionsPsWithAuthResponse get_partitions_ps_with_auth_req(GetPartitionsPsWithAuthRequest req)
-            throws MetaException, NoSuchObjectException, TException {
-      String dbName = MetaStoreUtils.prependCatalogToDbName(req.getCatName(), req.getDbName(), conf);
-      List<Partition> partitions = null;
-      if (req.getPartVals() == null) {
-        partitions = get_partitions_with_auth(dbName, req.getTblName(), req.getMaxParts(), req.getUserName(),
-            req.getGroupNames());
-      } else {
-        partitions =
-            get_partitions_ps_with_auth(dbName, req.getTblName(), req.getPartVals(), req.getMaxParts(),
-                req.getUserName(), req.getGroupNames());
-      }
-      GetPartitionsPsWithAuthResponse res = new GetPartitionsPsWithAuthResponse();
-      res.setPartitions(partitions);
-      return res;
-    }
-
-    /**
-     * Use {@link #get_partition_names_ps_req(GetPartitionNamesPsRequest)} ()} instead.
-     *
-     */
-    @Override
-    @Deprecated
-    public List<String> get_partition_names_ps(final String db_name,
-        final String tbl_name, final List<String> part_vals, final short max_parts)
-        throws TException {
-      String[] parsedDbName = parseDbName(db_name, conf);
-      startPartitionFunction("get_partitions_names_ps", parsedDbName[CAT_NAME],
-          parsedDbName[DB_NAME], tbl_name, part_vals);
-      fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-      List<String> ret = null;
-      Exception ex = null;
-      try {
-        authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
-        ret = getMS().listPartitionNamesPs(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name,
-            part_vals, max_parts);
-        ret = FilterUtils.filterPartitionNamesIfEnabled(isServerFilterEnabled,
-            filterHook, parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, ret);
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_partitions_names_ps", ret != null, ex, tbl_name);
-      }
-      return ret;
-    }
-
-    @Override
-    public GetPartitionNamesPsResponse get_partition_names_ps_req(GetPartitionNamesPsRequest req)
-            throws MetaException, NoSuchObjectException, TException {
-      String dbName = MetaStoreUtils.prependCatalogToDbName(req.getCatName(), req.getDbName(), conf);
-      List<String> names = get_partition_names_ps(dbName, req.getTblName(), req.getPartValues(),
-              req.getMaxParts());
-      GetPartitionNamesPsResponse res = new GetPartitionNamesPsResponse();
-      res.setNames(names);
-      return res;
-    }
-
-    @Override
-    public List<String> get_partition_names_req(PartitionsByExprRequest req)
-        throws MetaException, NoSuchObjectException, TException {
-      String catName = req.isSetCatName() ? req.getCatName() : getDefaultCatalog(conf);
-      String dbName = req.getDbName(), tblName = req.getTblName();
-      startTableFunction("get_partition_names_req", catName,
-          dbName, tblName);
-      fireReadTablePreEvent(catName, dbName, tblName);
-      List<String> ret = null;
-      Exception ex = null;
-      try {
-        authorizeTableForPartitionMetadata(catName, dbName, tblName);
-        ret = getMS().listPartitionNames(catName, dbName, tblName,
-            req.getDefaultPartitionName(), req.getExpr(), req.getOrder(), req.getMaxParts());
-        ret = FilterUtils.filterPartitionNamesIfEnabled(isServerFilterEnabled,
-            filterHook, catName, dbName, tblName, ret);
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_partition_names_req", ret != null, ex, tblName);
-      }
-      return ret;
-    }
-
-    @Override
-    public List<String> partition_name_to_vals(String part_name) throws TException {
-      if (part_name.length() == 0) {
-        return Collections.emptyList();
-      }
-      LinkedHashMap<String, String> map = Warehouse.makeSpecFromName(part_name);
-      return new ArrayList<>(map.values());
-    }
-
-    @Override
-    public Map<String, String> partition_name_to_spec(String part_name) throws TException {
-      if (part_name.length() == 0) {
-        return new HashMap<>();
-      }
-      return Warehouse.makeSpecFromName(part_name);
-    }
-
-    private String lowerCaseConvertPartName(String partName) throws MetaException {
-      if (partName == null) return partName;
-      boolean isFirst = true;
-      Map<String, String> partSpec = Warehouse.makeEscSpecFromName(partName);
-      String convertedPartName = new String();
-
-      for (Map.Entry<String, String> entry : partSpec.entrySet()) {
-        String partColName = entry.getKey();
-        String partColVal = entry.getValue();
-
-        if (!isFirst) {
-          convertedPartName += "/";
-        } else {
-          isFirst = false;
-        }
-        convertedPartName += partColName.toLowerCase() + "=" + partColVal;
-      }
-      return convertedPartName;
-    }
-
-    @Override
-    @Deprecated
-    public ColumnStatistics get_table_column_statistics(String dbName, String tableName,
-      String colName) throws TException {
-      String[] parsedDbName = parseDbName(dbName, conf);
-      parsedDbName[CAT_NAME] = parsedDbName[CAT_NAME].toLowerCase();
-      parsedDbName[DB_NAME] = parsedDbName[DB_NAME].toLowerCase();
-      tableName = tableName.toLowerCase();
-      colName = colName.toLowerCase();
-      startFunction("get_column_statistics_by_table", ": table=" +
-          TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-              tableName) + " column=" + colName);
-      ColumnStatistics statsObj = null;
-      try {
-        statsObj = getMS().getTableColumnStatistics(
-            parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, Lists.newArrayList(colName),
-            "hive", null);
-        if (statsObj != null) {
-          assert statsObj.getStatsObjSize() <= 1;
-        }
-        return statsObj;
-      } finally {
-        endFunction("get_column_statistics_by_table", statsObj != null, null, tableName);
-      }
-    }
-
-    @Override
-    public TableStatsResult get_table_statistics_req(TableStatsRequest request) throws TException {
-      String catName = request.isSetCatName() ? request.getCatName().toLowerCase() :
-          getDefaultCatalog(conf);
-      String dbName = request.getDbName().toLowerCase();
-      String tblName = request.getTblName().toLowerCase();
-      startFunction("get_table_statistics_req", ": table=" +
-          TableName.getQualified(catName, dbName, tblName));
-      TableStatsResult result = null;
-      List<String> lowerCaseColNames = new ArrayList<>(request.getColNames().size());
-      for (String colName : request.getColNames()) {
-        lowerCaseColNames.add(colName.toLowerCase());
-      }
-      try {
-        ColumnStatistics cs = getMS().getTableColumnStatistics(
-            catName, dbName, tblName, lowerCaseColNames,
-            request.getEngine(), request.getValidWriteIdList());
-        // Note: stats compliance is not propagated to the client; instead, we just return nothing
-        //       if stats are not compliant for now. This won't work for stats merging, but that
-        //       is currently only done on metastore size (see set_aggr...).
-        //       For some optimizations we might make use of incorrect stats that are "better than
-        //       nothing", so this may change in future.
-        result = new TableStatsResult((cs == null || cs.getStatsObj() == null
-            || (cs.isSetIsStatsCompliant() && !cs.isIsStatsCompliant()))
-            ? Lists.newArrayList() : cs.getStatsObj());
-      } finally {
-        endFunction("get_table_statistics_req", result == null, null, tblName);
-      }
-      return result;
-    }
-
-    @Override
-    @Deprecated
-    public ColumnStatistics get_partition_column_statistics(String dbName, String tableName,
-      String partName, String colName) throws TException {
-      // Note: this method appears to be unused within Hive.
-      //       It doesn't take txn stats into account.
-      dbName = dbName.toLowerCase();
-      String[] parsedDbName = parseDbName(dbName, conf);
-      tableName = tableName.toLowerCase();
-      colName = colName.toLowerCase();
-      String convertedPartName = lowerCaseConvertPartName(partName);
-      startFunction("get_column_statistics_by_partition", ": table=" +
-          TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-          tableName) + " partition=" + convertedPartName + " column=" + colName);
-      ColumnStatistics statsObj = null;
-
-      try {
-        List<ColumnStatistics> list = getMS().getPartitionColumnStatistics(
-            parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName,
-            Lists.newArrayList(convertedPartName), Lists.newArrayList(colName),
-            "hive");
-        if (list.isEmpty()) {
-          return null;
-        }
-        if (list.size() != 1) {
-          throw new MetaException(list.size() + " statistics for single column and partition");
-        }
-        statsObj = list.get(0);
-      } finally {
-        endFunction("get_column_statistics_by_partition", statsObj != null, null, tableName);
-      }
-      return statsObj;
-    }
-
-    @Override
-    public PartitionsStatsResult get_partitions_statistics_req(PartitionsStatsRequest request)
-        throws TException {
-      String catName = request.isSetCatName() ? request.getCatName().toLowerCase() : getDefaultCatalog(conf);
-      String dbName = request.getDbName().toLowerCase();
-      String tblName = request.getTblName().toLowerCase();
-      startFunction("get_partitions_statistics_req", ": table=" +
-          TableName.getQualified(catName, dbName, tblName));
-
-      PartitionsStatsResult result = null;
-      List<String> lowerCaseColNames = new ArrayList<>(request.getColNames().size());
-      for (String colName : request.getColNames()) {
-        lowerCaseColNames.add(colName.toLowerCase());
-      }
-      List<String> lowerCasePartNames = new ArrayList<>(request.getPartNames().size());
-      for (String partName : request.getPartNames()) {
-        lowerCasePartNames.add(lowerCaseConvertPartName(partName));
-      }
-      try {
-        List<ColumnStatistics> stats = getMS().getPartitionColumnStatistics(
-            catName, dbName, tblName, lowerCasePartNames, lowerCaseColNames,
-            request.getEngine(), request.isSetValidWriteIdList() ? request.getValidWriteIdList() : null);
-        Map<String, List<ColumnStatisticsObj>> map = new HashMap<>();
-        if (stats != null) {
-          for (ColumnStatistics stat : stats) {
-            // Note: stats compliance is not propagated to the client; instead, we just return nothing
-            //       if stats are not compliant for now. This won't work for stats merging, but that
-            //       is currently only done on metastore size (see set_aggr...).
-            //       For some optimizations we might make use of incorrect stats that are "better than
-            //       nothing", so this may change in future.
-            if (stat.isSetIsStatsCompliant() && !stat.isIsStatsCompliant()) continue;
-            map.put(stat.getStatsDesc().getPartName(), stat.getStatsObj());
-          }
-        }
-        result = new PartitionsStatsResult(map);
-      } finally {
-        endFunction("get_partitions_statistics_req", result == null, null, tblName);
-      }
-      return result;
-    }
-
-    @Override
-    public boolean update_table_column_statistics(ColumnStatistics colStats) throws TException {
-      // Deprecated API, won't work for transactional tables
-      return updateTableColumnStatsInternal(colStats, null, -1);
-    }
-
-    @Override
-    public SetPartitionsStatsResponse update_table_column_statistics_req(
-        SetPartitionsStatsRequest req) throws NoSuchObjectException,
-        InvalidObjectException, MetaException, InvalidInputException,
-        TException {
-      if (req.getColStatsSize() != 1) {
-        throw new InvalidInputException("Only one stats object expected");
-      }
-      if (req.isNeedMerge()) {
-        throw new InvalidInputException("Merge is not supported for non-aggregate stats");
-      }
-      ColumnStatistics colStats = req.getColStatsIterator().next();
-      boolean ret = updateTableColumnStatsInternal(colStats,
-          req.getValidWriteIdList(), req.getWriteId());
-      return new SetPartitionsStatsResponse(ret);
-    }
-
-    private boolean updateTableColumnStatsInternal(ColumnStatistics colStats,
-        String validWriteIds, long writeId)
-        throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
-      normalizeColStatsInput(colStats);
-
-      startFunction("write_column_statistics", ":  table=" + TableName.getQualified(
-          colStats.getStatsDesc().getCatName(), colStats.getStatsDesc().getDbName(),
-          colStats.getStatsDesc().getTableName()));
-
-      Map<String, String> parameters = null;
-      getMS().openTransaction();
-      boolean committed = false;
-      try {
-        parameters = getMS().updateTableColumnStatistics(colStats, validWriteIds, writeId);
-        if (parameters != null) {
-          Table tableObj = getMS().getTable(colStats.getStatsDesc().getCatName(),
-                                            colStats.getStatsDesc().getDbName(),
-                                            colStats.getStatsDesc().getTableName(), validWriteIds);
-          if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                    EventType.UPDATE_TABLE_COLUMN_STAT,
-                    new UpdateTableColumnStatEvent(colStats, tableObj, parameters,
-                            writeId, this));
-          }
-          if (!listeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(listeners,
-                    EventType.UPDATE_TABLE_COLUMN_STAT,
-                    new UpdateTableColumnStatEvent(colStats, tableObj, parameters,
-                            writeId,this));
-          }
-        }
-        committed = getMS().commitTransaction();
-      } finally {
-        if (!committed) {
-          getMS().rollbackTransaction();
-        }
-        endFunction("write_column_statistics", parameters != null, null,
-            colStats.getStatsDesc().getTableName());
-      }
-
-      return parameters != null;
-    }
-
-    private void normalizeColStatsInput(ColumnStatistics colStats) throws MetaException {
-      // TODO: is this really needed? this code is propagated from HIVE-1362 but most of it is useless.
-      ColumnStatisticsDesc statsDesc = colStats.getStatsDesc();
-      statsDesc.setCatName(statsDesc.isSetCatName() ? statsDesc.getCatName().toLowerCase() : getDefaultCatalog(conf));
-      statsDesc.setDbName(statsDesc.getDbName().toLowerCase());
-      statsDesc.setTableName(statsDesc.getTableName().toLowerCase());
-      statsDesc.setPartName(lowerCaseConvertPartName(statsDesc.getPartName()));
-      long time = System.currentTimeMillis() / 1000;
-      statsDesc.setLastAnalyzed(time);
-
-      for (ColumnStatisticsObj statsObj : colStats.getStatsObj()) {
-        statsObj.setColName(statsObj.getColName().toLowerCase());
-        statsObj.setColType(statsObj.getColType().toLowerCase());
-      }
-      colStats.setStatsDesc(statsDesc);
-      colStats.setStatsObj(colStats.getStatsObj());
-    }
-
-    private boolean updatePartitonColStatsInternal(Table tbl, ColumnStatistics colStats,
-        String validWriteIds, long writeId)
-        throws MetaException, InvalidObjectException, NoSuchObjectException, InvalidInputException {
-      normalizeColStatsInput(colStats);
-
-      ColumnStatisticsDesc csd = colStats.getStatsDesc();
-      String catName = csd.getCatName(), dbName = csd.getDbName(), tableName = csd.getTableName();
-      startFunction("write_partition_column_statistics", ":  db=" + dbName  + " table=" + tableName
-              + " part=" + csd.getPartName());
-
-      boolean ret = false;
-
-      Map<String, String> parameters;
-      List<String> partVals;
-      boolean committed = false;
-      getMS().openTransaction();
-      try {
-        if (tbl == null) {
-          tbl = getTable(catName, dbName, tableName);
-        }
-        partVals = getPartValsFromName(tbl, csd.getPartName());
-        parameters = getMS().updatePartitionColumnStatistics(colStats, partVals, validWriteIds, writeId);
-        if (parameters != null) {
-          if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-              EventType.UPDATE_PARTITION_COLUMN_STAT,
-              new UpdatePartitionColumnStatEvent(colStats, partVals, parameters, tbl,
-                      writeId, this));
-          }
-          if (!listeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(listeners,
-              EventType.UPDATE_PARTITION_COLUMN_STAT,
-              new UpdatePartitionColumnStatEvent(colStats, partVals, parameters, tbl,
-                      writeId, this));
-          }
-        }
-        committed = getMS().commitTransaction();
-      } finally {
-        if (!committed) {
-          getMS().rollbackTransaction();
-        }
-        endFunction("write_partition_column_statistics", ret != false, null, tableName);
-      }
-
-      return parameters != null;
-    }
-
-    @Override
-    public boolean update_partition_column_statistics(ColumnStatistics colStats) throws TException {
-      // Deprecated API.
-      return updatePartitonColStatsInternal(null, colStats, null, -1);
-    }
-
-
-    @Override
-    public SetPartitionsStatsResponse update_partition_column_statistics_req(
-        SetPartitionsStatsRequest req) throws NoSuchObjectException,
-        InvalidObjectException, MetaException, InvalidInputException,
-        TException {
-      if (req.getColStatsSize() != 1) {
-        throw new InvalidInputException("Only one stats object expected");
-      }
-      if (req.isNeedMerge()) {
-        throw new InvalidInputException("Merge is not supported for non-aggregate stats");
-      }
-      ColumnStatistics colStats = req.getColStatsIterator().next();
-      boolean ret = updatePartitonColStatsInternal(null, colStats,
-          req.getValidWriteIdList(), req.getWriteId());
-      return new SetPartitionsStatsResponse(ret);
-    }
-
-    @Override
-    public boolean delete_partition_column_statistics(String dbName, String tableName,
-        String partName, String colName, String engine) throws TException {
-      dbName = dbName.toLowerCase();
-      String[] parsedDbName = parseDbName(dbName, conf);
-      tableName = tableName.toLowerCase();
-      if (colName != null) {
-        colName = colName.toLowerCase();
-      }
-      String convertedPartName = lowerCaseConvertPartName(partName);
-      startFunction("delete_column_statistics_by_partition",": table=" +
-          TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName) +
-          " partition=" + convertedPartName + " column=" + colName);
-      boolean ret = false, committed = false;
-
-      getMS().openTransaction();
-      try {
-        List<String> partVals = getPartValsFromName(getMS(), parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, convertedPartName);
-        Table table = getMS().getTable(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName);
-        // This API looks unused; if it were used we'd need to update stats state and write ID.
-        // We cannot just randomly nuke some txn stats.
-        if (TxnUtils.isTransactionalTable(table)) {
-          throw new MetaException("Cannot delete stats via this API for a transactional table");
-        }
-
-        ret = getMS().deletePartitionColumnStatistics(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName,
-            convertedPartName, partVals, colName, engine);
-        if (ret) {
-          if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                    EventType.DELETE_PARTITION_COLUMN_STAT,
-                    new DeletePartitionColumnStatEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName,
-                            convertedPartName, partVals, colName, engine, this));
-          }
-          if (!listeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(listeners,
-                    EventType.DELETE_PARTITION_COLUMN_STAT,
-                    new DeletePartitionColumnStatEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName,
-                            convertedPartName, partVals, colName, engine, this));
-          }
-        }
-        committed = getMS().commitTransaction();
-      } finally {
-        if (!committed) {
-          getMS().rollbackTransaction();
-        }
-        endFunction("delete_column_statistics_by_partition", ret != false, null, tableName);
-      }
-      return ret;
-    }
-
-    @Override
-    public boolean delete_table_column_statistics(String dbName, String tableName, String colName, String engine)
-        throws TException {
-      dbName = dbName.toLowerCase();
-      tableName = tableName.toLowerCase();
-
-      String[] parsedDbName = parseDbName(dbName, conf);
-
-      if (colName != null) {
-        colName = colName.toLowerCase();
-      }
-      startFunction("delete_column_statistics_by_table", ": table=" +
-          TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName) + " column=" +
-          colName);
-
-
-      boolean ret = false, committed = false;
-      getMS().openTransaction();
-      try {
-        Table table = getMS().getTable(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName);
-        // This API looks unused; if it were used we'd need to update stats state and write ID.
-        // We cannot just randomly nuke some txn stats.
-        if (TxnUtils.isTransactionalTable(table)) {
-          throw new MetaException("Cannot delete stats via this API for a transactional table");
-        }
-
-        ret = getMS().deleteTableColumnStatistics(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, colName, engine);
-        if (ret) {
-          if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                    EventType.DELETE_TABLE_COLUMN_STAT,
-                    new DeleteTableColumnStatEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-                            tableName, colName, engine, this));
-          }
-          if (!listeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(listeners,
-                    EventType.DELETE_TABLE_COLUMN_STAT,
-                    new DeleteTableColumnStatEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-                            tableName, colName, engine, this));
-          }
-        }
-        committed = getMS().commitTransaction();
-      } finally {
-        if (!committed) {
-          getMS().rollbackTransaction();
-        }
-        endFunction("delete_column_statistics_by_table", ret != false, null, tableName);
-      }
-      return ret;
-    }
-
-    @Override
-    @Deprecated
-    public List<Partition> get_partitions_by_filter(final String dbName, final String tblName,
-                                                    final String filter, final short maxParts)
-        throws TException {
-      String[] parsedDbName = parseDbName(dbName, conf);
-      startTableFunction("get_partitions_by_filter", parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-          tblName);
-      fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName);
-      List<Partition> ret = null;
-      Exception ex = null;
-      try {
-        checkLimitNumberOfPartitionsByFilter(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-            tblName, filter, maxParts);
-
-        authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName);
-
-        ret = getMS().getPartitionsByFilter(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName,
-            filter, maxParts);
-        ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret);
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_partitions_by_filter", ret != null, ex, tblName);
-      }
-      return ret;
-    }
-
-    @Override
-    @Deprecated
-    public List<PartitionSpec> get_part_specs_by_filter(final String dbName, final String tblName,
-                                                        final String filter, final int maxParts)
-        throws TException {
-
-      String[] parsedDbName = parseDbName(dbName, conf);
-      startTableFunction("get_partitions_by_filter_pspec", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName);
-
-      List<PartitionSpec> partitionSpecs = null;
-      try {
-        Table table = get_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName);
-        // Don't pass the parsed db name, as get_partitions_by_filter will parse it itself
-        List<Partition> partitions = get_partitions_by_filter(dbName, tblName, filter, (short) maxParts);
-
-        if (is_partition_spec_grouping_enabled(table)) {
-          partitionSpecs = MetaStoreServerUtils
-              .getPartitionspecsGroupedByStorageDescriptor(table, partitions);
-        }
-        else {
-          PartitionSpec pSpec = new PartitionSpec();
-          pSpec.setPartitionList(new PartitionListComposingSpec(partitions));
-          pSpec.setRootPath(table.getSd().getLocation());
-          pSpec.setCatName(parsedDbName[CAT_NAME]);
-          pSpec.setDbName(parsedDbName[DB_NAME]);
-          pSpec.setTableName(tblName);
-          partitionSpecs = Arrays.asList(pSpec);
-        }
-
-        return partitionSpecs;
-      }
-      finally {
-        endFunction("get_partitions_by_filter_pspec", partitionSpecs != null && !partitionSpecs.isEmpty(), null, tblName);
-      }
-    }
-
-    @Override
-    public PartitionsSpecByExprResult get_partitions_spec_by_expr(
-        PartitionsByExprRequest req) throws TException {
-      String dbName = req.getDbName(), tblName = req.getTblName();
-      String catName = req.isSetCatName() ? req.getCatName() : getDefaultCatalog(conf);
-      startTableFunction("get_partitions_spec_by_expr", catName, dbName, tblName);
-      fireReadTablePreEvent(catName, dbName, tblName);
-      PartitionsSpecByExprResult ret = null;
-      Exception ex = null;
-      try {
-        checkLimitNumberOfPartitionsByExpr(catName, dbName, tblName, req.getExpr(), UNLIMITED_MAX_PARTITIONS);
-        List<Partition> partitions = new LinkedList<>();
-        boolean hasUnknownPartitions = getMS().getPartitionsByExpr(catName, dbName, tblName,
-            req.getExpr(), req.getDefaultPartitionName(), req.getMaxParts(), partitions);
-        Table table = get_table_core(catName, dbName, tblName);
-        List<PartitionSpec> partitionSpecs =
-            MetaStoreServerUtils.getPartitionspecsGroupedByStorageDescriptor(table, partitions);
-        ret = new PartitionsSpecByExprResult(partitionSpecs, hasUnknownPartitions);
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_partitions_spec_by_expr", ret != null, ex, tblName);
-      }
-      return ret;
-    }
-
-    @Override
-    public PartitionsByExprResult get_partitions_by_expr(
-        PartitionsByExprRequest req) throws TException {
-      String dbName = req.getDbName(), tblName = req.getTblName();
-      String catName = req.isSetCatName() ? req.getCatName() : getDefaultCatalog(conf);
-      startTableFunction("get_partitions_by_expr", catName, dbName, tblName);
-      fireReadTablePreEvent(catName, dbName, tblName);
-      PartitionsByExprResult ret = null;
-      Exception ex = null;
-      try {
-        checkLimitNumberOfPartitionsByExpr(catName, dbName, tblName, req.getExpr(), UNLIMITED_MAX_PARTITIONS);
-        List<Partition> partitions = new LinkedList<>();
-        boolean hasUnknownPartitions = getMS().getPartitionsByExpr(catName, dbName, tblName,
-            req.getExpr(), req.getDefaultPartitionName(), req.getMaxParts(), partitions);
-        ret = new PartitionsByExprResult(partitions, hasUnknownPartitions);
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_partitions_by_expr", ret != null, ex, tblName);
-      }
-      return ret;
-    }
-
-    private void rethrowException(Exception e) throws TException {
-      // TODO: Both of these are TException, why do we need these separate clauses?
-      if (e instanceof MetaException) {
-        throw (MetaException) e;
-      } else if (e instanceof NoSuchObjectException) {
-        throw (NoSuchObjectException) e;
-      } else if (e instanceof TException) {
-        throw (TException) e;
-      } else {
-        throw newMetaException(e);
-      }
-    }
-
-    @Override
-    @Deprecated
-    public int get_num_partitions_by_filter(final String dbName,
-                                            final String tblName, final String filter)
-            throws TException {
-      String[] parsedDbName = parseDbName(dbName, conf);
-      if (parsedDbName[DB_NAME] == null || tblName == null) {
-        throw new MetaException("The DB and table name cannot be null.");
-      }
-      startTableFunction("get_num_partitions_by_filter", parsedDbName[CAT_NAME],
-          parsedDbName[DB_NAME], tblName);
-
-      int ret = -1;
-      Exception ex = null;
-      try {
-        ret = getMS().getNumPartitionsByFilter(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-            tblName, filter);
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_num_partitions_by_filter", ret != -1, ex, tblName);
-      }
-      return ret;
-    }
-
-    private int get_num_partitions_by_expr(final String catName, final String dbName,
-                                           final String tblName, final byte[] expr)
-        throws TException {
-      int ret = -1;
-      Exception ex = null;
-      try {
-        ret = getMS().getNumPartitionsByExpr(catName, dbName, tblName, expr);
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_num_partitions_by_expr", ret != -1, ex, tblName);
-      }
-      return ret;
-    }
-
-    @Override
-    @Deprecated
-    public List<Partition> get_partitions_by_names(final String dbName, final String tblName,
-                                                   final List<String> partNames)
-            throws TException {
-      return get_partitions_by_names(dbName, tblName, partNames);
-    }
-
-    @Override
-    public GetPartitionsByNamesResult get_partitions_by_names_req(GetPartitionsByNamesRequest gpbnr)
-            throws TException {
-      List<Partition> partitions = get_partitions_by_names(gpbnr.getDb_name(),
-              gpbnr.getTbl_name(), gpbnr.getNames(),
-              gpbnr.isSetGet_col_stats() && gpbnr.isGet_col_stats(), gpbnr.getEngine(),
-              gpbnr.getProcessorCapabilities(), gpbnr.getProcessorIdentifier());
-      return new GetPartitionsByNamesResult(partitions);
-    }
-
-    public List<Partition> get_partitions_by_names(final String dbName, final String tblName,
-           final List<String> partNames, boolean getColStats, String engine, String validWriteIdList)
-        throws TException {
-      return get_partitions_by_names(
-          dbName, tblName, partNames, getColStats, engine, null, null);
-    }
-
-    public List<Partition> get_partitions_by_names(final String dbName, final String tblName,
-           final List<String> partNames, boolean getColStats, String engine,
-           List<String> processorCapabilities, String processorId) throws TException {
-
-      String[] dbNameParts = parseDbName(dbName, conf);
-      String parsedCatName = dbNameParts[CAT_NAME];
-      String parsedDbName = dbNameParts[DB_NAME];
-      List<Partition> ret = null;
-      Table table = null;
-      Exception ex = null;
-      boolean success = false;
-      startTableFunction("get_partitions_by_names", parsedCatName, parsedDbName,
-              tblName);
-      try {
-        getMS().openTransaction();
-        authorizeTableForPartitionMetadata(parsedCatName, parsedDbName, tblName);
-
-        fireReadTablePreEvent(parsedCatName, parsedDbName, tblName);
-
-        ret = getMS().getPartitionsByNames(parsedCatName, parsedDbName, tblName, partNames);
-        ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret);
-        table = getTable(parsedCatName, parsedDbName, tblName);
-
-        // If requested add column statistics in each of the partition objects
-        if (getColStats) {
-          // Since each partition may have stats collected for different set of columns, we
-          // request them separately.
-          for (Partition part: ret) {
-            String partName = Warehouse.makePartName(table.getPartitionKeys(), part.getValues());
-            List<ColumnStatistics> partColStatsList =
-                    getMS().getPartitionColumnStatistics(parsedCatName, parsedDbName, tblName,
-                            Collections.singletonList(partName),
-                            StatsSetupConst.getColumnsHavingStats(part.getParameters()),
-                            engine);
-            if (partColStatsList != null && !partColStatsList.isEmpty()) {
-              ColumnStatistics partColStats = partColStatsList.get(0);
-              if (partColStats != null) {
-                part.setColStats(partColStats);
-              }
-            }
-          }
-        }
-
-        if (processorCapabilities == null || processorCapabilities.size() == 0 ||
-              processorCapabilities.contains("MANAGERAWMETADATA")) {
-          LOG.info("Skipping translation for processor with " + processorId);
-        } else {
-          if (transformer != null) {
-            ret = transformer.transformPartitions(ret, table, processorCapabilities, processorId);
-          }
-        }
-        success = getMS().commitTransaction();
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        if (!success) {
-          getMS().rollbackTransaction();
-        }
-        endFunction("get_partitions_by_names", ret != null, ex, tblName);
-      }
-      return ret;
-    }
-
-    @Override
-    public PrincipalPrivilegeSet get_privilege_set(HiveObjectRef hiveObject, String userName,
-                                                   List<String> groupNames) throws TException {
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      String catName = hiveObject.isSetCatName() ? hiveObject.getCatName() : getDefaultCatalog(conf);
-      if (hiveObject.getObjectType() == HiveObjectType.COLUMN) {
-        String partName = getPartName(hiveObject);
-        return this.get_column_privilege_set(catName, hiveObject.getDbName(), hiveObject
-            .getObjectName(), partName, hiveObject.getColumnName(), userName,
-            groupNames);
-      } else if (hiveObject.getObjectType() == HiveObjectType.PARTITION) {
-        String partName = getPartName(hiveObject);
-        return this.get_partition_privilege_set(catName, hiveObject.getDbName(),
-            hiveObject.getObjectName(), partName, userName, groupNames);
-      } else if (hiveObject.getObjectType() == HiveObjectType.DATABASE) {
-        return this.get_db_privilege_set(catName, hiveObject.getDbName(), userName,
-            groupNames);
-      } else if (hiveObject.getObjectType() == HiveObjectType.TABLE) {
-        return this.get_table_privilege_set(catName, hiveObject.getDbName(), hiveObject
-            .getObjectName(), userName, groupNames);
-      } else if (hiveObject.getObjectType() == HiveObjectType.GLOBAL) {
-        return this.get_user_privilege_set(userName, groupNames);
-      }
-      return null;
-    }
-
-    private String getPartName(HiveObjectRef hiveObject) throws MetaException {
-      String partName = null;
-      List<String> partValue = hiveObject.getPartValues();
-      if (partValue != null && partValue.size() > 0) {
-        try {
-          String catName = hiveObject.isSetCatName() ? hiveObject.getCatName() :
-              getDefaultCatalog(conf);
-          Table table = get_table_core(catName, hiveObject.getDbName(), hiveObject
-              .getObjectName());
-          partName = Warehouse
-              .makePartName(table.getPartitionKeys(), partValue);
-        } catch (NoSuchObjectException e) {
-          throw new MetaException(e.getMessage());
-        }
-      }
-      return partName;
-    }
-
-    private PrincipalPrivilegeSet get_column_privilege_set(String catName, final String dbName,
-        final String tableName, final String partName, final String columnName,
-        final String userName, final List<String> groupNames) throws TException {
-      incrementCounter("get_column_privilege_set");
-
-      PrincipalPrivilegeSet ret;
-      try {
-        ret = getMS().getColumnPrivilegeSet(
-            catName, dbName, tableName, partName, columnName, userName, groupNames);
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-      return ret;
-    }
-
-    private PrincipalPrivilegeSet get_db_privilege_set(String catName, final String dbName,
-        final String userName, final List<String> groupNames) throws TException {
-      incrementCounter("get_db_privilege_set");
-
-      PrincipalPrivilegeSet ret;
-      try {
-        ret = getMS().getDBPrivilegeSet(catName, dbName, userName, groupNames);
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-      return ret;
-    }
-
-    private PrincipalPrivilegeSet get_partition_privilege_set(
-        String catName, final String dbName, final String tableName, final String partName,
-        final String userName, final List<String> groupNames)
-        throws TException {
-      incrementCounter("get_partition_privilege_set");
-
-      PrincipalPrivilegeSet ret;
-      try {
-        ret = getMS().getPartitionPrivilegeSet(catName, dbName, tableName, partName,
-            userName, groupNames);
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-      return ret;
-    }
-
-    private PrincipalPrivilegeSet get_table_privilege_set(String catName, final String dbName,
-        final String tableName, final String userName,
-        final List<String> groupNames) throws TException {
-      incrementCounter("get_table_privilege_set");
-
-      PrincipalPrivilegeSet ret;
-      try {
-        ret = getMS().getTablePrivilegeSet(catName, dbName, tableName, userName,
-            groupNames);
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-      return ret;
-    }
-
-    @Override
-    public boolean grant_role(final String roleName,
-        final String principalName, final PrincipalType principalType,
-        final String grantor, final PrincipalType grantorType, final boolean grantOption)
-        throws TException {
-      incrementCounter("add_role_member");
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      if (PUBLIC.equals(roleName)) {
-        throw new MetaException("No user can be added to " + PUBLIC +". Since all users implicitly"
-        + " belong to " + PUBLIC + " role.");
-      }
-      Boolean ret;
-      try {
-        RawStore ms = getMS();
-        Role role = ms.getRole(roleName);
-        if(principalType == PrincipalType.ROLE){
-          //check if this grant statement will end up creating a cycle
-          if(isNewRoleAParent(principalName, roleName)){
-            throw new MetaException("Cannot grant role " + principalName + " to " + roleName +
-                " as " + roleName + " already belongs to the role " + principalName +
-                ". (no cycles allowed)");
-          }
-        }
-        ret = ms.grantRole(role, principalName, principalType, grantor, grantorType, grantOption);
-      } catch (MetaException e) {
-        throw e;
-      } catch (InvalidObjectException | NoSuchObjectException e) {
-        ret = false;
-        MetaStoreUtils.logAndThrowMetaException(e);
-      } catch (Exception e) {
-        throw new TException(e);
-      }
-      return ret;
-    }
-
-
-
-    /**
-     * Check if newRole is in parent hierarchy of curRole
-     * @param newRole
-     * @param curRole
-     * @return true if newRole is curRole or present in its hierarchy
-     * @throws MetaException
-     */
-    private boolean isNewRoleAParent(String newRole, String curRole) throws MetaException {
-      if(newRole.equals(curRole)){
-        return true;
-      }
-      //do this check recursively on all the parent roles of curRole
-      List<Role> parentRoleMaps = getMS().listRoles(curRole, PrincipalType.ROLE);
-      for(Role parentRole : parentRoleMaps){
-        if(isNewRoleAParent(newRole, parentRole.getRoleName())){
-          return true;
-        }
-      }
-      return false;
-    }
-
-    @Override
-    public List<Role> list_roles(final String principalName,
-        final PrincipalType principalType) throws TException {
-      incrementCounter("list_roles");
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      return getMS().listRoles(principalName, principalType);
-    }
-
-    @Override
-    public boolean create_role(final Role role) throws TException {
-      incrementCounter("create_role");
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      if (PUBLIC.equals(role.getRoleName())) {
-         throw new MetaException(PUBLIC + " role implicitly exists. It can't be created.");
-      }
-      Boolean ret;
-      try {
-        ret = getMS().addRole(role.getRoleName(), role.getOwnerName());
-      } catch (MetaException e) {
-        throw e;
-      } catch (InvalidObjectException | NoSuchObjectException e) {
-        ret = false;
-        MetaStoreUtils.logAndThrowMetaException(e);
-      } catch (Exception e) {
-        throw new TException(e);
-      }
-      return ret;
-    }
-
-    @Override
-    public boolean drop_role(final String roleName) throws TException {
-      incrementCounter("drop_role");
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      if (ADMIN.equals(roleName) || PUBLIC.equals(roleName)) {
-        throw new MetaException(PUBLIC + "," + ADMIN + " roles can't be dropped.");
-      }
-      Boolean ret;
-      try {
-        ret = getMS().removeRole(roleName);
-      } catch (MetaException e) {
-        throw e;
-      } catch (NoSuchObjectException e) {
-        ret = false;
-        MetaStoreUtils.logAndThrowMetaException(e);
-      } catch (Exception e) {
-        throw new TException(e);
-      }
-      return ret;
-    }
-
-    @Override
-    public List<String> get_role_names() throws TException {
-      incrementCounter("get_role_names");
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      List<String> ret;
-      try {
-        ret = getMS().listRoleNames();
-        return ret;
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    @Override
-    public boolean grant_privileges(final PrivilegeBag privileges) throws TException {
-      incrementCounter("grant_privileges");
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      Boolean ret;
-      try {
-        ret = getMS().grantPrivileges(privileges);
-      } catch (MetaException e) {
-        throw e;
-      } catch (InvalidObjectException | NoSuchObjectException e) {
-        ret = false;
-        MetaStoreUtils.logAndThrowMetaException(e);
-      } catch (Exception e) {
-        throw new TException(e);
-      }
-      return ret;
-    }
-
-    @Override
-    public boolean revoke_role(final String roleName, final String userName,
-        final PrincipalType principalType) throws TException {
-      return revoke_role(roleName, userName, principalType, false);
-    }
-
-    private boolean revoke_role(final String roleName, final String userName,
-        final PrincipalType principalType, boolean grantOption) throws TException {
-      incrementCounter("remove_role_member");
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      if (PUBLIC.equals(roleName)) {
-        throw new MetaException(PUBLIC + " role can't be revoked.");
-      }
-      Boolean ret;
-      try {
-        RawStore ms = getMS();
-        Role mRole = ms.getRole(roleName);
-        ret = ms.revokeRole(mRole, userName, principalType, grantOption);
-      } catch (MetaException e) {
-        throw e;
-      } catch (NoSuchObjectException e) {
-        ret = false;
-        MetaStoreUtils.logAndThrowMetaException(e);
-      } catch (Exception e) {
-        throw new TException(e);
-      }
-      return ret;
-    }
-
-    @Override
-    public GrantRevokeRoleResponse grant_revoke_role(GrantRevokeRoleRequest request)
-        throws TException {
-      GrantRevokeRoleResponse response = new GrantRevokeRoleResponse();
-      boolean grantOption = false;
-      if (request.isSetGrantOption()) {
-        grantOption = request.isGrantOption();
-      }
-      switch (request.getRequestType()) {
-        case GRANT: {
-          boolean result = grant_role(request.getRoleName(),
-              request.getPrincipalName(), request.getPrincipalType(),
-              request.getGrantor(), request.getGrantorType(), grantOption);
-          response.setSuccess(result);
-          break;
-        }
-        case REVOKE: {
-          boolean result = revoke_role(request.getRoleName(), request.getPrincipalName(),
-              request.getPrincipalType(), grantOption);
-          response.setSuccess(result);
-          break;
-        }
-        default:
-          throw new MetaException("Unknown request type " + request.getRequestType());
-      }
-
-      return response;
-    }
-
-    @Override
-    public GrantRevokePrivilegeResponse grant_revoke_privileges(GrantRevokePrivilegeRequest request)
-        throws TException {
-      GrantRevokePrivilegeResponse response = new GrantRevokePrivilegeResponse();
-      switch (request.getRequestType()) {
-        case GRANT: {
-          boolean result = grant_privileges(request.getPrivileges());
-          response.setSuccess(result);
-          break;
-        }
-        case REVOKE: {
-          boolean revokeGrantOption = false;
-          if (request.isSetRevokeGrantOption()) {
-            revokeGrantOption = request.isRevokeGrantOption();
-          }
-          boolean result = revoke_privileges(request.getPrivileges(), revokeGrantOption);
-          response.setSuccess(result);
-          break;
-        }
-        default:
-          throw new MetaException("Unknown request type " + request.getRequestType());
-      }
-
-      return response;
-    }
-
-    @Override
-    public GrantRevokePrivilegeResponse refresh_privileges(HiveObjectRef objToRefresh, String authorizer,
-        GrantRevokePrivilegeRequest grantRequest)
-        throws TException {
-      incrementCounter("refresh_privileges");
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      GrantRevokePrivilegeResponse response = new GrantRevokePrivilegeResponse();
-      try {
-        boolean result = getMS().refreshPrivileges(objToRefresh, authorizer, grantRequest.getPrivileges());
-        response.setSuccess(result);
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-      return response;
-    }
-
-    @Override
-    public boolean revoke_privileges(final PrivilegeBag privileges) throws TException {
-      return revoke_privileges(privileges, false);
-    }
-
-    public boolean revoke_privileges(final PrivilegeBag privileges, boolean grantOption)
-        throws TException {
-      incrementCounter("revoke_privileges");
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      Boolean ret;
-      try {
-        ret = getMS().revokePrivileges(privileges, grantOption);
-      } catch (MetaException e) {
-        throw e;
-      } catch (InvalidObjectException | NoSuchObjectException e) {
-        ret = false;
-        MetaStoreUtils.logAndThrowMetaException(e);
-      } catch (Exception e) {
-        throw new TException(e);
-      }
-      return ret;
-    }
-
-    private PrincipalPrivilegeSet get_user_privilege_set(final String userName,
-        final List<String> groupNames) throws TException {
-      incrementCounter("get_user_privilege_set");
-      PrincipalPrivilegeSet ret;
-      try {
-        ret = getMS().getUserPrivilegeSet(userName, groupNames);
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-      return ret;
-    }
-
-    @Override
-    public List<HiveObjectPrivilege> list_privileges(String principalName,
-        PrincipalType principalType, HiveObjectRef hiveObject)
-        throws TException {
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      String catName = hiveObject.isSetCatName() ? hiveObject.getCatName() : getDefaultCatalog(conf);
-      if (hiveObject.getObjectType() == null) {
-        return getAllPrivileges(principalName, principalType, catName);
-      }
-      if (hiveObject.getObjectType() == HiveObjectType.GLOBAL) {
-        return list_global_privileges(principalName, principalType);
-      }
-      if (hiveObject.getObjectType() == HiveObjectType.DATABASE) {
-        return list_db_privileges(principalName, principalType, catName, hiveObject
-            .getDbName());
-      }
-      if (hiveObject.getObjectType() == HiveObjectType.TABLE) {
-        return list_table_privileges(principalName, principalType,
-            catName, hiveObject.getDbName(), hiveObject.getObjectName());
-      }
-      if (hiveObject.getObjectType() == HiveObjectType.PARTITION) {
-        return list_partition_privileges(principalName, principalType,
-            catName, hiveObject.getDbName(), hiveObject.getObjectName(), hiveObject
-            .getPartValues());
-      }
-      if (hiveObject.getObjectType() == HiveObjectType.COLUMN) {
-        if (hiveObject.getPartValues() == null || hiveObject.getPartValues().isEmpty()) {
-          return list_table_column_privileges(principalName, principalType,
-              catName, hiveObject.getDbName(), hiveObject.getObjectName(), hiveObject.getColumnName());
-        }
-        return list_partition_column_privileges(principalName, principalType,
-            catName, hiveObject.getDbName(), hiveObject.getObjectName(), hiveObject
-            .getPartValues(), hiveObject.getColumnName());
-      }
-      return null;
-    }
-
-    private List<HiveObjectPrivilege> getAllPrivileges(String principalName,
-        PrincipalType principalType, String catName) throws TException {
-      List<HiveObjectPrivilege> privs = new ArrayList<>();
-      privs.addAll(list_global_privileges(principalName, principalType));
-      privs.addAll(list_db_privileges(principalName, principalType, catName, null));
-      privs.addAll(list_table_privileges(principalName, principalType, catName, null, null));
-      privs.addAll(list_partition_privileges(principalName, principalType, catName, null, null, null));
-      privs.addAll(list_table_column_privileges(principalName, principalType, catName, null, null, null));
-      privs.addAll(list_partition_column_privileges(principalName, principalType,
-          catName, null, null, null, null));
-      return privs;
-    }
-
-    private List<HiveObjectPrivilege> list_table_column_privileges(
-        final String principalName, final PrincipalType principalType, String catName,
-        final String dbName, final String tableName, final String columnName) throws TException {
-      incrementCounter("list_table_column_privileges");
-
-      try {
-        if (dbName == null) {
-          return getMS().listPrincipalTableColumnGrantsAll(principalName, principalType);
-        }
-        if (principalName == null) {
-          return getMS().listTableColumnGrantsAll(catName, dbName, tableName, columnName);
-        }
-        return getMS().listPrincipalTableColumnGrants(principalName, principalType,
-                catName, dbName, tableName, columnName);
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    private List<HiveObjectPrivilege> list_partition_column_privileges(
-        final String principalName, final PrincipalType principalType,
-        String catName, final String dbName, final String tableName, final List<String> partValues,
-        final String columnName) throws TException {
-      incrementCounter("list_partition_column_privileges");
-
-      try {
-        if (dbName == null) {
-          return getMS().listPrincipalPartitionColumnGrantsAll(principalName, principalType);
-        }
-        Table tbl = get_table_core(catName, dbName, tableName);
-        String partName = Warehouse.makePartName(tbl.getPartitionKeys(), partValues);
-        if (principalName == null) {
-          return getMS().listPartitionColumnGrantsAll(catName, dbName, tableName, partName, columnName);
-        }
-
-        return getMS().listPrincipalPartitionColumnGrants(principalName, principalType, catName, dbName,
-                tableName, partValues, partName, columnName);
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    private List<HiveObjectPrivilege> list_db_privileges(final String principalName,
-        final PrincipalType principalType, String catName, final String dbName) throws TException {
-      incrementCounter("list_security_db_grant");
-
-      try {
-        if (dbName == null) {
-          return getMS().listPrincipalDBGrantsAll(principalName, principalType);
-        }
-        if (principalName == null) {
-          return getMS().listDBGrantsAll(catName, dbName);
-        } else {
-          return getMS().listPrincipalDBGrants(principalName, principalType, catName, dbName);
-        }
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    private List<HiveObjectPrivilege> list_partition_privileges(
-        final String principalName, final PrincipalType principalType,
-        String catName, final String dbName, final String tableName, final List<String> partValues)
-        throws TException {
-      incrementCounter("list_security_partition_grant");
-
-      try {
-        if (dbName == null) {
-          return getMS().listPrincipalPartitionGrantsAll(principalName, principalType);
-        }
-        Table tbl = get_table_core(catName, dbName, tableName);
-        String partName = Warehouse.makePartName(tbl.getPartitionKeys(), partValues);
-        if (principalName == null) {
-          return getMS().listPartitionGrantsAll(catName, dbName, tableName, partName);
-        }
-        return getMS().listPrincipalPartitionGrants(
-            principalName, principalType, catName, dbName, tableName, partValues, partName);
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    private List<HiveObjectPrivilege> list_table_privileges(
-        final String principalName, final PrincipalType principalType,
-        String catName, final String dbName, final String tableName) throws TException {
-      incrementCounter("list_security_table_grant");
-
-      try {
-        if (dbName == null) {
-          return getMS().listPrincipalTableGrantsAll(principalName, principalType);
-        }
-        if (principalName == null) {
-          return getMS().listTableGrantsAll(catName, dbName, tableName);
-        }
-        return getMS().listAllTableGrants(principalName, principalType, catName, dbName, tableName);
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    private List<HiveObjectPrivilege> list_global_privileges(
-        final String principalName, final PrincipalType principalType) throws TException {
-      incrementCounter("list_security_user_grant");
-
-      try {
-        if (principalName == null) {
-          return getMS().listGlobalGrantsAll();
-        }
-        return getMS().listPrincipalGlobalGrants(principalName, principalType);
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    @Override
-    public void cancel_delegation_token(String token_str_form) throws TException {
-      startFunction("cancel_delegation_token");
-      boolean success = false;
-      Exception ex = null;
-      try {
-        HiveMetaStore.cancelDelegationToken(token_str_form);
-        success = true;
-      } catch (IOException e) {
-        ex = e;
-        throw new MetaException(e.getMessage());
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("cancel_delegation_token", success, ex);
-      }
-    }
-
-    @Override
-    public long renew_delegation_token(String token_str_form) throws TException {
-      startFunction("renew_delegation_token");
-      Long ret = null;
-      Exception ex = null;
-      try {
-        ret = HiveMetaStore.renewDelegationToken(token_str_form);
-      } catch (IOException e) {
-        ex = e;
-        throw new MetaException(e.getMessage());
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("renew_delegation_token", ret != null, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public String get_delegation_token(String token_owner, String renewer_kerberos_principal_name)
-        throws TException {
-      startFunction("get_delegation_token");
-      String ret = null;
-      Exception ex = null;
-      try {
-        ret =
-            HiveMetaStore.getDelegationToken(token_owner,
-                renewer_kerberos_principal_name, getIPAddress());
-      } catch (IOException | InterruptedException e) {
-        ex = e;
-        throw new MetaException(e.getMessage());
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_delegation_token", ret != null, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public boolean add_token(String token_identifier, String delegation_token) throws TException {
-      startFunction("add_token", ": " + token_identifier);
-      boolean ret = false;
-      Exception ex = null;
-      try {
-        ret = getMS().addToken(token_identifier, delegation_token);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("add_token", ret == true, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public boolean remove_token(String token_identifier) throws TException {
-      startFunction("remove_token", ": " + token_identifier);
-      boolean ret = false;
-      Exception ex = null;
-      try {
-        ret = getMS().removeToken(token_identifier);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("remove_token", ret == true, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public String get_token(String token_identifier) throws TException {
-      startFunction("get_token for", ": " + token_identifier);
-      String ret = null;
-      Exception ex = null;
-      try {
-        ret = getMS().getToken(token_identifier);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_token", ret != null, ex);
-      }
-      //Thrift cannot return null result
-      return ret == null ? "" : ret;
-    }
-
-    @Override
-    public List<String> get_all_token_identifiers() throws TException {
-      startFunction("get_all_token_identifiers.");
-      List<String> ret;
-      Exception ex = null;
-      try {
-        ret = getMS().getAllTokenIdentifiers();
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_all_token_identifiers.", ex == null, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public int add_master_key(String key) throws TException {
-      startFunction("add_master_key.");
-      int ret;
-      Exception ex = null;
-      try {
-        ret = getMS().addMasterKey(key);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("add_master_key.", ex == null, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public void update_master_key(int seq_number, String key) throws TException {
-      startFunction("update_master_key.");
-      Exception ex = null;
-      try {
-        getMS().updateMasterKey(seq_number, key);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("update_master_key.", ex == null, ex);
-      }
-    }
-
-    @Override
-    public boolean remove_master_key(int key_seq) throws TException {
-      startFunction("remove_master_key.");
-      Exception ex = null;
-      boolean ret;
-      try {
-        ret = getMS().removeMasterKey(key_seq);
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("remove_master_key.", ex == null, ex);
-      }
-      return ret;
-    }
-
-    @Override
-    public List<String> get_master_keys() throws TException {
-      startFunction("get_master_keys.");
-      Exception ex = null;
-      String [] ret = null;
-      try {
-        ret = getMS().getMasterKeys();
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_master_keys.", ret != null, ex);
-      }
-      return Arrays.asList(ret);
-    }
-
-    @Override
-    public void markPartitionForEvent(final String db_name, final String tbl_name,
-        final Map<String, String> partName, final PartitionEventType evtType) throws TException {
-
-      Table tbl = null;
-      Exception ex = null;
-      RawStore ms  = getMS();
-      boolean success = false;
-      try {
-        String[] parsedDbName = parseDbName(db_name, conf);
-        ms.openTransaction();
-        startPartitionFunction("markPartitionForEvent", parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-            tbl_name, partName);
-        firePreEvent(new PreLoadPartitionDoneEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-            tbl_name, partName, this));
-        tbl = ms.markPartitionForEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name,
-            partName, evtType);
-        if (null == tbl) {
-          throw new UnknownTableException("Table: " + tbl_name + " not found.");
-        }
-
-        if (transactionalListeners.size() > 0) {
-          LoadPartitionDoneEvent lpde = new LoadPartitionDoneEvent(true, tbl, partName, this);
-          for (MetaStoreEventListener transactionalListener : transactionalListeners) {
-            transactionalListener.onLoadPartitionDone(lpde);
-          }
-        }
-
-        success = ms.commitTransaction();
-        for (MetaStoreEventListener listener : listeners) {
-          listener.onLoadPartitionDone(new LoadPartitionDoneEvent(true, tbl, partName, this));
-        }
-      } catch (UnknownTableException | InvalidPartitionException | MetaException e) {
-        ex = e;
-        LOG.error("Exception caught in mark partition event ", e);
-        throw e;
-      } catch (Exception original) {
-        ex = original;
-        LOG.error("Exception caught in mark partition event ", original);
-        throw newMetaException(original);
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-
-        endFunction("markPartitionForEvent", tbl != null, ex, tbl_name);
-      }
-    }
-
-    @Override
-    public boolean isPartitionMarkedForEvent(final String db_name, final String tbl_name,
-        final Map<String, String> partName, final PartitionEventType evtType) throws TException {
-
-      String[] parsedDbName = parseDbName(db_name, conf);
-      startPartitionFunction("isPartitionMarkedForEvent", parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-          tbl_name, partName);
-      Boolean ret = null;
-      Exception ex = null;
-      try {
-        ret = getMS().isPartitionMarkedForEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-            tbl_name, partName, evtType);
-      } catch (UnknownTableException | UnknownPartitionException | InvalidPartitionException | MetaException e) {
-        ex = e;
-        LOG.error("Exception caught for isPartitionMarkedForEvent ", e);
-        throw e;
-      } catch (Exception original) {
-        LOG.error("Exception caught for isPartitionMarkedForEvent ", original);
-        ex = original;
-        throw newMetaException(original);
-      } finally {
-                endFunction("isPartitionMarkedForEvent", ret != null, ex, tbl_name);
-      }
-
-      return ret;
-    }
-
-    @Override
-    public List<String> set_ugi(String username, List<String> groupNames) throws TException {
-      Collections.addAll(groupNames, username);
-      return groupNames;
-    }
-
-    @Override
-    public boolean partition_name_has_valid_characters(List<String> part_vals,
-        boolean throw_exception) throws TException {
-      startFunction("partition_name_has_valid_characters");
-      boolean ret;
-      Exception ex = null;
-      try {
-        if (throw_exception) {
-          MetaStoreServerUtils.validatePartitionNameCharacters(part_vals, partitionValidationPattern);
-          ret = true;
-        } else {
-          ret = MetaStoreServerUtils.partitionNameHasValidCharacters(part_vals,
-              partitionValidationPattern);
-        }
-      } catch (MetaException e) {
-        ex = e;
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("partition_name_has_valid_characters", true, ex);
-      }
-      return ret;
-    }
-
-    private static MetaException newMetaException(Exception e) {
-      if (e instanceof MetaException) {
-        return (MetaException)e;
-      }
-      MetaException me = new MetaException(e.toString());
-      me.initCause(e);
-      return me;
-    }
-
-    private void validateFunctionInfo(Function func) throws InvalidObjectException, MetaException {
-      if (func == null) {
-        throw new MetaException("Function cannot be null.");
-      }
-      if (func.getFunctionName() == null) {
-        throw new MetaException("Function name cannot be null.");
-      }
-      if (func.getDbName() == null) {
-        throw new MetaException("Database name in Function cannot be null.");
-      }
-      if (!MetaStoreUtils.validateName(func.getFunctionName(), null)) {
-        throw new InvalidObjectException(func.getFunctionName() + " is not a valid object name");
-      }
-      String className = func.getClassName();
-      if (className == null) {
-        throw new InvalidObjectException("Function class name cannot be null");
-      }
-      if (func.getOwnerType() == null) {
-        throw new MetaException("Function owner type cannot be null.");
-      }
-      if (func.getFunctionType() == null) {
-        throw new MetaException("Function type cannot be null.");
-      }
-    }
-
-    @Override
-    public void create_function(Function func) throws TException {
-      validateFunctionInfo(func);
-      boolean success = false;
-      RawStore ms = getMS();
-      Map<String, String> transactionalListenerResponses = Collections.emptyMap();
-      try {
-        String catName = func.isSetCatName() ? func.getCatName() : getDefaultCatalog(conf);
-        ms.openTransaction();
-        Database db = ms.getDatabase(catName, func.getDbName());
-        if (db == null) {
-          throw new NoSuchObjectException("The database " + func.getDbName() + " does not exist");
-        }
-
-        if (db.getType() == DatabaseType.REMOTE) {
-          throw new MetaException("Operation create_function not support for REMOTE database");
-        }
-
-        Function existingFunc = ms.getFunction(catName, func.getDbName(), func.getFunctionName());
-        if (existingFunc != null) {
-          throw new AlreadyExistsException(
-              "Function " + func.getFunctionName() + " already exists");
-        }
-
-        long time = System.currentTimeMillis() / 1000;
-        func.setCreateTime((int) time);
-        ms.createFunction(func);
-        if (!transactionalListeners.isEmpty()) {
-          transactionalListenerResponses =
-              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                    EventType.CREATE_FUNCTION,
-                                                    new CreateFunctionEvent(func, true, this));
-        }
-
-        success = ms.commitTransaction();
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-
-        if (!listeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                EventType.CREATE_FUNCTION,
-                                                new CreateFunctionEvent(func, success, this),
-                                                null,
-                                                transactionalListenerResponses, ms);
-        }
-      }
-    }
-
-    @Override
-    public void drop_function(String dbName, String funcName)
-        throws NoSuchObjectException, MetaException,
-        InvalidObjectException, InvalidInputException {
-      if (funcName == null) {
-        throw new MetaException("Function name cannot be null.");
-      }
-      boolean success = false;
-      Function func = null;
-      RawStore ms = getMS();
-      Map<String, String> transactionalListenerResponses = Collections.emptyMap();
-      String[] parsedDbName = parseDbName(dbName, conf);
-      if (parsedDbName[DB_NAME] == null) {
-        throw new MetaException("Database name cannot be null.");
-      }
-      try {
-        ms.openTransaction();
-        func = ms.getFunction(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], funcName);
-        if (func == null) {
-          throw new NoSuchObjectException("Function " + funcName + " does not exist");
-        }
-        Boolean needsCm =
-              ReplChangeManager.isSourceOfReplication(get_database_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME]));
-
-        // if copy of jar to change management fails we fail the metastore transaction, since the
-        // user might delete the jars on HDFS externally after dropping the function, hence having
-        // a copy is required to allow incremental replication to work correctly.
-        if (func.getResourceUris() != null && !func.getResourceUris().isEmpty()) {
-          for (ResourceUri uri : func.getResourceUris()) {
-            if (uri.getUri().toLowerCase().startsWith("hdfs:") && needsCm) {
-              wh.addToChangeManagement(new Path(uri.getUri()));
-            }
-          }
-        }
-
-        // if the operation on metastore fails, we don't do anything in change management, but fail
-        // the metastore transaction, as having a copy of the jar in change management is not going
-        // to cause any problem, the cleaner thread will remove this when this jar expires.
-        ms.dropFunction(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], funcName);
-        if (transactionalListeners.size() > 0) {
-          transactionalListenerResponses =
-              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                    EventType.DROP_FUNCTION,
-                                                    new DropFunctionEvent(func, true, this));
-        }
-        success = ms.commitTransaction();
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-
-        if (listeners.size() > 0) {
-          MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                EventType.DROP_FUNCTION,
-                                                new DropFunctionEvent(func, success, this),
-                                                null,
-                                                transactionalListenerResponses, ms);
-        }
-      }
-    }
-
-    @Override
-    public void alter_function(String dbName, String funcName, Function newFunc) throws TException {
-      String[] parsedDbName = parseDbName(dbName, conf);
-      validateForAlterFunction(parsedDbName[DB_NAME], funcName, newFunc);
-      boolean success = false;
-      RawStore ms = getMS();
-      try {
-        ms.openTransaction();
-        ms.alterFunction(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], funcName, newFunc);
-        success = ms.commitTransaction();
-      } catch (InvalidObjectException e) {
-        // Throwing MetaException instead of InvalidObjectException as the InvalidObjectException
-        // is not defined for the alter_function method in the Thrift interface.
-        throwMetaException(e);
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-      }
-    }
-
-    private void validateForAlterFunction(String dbName, String funcName, Function newFunc)
-        throws MetaException {
-      if (dbName == null || funcName == null) {
-        throw new MetaException("Database and function name cannot be null.");
-      }
-      try {
-        validateFunctionInfo(newFunc);
-      } catch (InvalidObjectException e) {
-        // The validateFunctionInfo method is used by the create and alter function methods as well
-        // and it can throw InvalidObjectException. But the InvalidObjectException is not defined
-        // for the alter_function method in the Thrift interface, therefore a TApplicationException
-        // will occur at the caller side. Re-throwing the InvalidObjectException as MetaException
-        // would eliminate the TApplicationException at caller side.
-        throw newMetaException(e);
-      }
-    }
-
-    @Override
-    public List<String> get_functions(String dbName, String pattern)
-        throws MetaException {
-      startFunction("get_functions", ": db=" + dbName + " pat=" + pattern);
-
-      RawStore ms = getMS();
-      Exception ex = null;
-      List<String> funcNames = null;
-      String[] parsedDbName = parseDbName(dbName, conf);
-
-      try {
-        funcNames = ms.getFunctions(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], pattern);
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_functions", funcNames != null, ex);
-      }
-
-      return funcNames;
-    }
-
-    @Override
-    public GetAllFunctionsResponse get_all_functions()
-            throws MetaException {
-      GetAllFunctionsResponse response = new GetAllFunctionsResponse();
-      startFunction("get_all_functions");
-      RawStore ms = getMS();
-      List<Function> allFunctions = null;
-      Exception ex = null;
-      try {
-        // Leaving this as the 'hive' catalog (rather than choosing the default from the
-        // configuration) because all the default UDFs are in that catalog, and I think that's
-        // would people really want here.
-        allFunctions = ms.getAllFunctions(DEFAULT_CATALOG_NAME);
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_all_functions", allFunctions != null, ex);
-      }
-      response.setFunctions(allFunctions);
-      return response;
-    }
-
-    @Override
-    public Function get_function(String dbName, String funcName) throws TException {
-      if (dbName == null || funcName == null) {
-        throw new MetaException("Database and function name cannot be null.");
-      }
-      startFunction("get_function", ": " + dbName + "." + funcName);
-
-      RawStore ms = getMS();
-      Function func = null;
-      Exception ex = null;
-      String[] parsedDbName = parseDbName(dbName, conf);
-
-      try {
-        func = ms.getFunction(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], funcName);
-        if (func == null) {
-          throw new NoSuchObjectException(
-              "Function " + dbName + "." + funcName + " does not exist");
-        }
-      } catch (NoSuchObjectException e) {
-        ex = e;
-        rethrowException(e);
-      } catch (Exception e) {
-        ex = e;
-        throw newMetaException(e);
-      } finally {
-        endFunction("get_function", func != null, ex);
-      }
-
-      return func;
-    }
-
-    // Transaction and locking methods
-    @Override
-    public GetOpenTxnsResponse get_open_txns() throws TException {
-      return getTxnHandler().getOpenTxns();
-    }
-
-    @Override
-    public GetOpenTxnsResponse get_open_txns_req(GetOpenTxnsRequest getOpenTxnsRequest) throws TException {
-      return getTxnHandler().getOpenTxns(getOpenTxnsRequest.getExcludeTxnTypes());
-    }
-
-    // Transaction and locking methods
-    @Override
-    public GetOpenTxnsInfoResponse get_open_txns_info() throws TException {
-      return getTxnHandler().getOpenTxnsInfo();
-    }
-
-    @Override
-    public OpenTxnsResponse open_txns(OpenTxnRequest rqst) throws TException {
-      OpenTxnsResponse response = getTxnHandler().openTxns(rqst);
-      List<Long> txnIds = response.getTxn_ids();
-      if (txnIds != null && listeners != null && !listeners.isEmpty()) {
-        MetaStoreListenerNotifier.notifyEvent(listeners, EventType.OPEN_TXN,
-            new OpenTxnEvent(txnIds, this));
-      }
-      return response;
-    }
-
-    @Override
-    public void abort_txn(AbortTxnRequest rqst) throws TException {
-      getTxnHandler().abortTxn(rqst);
-      if (listeners != null && !listeners.isEmpty()) {
-        MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ABORT_TXN,
-                new AbortTxnEvent(rqst.getTxnid(), this));
-      }
-    }
-
-    @Override
-    public void abort_txns(AbortTxnsRequest rqst) throws TException {
-      getTxnHandler().abortTxns(rqst);
-      if (listeners != null && !listeners.isEmpty()) {
-        for (Long txnId : rqst.getTxn_ids()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ABORT_TXN,
-                  new AbortTxnEvent(txnId, this));
-        }
-      }
-    }
-
-    @Override
-    public long get_latest_txnid_in_conflict(long txnId) throws MetaException {
-      return getTxnHandler().getLatestTxnIdInConflict(txnId);
-    }
-
-    @Override
-    public void commit_txn(CommitTxnRequest rqst) throws TException {
-      // in replication flow, the write notification log table will be updated here.
-      if (rqst.isSetWriteEventInfos()) {
-        long targetTxnId = getTxnHandler().getTargetTxnId(rqst.getReplPolicy(), rqst.getTxnid());
-        if (targetTxnId < 0) {
-          //looks like a retry
-          return;
-        }
-        for (WriteEventInfo writeEventInfo : rqst.getWriteEventInfos()) {
-          String[] filesAdded = ReplChangeManager.getListFromSeparatedString(writeEventInfo.getFiles());
-          List<String> partitionValue = null;
-          Partition ptnObj = null;
-          String root;
-          Table tbl = getTblObject(writeEventInfo.getDatabase(), writeEventInfo.getTable());
-
-          if (writeEventInfo.getPartition() != null && !writeEventInfo.getPartition().isEmpty()) {
-            partitionValue = Warehouse.getPartValuesFromPartName(writeEventInfo.getPartition());
-            ptnObj = getPartitionObj(writeEventInfo.getDatabase(), writeEventInfo.getTable(), partitionValue, tbl);
-            root = ptnObj.getSd().getLocation();
-          } else {
-            root = tbl.getSd().getLocation();
-          }
-
-          InsertEventRequestData insertData = new InsertEventRequestData();
-          insertData.setReplace(true);
-
-          // The files in the commit txn message during load will have files with path corresponding to source
-          // warehouse. Need to transform them to target warehouse using table or partition object location.
-          for (String file : filesAdded) {
-            String[] decodedPath = ReplChangeManager.decodeFileUri(file);
-            String name = (new Path(decodedPath[0])).getName();
-            Path newPath = FileUtils.getTransformedPath(name, decodedPath[3], root);
-            insertData.addToFilesAdded(newPath.toUri().toString());
-            insertData.addToSubDirectoryList(decodedPath[3]);
-            try {
-              insertData.addToFilesAddedChecksum(ReplChangeManager.checksumFor(newPath, newPath.getFileSystem(conf)));
-            } catch (IOException e) {
-              LOG.error("failed to get checksum for the file " + newPath + " with error: " + e.getMessage());
-              throw new TException(e.getMessage());
-            }
-          }
-
-          WriteNotificationLogRequest wnRqst = new WriteNotificationLogRequest(targetTxnId,
-                  writeEventInfo.getWriteId(), writeEventInfo.getDatabase(), writeEventInfo.getTable(), insertData);
-          if (partitionValue != null) {
-            wnRqst.setPartitionVals(partitionValue);
-          }
-          addTxnWriteNotificationLog(tbl, ptnObj, wnRqst);
-        }
-      }
-      getTxnHandler().commitTxn(rqst);
-      if (listeners != null && !listeners.isEmpty()) {
-        MetaStoreListenerNotifier.notifyEvent(listeners, EventType.COMMIT_TXN,
-                new CommitTxnEvent(rqst.getTxnid(), this));
-        Optional<CompactionInfo> compactionInfo = getTxnHandler().getCompactionByTxnId(rqst.getTxnid());
-        if (compactionInfo.isPresent()) {
-          MetaStoreListenerNotifier.notifyEvent(listeners, EventType.COMMIT_COMPACTION,
-              new CommitCompactionEvent(rqst.getTxnid(), compactionInfo.get(), this));
-        }
-      }
-    }
-
-    @Override
-    public void repl_tbl_writeid_state(ReplTblWriteIdStateRequest rqst) throws TException {
-      getTxnHandler().replTableWriteIdState(rqst);
-    }
-
-    @Override
-    public GetValidWriteIdsResponse get_valid_write_ids(GetValidWriteIdsRequest rqst) throws TException {
-      return getTxnHandler().getValidWriteIds(rqst);
-    }
-
-    @Override
-    public void set_hadoop_jobid(String jobId, long cqId) {
-      getTxnHandler().setHadoopJobId(jobId, cqId);
-    }
-
-    @Override
-    public OptionalCompactionInfoStruct find_next_compact(String workerId) throws MetaException{
-      return CompactionInfo.compactionInfoToOptionalStruct(
-          getTxnHandler().findNextToCompact(workerId));
-    }
-
-    @Override
-    public void mark_cleaned(CompactionInfoStruct cr) throws MetaException {
-      getTxnHandler().markCleaned(CompactionInfo.compactionStructToInfo(cr));
-    }
-
-    @Override
-    public void mark_compacted(CompactionInfoStruct cr) throws MetaException {
-      getTxnHandler().markCompacted(CompactionInfo.compactionStructToInfo(cr));
-    }
-
-    @Override
-    public void mark_failed(CompactionInfoStruct cr) throws MetaException {
-      getTxnHandler().markFailed(CompactionInfo.compactionStructToInfo(cr));
-    }
-
-    @Override
-    public List<String> find_columns_with_stats(CompactionInfoStruct cr) throws MetaException {
-      return getTxnHandler().findColumnsWithStats(CompactionInfo.compactionStructToInfo(cr));
-    }
-
-    @Override
-    public void update_compactor_state(CompactionInfoStruct cr, long highWaterMark) throws MetaException {
-      getTxnHandler().updateCompactorState(
-          CompactionInfo.compactionStructToInfo(cr), highWaterMark);
-    }
-
-    @Override
-    public AllocateTableWriteIdsResponse allocate_table_write_ids(
-            AllocateTableWriteIdsRequest rqst) throws TException {
-      AllocateTableWriteIdsResponse response = getTxnHandler().allocateTableWriteIds(rqst);
-      if (listeners != null && !listeners.isEmpty()) {
-        MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ALLOC_WRITE_ID,
-                new AllocWriteIdEvent(response.getTxnToWriteIds(), rqst.getDbName(),
-                        rqst.getTableName(), this));
-      }
-      return response;
-    }
-
-    @Override
-    public MaxAllocatedTableWriteIdResponse get_max_allocated_table_write_id(MaxAllocatedTableWriteIdRequest rqst)
-        throws MetaException {
-      return getTxnHandler().getMaxAllocatedTableWrited(rqst);
-    }
-
-    @Override
-    public void seed_write_id(SeedTableWriteIdsRequest rqst) throws MetaException {
-      getTxnHandler().seedWriteId(rqst);
-    }
-
-    @Override
-    public void seed_txn_id(SeedTxnIdRequest rqst) throws MetaException {
-      getTxnHandler().seedTxnId(rqst);
-    }
-
-    private void addTxnWriteNotificationLog(Table tableObj, Partition ptnObj, WriteNotificationLogRequest rqst)
-            throws MetaException {
-      String partition = ""; //Empty string is an invalid partition name. Can be used for non partitioned table.
-      if (ptnObj != null) {
-        partition = Warehouse.makePartName(tableObj.getPartitionKeys(), rqst.getPartitionVals());
-      }
-      AcidWriteEvent event = new AcidWriteEvent(partition, tableObj, ptnObj, rqst);
-      getTxnHandler().addWriteNotificationLog(event);
-      if (listeners != null && !listeners.isEmpty()) {
-        MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ACID_WRITE, event);
-      }
-    }
-
-    private Table getTblObject(String db, String table) throws MetaException, NoSuchObjectException {
-      GetTableRequest req = new GetTableRequest(db, table);
-      req.setCapabilities(new ClientCapabilities(Lists.newArrayList(ClientCapability.TEST_CAPABILITY, ClientCapability.INSERT_ONLY_TABLES)));
-      return get_table_req(req).getTable();
-    }
-
-    private Partition getPartitionObj(String db, String table, List<String> partitionVals, Table tableObj)
-            throws MetaException, NoSuchObjectException {
-      if (tableObj.isSetPartitionKeys() && !tableObj.getPartitionKeys().isEmpty()) {
-        return get_partition(db, table, partitionVals);
-      }
-      return null;
-    }
-
-    @Override
-    public WriteNotificationLogResponse add_write_notification_log(WriteNotificationLogRequest rqst)
-            throws TException {
-      Table tableObj = getTblObject(rqst.getDb(), rqst.getTable());
-      Partition ptnObj = getPartitionObj(rqst.getDb(), rqst.getTable(), rqst.getPartitionVals(), tableObj);
-      addTxnWriteNotificationLog(tableObj, ptnObj, rqst);
-      return new WriteNotificationLogResponse();
-    }
-
-    @Override
-    public LockResponse lock(LockRequest rqst) throws TException {
-      return getTxnHandler().lock(rqst);
-    }
-
-    @Override
-    public LockResponse check_lock(CheckLockRequest rqst) throws TException {
-      return getTxnHandler().checkLock(rqst);
-    }
-
-    @Override
-    public void unlock(UnlockRequest rqst) throws TException {
-      getTxnHandler().unlock(rqst);
-    }
-
-    @Override
-    public ShowLocksResponse show_locks(ShowLocksRequest rqst) throws TException {
-      return getTxnHandler().showLocks(rqst);
-    }
-
-    @Override
-    public void heartbeat(HeartbeatRequest ids) throws TException {
-      getTxnHandler().heartbeat(ids);
-    }
-
-    @Override
-    public HeartbeatTxnRangeResponse heartbeat_txn_range(HeartbeatTxnRangeRequest rqst)
-      throws TException {
-      return getTxnHandler().heartbeatTxnRange(rqst);
-    }
-    @Deprecated
-    @Override
-    public void compact(CompactionRequest rqst) throws TException {
-      compact2(rqst);
-    }
-    @Override
-    public CompactionResponse compact2(CompactionRequest rqst) throws TException {
-      return getTxnHandler().compact(rqst);
-    }
-
-    @Override
-    public ShowCompactResponse show_compact(ShowCompactRequest rqst) throws TException {
-      ShowCompactResponse response = getTxnHandler().showCompact(rqst);
-      response.setCompacts(FilterUtils.filterCompactionsIfEnabled(isServerFilterEnabled,
-              filterHook, getDefaultCatalog(conf), response.getCompacts()));
-      return response;
-    }
-
-    @Override
-    public void flushCache() throws TException {
-      getMS().flushCache();
-    }
-
-    @Override
-    public void add_dynamic_partitions(AddDynamicPartitions rqst) throws TException {
-      getTxnHandler().addDynamicPartitions(rqst);
-    }
-
-    @Override
-    public GetPrincipalsInRoleResponse get_principals_in_role(GetPrincipalsInRoleRequest request)
-        throws TException {
-
-      incrementCounter("get_principals_in_role");
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      Exception ex = null;
-      GetPrincipalsInRoleResponse response = null;
-      try {
-        response = new GetPrincipalsInRoleResponse(getMS().listRoleMembers(request.getRoleName()));
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_principals_in_role", ex == null, ex);
-      }
-      return response;
-    }
-
-    @Override
-    public GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(
-        GetRoleGrantsForPrincipalRequest request) throws TException {
-
-      incrementCounter("get_role_grants_for_principal");
-      firePreEvent(new PreAuthorizationCallEvent(this));
-      Exception ex = null;
-      List<RolePrincipalGrant> roleMaps = null;
-      try {
-        roleMaps = getMS().listRolesWithGrants(request.getPrincipal_name(), request.getPrincipal_type());
-      } catch (MetaException e) {
-        throw e;
-      } catch (Exception e) {
-        ex = e;
-        rethrowException(e);
-      } finally {
-        endFunction("get_role_grants_for_principal", ex == null, ex);
-      }
-
-      //List<RolePrincipalGrant> roleGrantsList = getRolePrincipalGrants(roleMaps);
-      return new GetRoleGrantsForPrincipalResponse(roleMaps);
-    }
-
-    @Override
-    public AggrStats get_aggr_stats_for(PartitionsStatsRequest request) throws TException {
-      String catName = request.isSetCatName() ? request.getCatName().toLowerCase() :
-          getDefaultCatalog(conf);
-      String dbName = request.getDbName().toLowerCase();
-      String tblName = request.getTblName().toLowerCase();
-      startFunction("get_aggr_stats_for", ": table=" +
-          TableName.getQualified(catName, dbName, tblName));
-
-      List<String> lowerCaseColNames = new ArrayList<>(request.getColNames().size());
-      for (String colName : request.getColNames()) {
-        lowerCaseColNames.add(colName.toLowerCase());
-      }
-      List<String> lowerCasePartNames = new ArrayList<>(request.getPartNames().size());
-      for (String partName : request.getPartNames()) {
-        lowerCasePartNames.add(lowerCaseConvertPartName(partName));
-      }
-      AggrStats aggrStats = null;
-
-      try {
-        aggrStats = getMS().get_aggr_stats_for(catName, dbName, tblName,
-            lowerCasePartNames, lowerCaseColNames, request.getEngine(), request.getValidWriteIdList());
-        return aggrStats;
-      } finally {
-          endFunction("get_aggr_stats_for", aggrStats == null, null, request.getTblName());
-      }
-
-    }
-
-    @Override
-    public boolean set_aggr_stats_for(SetPartitionsStatsRequest request) throws TException {
-      boolean ret = true;
-      List<ColumnStatistics> csNews = request.getColStats();
-      if (csNews == null || csNews.isEmpty()) {
-        return ret;
-      }
-      // figure out if it is table level or partition level
-      ColumnStatistics firstColStats = csNews.get(0);
-      ColumnStatisticsDesc statsDesc = firstColStats.getStatsDesc();
-      String catName = statsDesc.isSetCatName() ? statsDesc.getCatName() : getDefaultCatalog(conf);
-      String dbName = statsDesc.getDbName();
-      String tableName = statsDesc.getTableName();
-      List<String> colNames = new ArrayList<>();
-      for (ColumnStatisticsObj obj : firstColStats.getStatsObj()) {
-        colNames.add(obj.getColName());
-      }
-      if (statsDesc.isIsTblLevel()) {
-        // there should be only one ColumnStatistics
-        if (request.getColStatsSize() != 1) {
-          throw new MetaException(
-              "Expecting only 1 ColumnStatistics for table's column stats, but find "
-                  + request.getColStatsSize());
-        }
-        if (request.isSetNeedMerge() && request.isNeedMerge()) {
-          return updateTableColumnStatsWithMerge(catName, dbName, tableName, colNames, request);
-        } else {
-          // This is the overwrite case, we do not care about the accuracy.
-          return updateTableColumnStatsInternal(firstColStats,
-              request.getValidWriteIdList(), request.getWriteId());
-        }
-      } else {
-        // partition level column stats merging
-        // note that we may have two or more duplicate partition names.
-        // see autoColumnStats_2.q under TestMiniLlapLocalCliDriver
-        Map<String, ColumnStatistics> newStatsMap = new HashMap<>();
-        for (ColumnStatistics csNew : csNews) {
-          String partName = csNew.getStatsDesc().getPartName();
-          if (newStatsMap.containsKey(partName)) {
-            MetaStoreServerUtils.mergeColStats(csNew, newStatsMap.get(partName));
-          }
-          newStatsMap.put(partName, csNew);
-        }
-
-        if (request.isSetNeedMerge() && request.isNeedMerge()) {
-          ret = updatePartColumnStatsWithMerge(catName, dbName, tableName,
-              colNames, newStatsMap, request);
-        } else { // No merge.
-          Table t = getTable(catName, dbName, tableName);
-          for (Entry<String, ColumnStatistics> entry : newStatsMap.entrySet()) {
-            // We don't short-circuit on errors here anymore. That can leave acid stats invalid.
-            ret = updatePartitonColStatsInternal(t, entry.getValue(),
-                request.getValidWriteIdList(), request.getWriteId()) && ret;
-          }
-        }
... 1435 lines suppressed ...