You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by dk...@apache.org on 2023/04/24 20:02:13 UTC

[hive] branch master updated: HIVE-27150: Support direct sql in drop single partition (Wechar Yu, reviewed by Sai Hemanth Gantasala, Denys Kuzmenko)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 7f46b40a5db HIVE-27150: Support direct sql in drop single partition (Wechar Yu, reviewed by Sai Hemanth Gantasala, Denys Kuzmenko)
7f46b40a5db is described below

commit 7f46b40a5dbb5ae546626eb9b5846ea05f2379e8
Author: Wechar Yu <yu...@gmail.com>
AuthorDate: Tue Apr 25 04:02:05 2023 +0800

    HIVE-27150: Support direct sql in drop single partition (Wechar Yu, reviewed by Sai Hemanth Gantasala, Denys Kuzmenko)
    
    Closes #4123
---
 .../hcatalog/listener/DummyRawStoreFailEvent.java  | 11 ++++
 .../apache/hadoop/hive/metastore/HMSHandler.java   | 24 ++++-----
 .../apache/hadoop/hive/metastore/ObjectStore.java  | 19 ++++++-
 .../org/apache/hadoop/hive/metastore/RawStore.java | 16 ++++++
 .../hadoop/hive/metastore/cache/CachedStore.java   | 15 ++++++
 .../metastore/DummyRawStoreControlledCommit.java   |  7 +++
 .../metastore/DummyRawStoreForJdoConnection.java   |  7 +++
 .../hadoop/hive/metastore/TestObjectStore.java     | 63 +++++++++++++++++++++-
 .../hadoop/hive/metastore/TestOldSchema.java       |  6 +--
 .../hadoop/hive/metastore/tools/BenchmarkTool.java |  4 +-
 .../hadoop/hive/metastore/tools/HMSBenchmarks.java | 22 ++++----
 .../hadoop/hive/metastore/tools/HMSClient.java     |  8 +++
 12 files changed, 169 insertions(+), 33 deletions(-)

diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index de76a7d8155..464555da4d6 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -371,6 +371,17 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
     }
   }
 
+  @Override
+  public boolean dropPartition(String catName, String dbName, String tableName, String partName)
+      throws MetaException, NoSuchObjectException,
+      InvalidObjectException, InvalidInputException {
+    if (shouldEventSucceed) {
+      return objectStore.dropPartition(catName, dbName, tableName, partName);
+    } else {
+      throw new RuntimeException("Event failed.");
+    }
+  }
+
   @Override
   public List<Partition> getPartitions(String catName, String dbName, String tableName, int max)
       throws MetaException, NoSuchObjectException {
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
index 52740a4e103..15bb8d82245 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
@@ -4903,7 +4903,7 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
         ms.addPartition(destPartition);
         destPartitions.add(destPartition);
         ms.dropPartition(parsedSourceDbName[CAT_NAME], partition.getDbName(), sourceTable.getTableName(),
-            partition.getValues());
+            Warehouse.makePartName(sourceTable.getPartitionKeys(), partition.getValues()));
       }
       Path destParentPath = destPath.getParent();
       if (!wh.isDir(destParentPath)) {
@@ -5026,20 +5026,18 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
         verifyIsWritablePath(partPath);
       }
 
-      if (!ms.dropPartition(catName, db_name, tbl_name, part_vals)) {
-        throw new MetaException("Unable to drop partition");
-      } else {
-        if (!transactionalListeners.isEmpty()) {
+      String partName = Warehouse.makePartName(tbl.getPartitionKeys(), part_vals);
+      ms.dropPartition(catName, db_name, tbl_name, partName);
 
-          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();
+      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();
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index e11677dfe34..085e272baf4 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -37,6 +37,7 @@ import java.time.ZoneId;
 import java.time.ZonedDateTime;
 import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -65,7 +66,6 @@ import javax.jdo.Transaction;
 import javax.jdo.datastore.JDOConnection;
 import javax.jdo.identity.IntIdentity;
 
-import com.google.common.base.Supplier;
 import com.google.common.util.concurrent.Striped;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.ArrayUtils;
@@ -251,7 +251,6 @@ import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.metastore.utils.FileUtils;
-import org.apache.hadoop.hive.metastore.utils.JavaUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.thrift.TException;
@@ -3101,6 +3100,22 @@ public class ObjectStore implements RawStore, Configurable {
     return success;
   }
 
+  @Override
+  public boolean dropPartition(String catName, String dbName, String tableName, String partName)
+      throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException {
+    boolean success = false;
+    try {
+      openTransaction();
+      dropPartitionsInternal(catName, dbName, tableName, Arrays.asList(partName), true, true);
+      success = commitTransaction();
+    } finally {
+      if (!success) {
+        rollbackTransaction();
+      }
+    }
+    return success;
+  }
+
   @Override
   public void dropPartitions(String catName, String dbName, String tblName, List<String> partNames)
       throws MetaException, NoSuchObjectException {
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index acfb617805d..b0f2df9c998 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -466,10 +466,26 @@ public interface RawStore extends Configurable {
    * @throws InvalidObjectException error dropping the statistics for the partition
    * @throws InvalidInputException error dropping the statistics for the partition
    */
+  @Deprecated
   boolean dropPartition(String catName, String dbName, String tableName,
       List<String> part_vals) throws MetaException, NoSuchObjectException, InvalidObjectException,
       InvalidInputException;
 
+  /**
+   * Drop a partition.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tableName table name.
+   * @param partName partition name.
+   * @return true if the partition was dropped.
+   * @throws MetaException Error accessing the RDBMS.
+   * @throws NoSuchObjectException no partition matching this description exists
+   * @throws InvalidObjectException error dropping the statistics for the partition
+   * @throws InvalidInputException error dropping the statistics for the partition
+   */
+  boolean dropPartition(String catName, String dbName, String tableName, String partName)
+      throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException;
+
   /**
    * Get some or all partitions for a table.
    * @param catName catalog name.
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index b11665c6543..1d475605b3b 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -1436,6 +1436,21 @@ public class CachedStore implements RawStore, Configurable {
     return succ;
   }
 
+  @Override public boolean dropPartition(String catName, String dbName, String tblName, String partName)
+      throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException {
+    boolean succ = rawStore.dropPartition(catName, dbName, tblName, partName);
+    // in case of event based cache update, cache will be updated during commit.
+    if (succ && !canUseEvents) {
+      catName = normalizeIdentifier(catName);
+      dbName = normalizeIdentifier(dbName);
+      tblName = normalizeIdentifier(tblName);
+      if (shouldCacheTable(catName, dbName, tblName)) {
+        sharedCache.removePartitionFromCache(catName, dbName, tblName, partNameToVals(partName));
+      }
+    }
+    return succ;
+  }
+
   @Override public void dropPartitions(String catName, String dbName, String tblName, List<String> partNames)
       throws MetaException, NoSuchObjectException {
     rawStore.dropPartitions(catName, dbName, tblName, partNames);
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index f3d79dabfdd..bc19b961e91 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -340,6 +340,13 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
     return objectStore.dropPartition(catName, dbName, tableName, partVals);
   }
 
+  @Override
+  public boolean dropPartition(String catName, String dbName, String tableName, String partName)
+      throws MetaException, NoSuchObjectException,
+      InvalidObjectException, InvalidInputException {
+    return objectStore.dropPartition(catName, dbName, tableName, partName);
+  }
+
   @Override
   public List<Partition> getPartitions(String catName, String dbName, String tableName, int max)
       throws MetaException, NoSuchObjectException {
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 662a21ccb8d..52ae4b8b12c 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -351,6 +351,13 @@ public class DummyRawStoreForJdoConnection implements RawStore {
     return false;
   }
 
+  @Override
+  public boolean dropPartition(String catName, String dbName, String tableName, String partName)
+      throws MetaException {
+
+    return false;
+  }
+
   @Override
   public List<Partition> getPartitions(String catName, String dbName, String tableName, int max)
       throws MetaException {
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
index 79036c2c22c..bc9e04a227b 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -59,7 +59,6 @@ import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
-import org.apache.hadoop.hive.metastore.api.SourceTable;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.StoredProcedure;
 import org.apache.hadoop.hive.metastore.api.Table;
@@ -498,6 +497,68 @@ public class TestObjectStore {
     }
   }
 
+  @Test
+  public void testDropPartitionByName() throws Exception {
+    Database db1 = new DatabaseBuilder()
+        .setName(DB1)
+        .setDescription("description")
+        .setLocation("locationurl")
+        .build(conf);
+    try (AutoCloseable c = deadline()) {
+      objectStore.createDatabase(db1);
+    }
+    StorageDescriptor sd = createFakeSd("location");
+    HashMap<String, String> tableParams = new HashMap<>();
+    tableParams.put("EXTERNAL", "false");
+    FieldSchema partitionKey1 = new FieldSchema("Country", ColumnType.STRING_TYPE_NAME, "");
+    FieldSchema partitionKey2 = new FieldSchema("State", ColumnType.STRING_TYPE_NAME, "");
+    Table tbl1 =
+        new Table(TABLE1, DB1, "owner", 1, 2, 3, sd, Arrays.asList(partitionKey1, partitionKey2),
+            tableParams, null, null, "MANAGED_TABLE");
+    try (AutoCloseable c = deadline()) {
+      objectStore.createTable(tbl1);
+    }
+    HashMap<String, String> partitionParams = new HashMap<>();
+    partitionParams.put("PARTITION_LEVEL_PRIVILEGE", "true");
+    List<String> value1 = Arrays.asList("US", "CA");
+    Partition part1 = new Partition(value1, DB1, TABLE1, 111, 111, sd, partitionParams);
+    part1.setCatName(DEFAULT_CATALOG_NAME);
+    try (AutoCloseable c = deadline()) {
+      objectStore.addPartition(part1);
+    }
+    List<String> value2 = Arrays.asList("US", "MA");
+    Partition part2 = new Partition(value2, DB1, TABLE1, 222, 222, sd, partitionParams);
+    part2.setCatName(DEFAULT_CATALOG_NAME);
+    try (AutoCloseable c = deadline()) {
+      objectStore.addPartition(part2);
+    }
+
+    List<Partition> partitions;
+    try (AutoCloseable c = deadline()) {
+      objectStore.dropPartition(DEFAULT_CATALOG_NAME, DB1, TABLE1, "country=US/state=CA");
+      partitions = objectStore.getPartitions(DEFAULT_CATALOG_NAME, DB1, TABLE1, 10);
+    }
+    Assert.assertEquals(1, partitions.size());
+    Assert.assertEquals(222, partitions.get(0).getCreateTime());
+    try (AutoCloseable c = deadline()) {
+      objectStore.dropPartition(DEFAULT_CATALOG_NAME, DB1, TABLE1, "country=US/state=MA");
+      partitions = objectStore.getPartitions(DEFAULT_CATALOG_NAME, DB1, TABLE1, 10);
+    }
+    Assert.assertEquals(0, partitions.size());
+
+    try (AutoCloseable c = deadline()) {
+      // Illegal partName will do nothing, it doesn't matter
+      // because the real HMSHandler will guarantee the partName is legal and exists.
+      objectStore.dropPartition(DEFAULT_CATALOG_NAME, DB1, TABLE1, "country=US/state=NON_EXIST");
+      objectStore.dropPartition(DEFAULT_CATALOG_NAME, DB1, TABLE1, "country=US/st=CA");
+    }
+
+    try (AutoCloseable c = deadline()) {
+      objectStore.dropTable(DEFAULT_CATALOG_NAME, DB1, TABLE1);
+      objectStore.dropDatabase(db1.getCatalogName(), DB1);
+    }
+  }
+
   /**
    * Test the concurrent drop of same partition would leak transaction.
    * https://issues.apache.org/jira/browse/HIVE-16839
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
index 55bd6c7b7bf..14110c6dbe1 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
@@ -221,10 +221,8 @@ public class TestOldSchema {
         String db = dbs.get(i);
         List<String> tbls = store.getAllTables(DEFAULT_CATALOG_NAME, db);
         for (String tbl : tbls) {
-          List<Partition> parts = store.getPartitions(DEFAULT_CATALOG_NAME, db, tbl, 100);
-          for (Partition part : parts) {
-            store.dropPartition(DEFAULT_CATALOG_NAME, db, tbl, part.getValues());
-          }
+          List<String> partNames = store.listPartitionNames(DEFAULT_CATALOG_NAME, db, tbl, (short) -1);
+          store.dropPartitions(DEFAULT_CATALOG_NAME, db, tbl, partNames);
           store.dropTable(DEFAULT_CATALOG_NAME, db, tbl);
         }
         store.dropDatabase(DEFAULT_CATALOG_NAME, db);
diff --git a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java
index 08661b18950..943b87e9c97 100644
--- a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java
+++ b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java
@@ -272,7 +272,7 @@ public class BenchmarkTool implements Runnable {
         .add("dropTableWithPartitions",
             () -> benchmarkDeleteWithPartitions(bench, bData, 1, nParameters[0]))
         .add("addPartition", () -> benchmarkCreatePartition(bench, bData))
-        .add("dropPartition", () -> benchmarkDropPartition(bench, bData))
+        .add("dropPartition", () -> benchmarkDropPartition(bench, bData, 1))
         .add("listPartition", () -> benchmarkListPartition(bench, bData))
         .add("getPartition",
             () -> benchmarkGetPartitions(bench, bData, 1))
@@ -310,6 +310,8 @@ public class BenchmarkTool implements Runnable {
               () -> benchmarkRenameTable(bench, bData, howMany))
           .add("dropDatabase" + '.' + howMany,
               () -> benchmarkDropDatabase(bench, bData, howMany))
+          .add("dropPartition" + '.' + howMany,
+              () -> benchmarkDropPartition(bench, bData, howMany))
           .add("openTxns" + '.' + howMany,
               () -> benchmarkOpenTxns(bench, bData, howMany))
           .add("PartitionManagementTask" + "." + howMany,
diff --git a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java
index e91ab78fbb3..3363b9d7ff7 100644
--- a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java
+++ b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java
@@ -246,26 +246,24 @@ final class HMSBenchmarks {
   }
 
   static DescriptiveStatistics benchmarkDropPartition(@NotNull MicroBenchmark bench,
-                                                      @NotNull BenchData data) {
+                                                      @NotNull BenchData data,
+                                                      int count) {
     final HMSClient client = data.getClient();
     String dbName = data.dbName;
     String tableName = data.tableName;
 
     BenchmarkUtils.createPartitionedTable(client, dbName, tableName);
-    final List<String> values = Collections.singletonList("d1");
     try {
-      Table t = client.getTable(dbName, tableName);
-      Partition partition = new Util.PartitionBuilder(t)
-          .withValues(values)
-          .build();
-
       return bench.measure(
-          () -> throwingSupplierWrapper(() -> client.addPartition(partition)),
-          () -> throwingSupplierWrapper(() -> client.dropPartition(dbName, tableName, values)),
+          () -> addManyPartitionsNoException(client, dbName, tableName, null,
+                  Collections.singletonList("d"), count),
+          () -> throwingSupplierWrapper(() -> {
+            List<String> partNames = client.getPartitionNames(dbName, tableName);
+            partNames.forEach(partName ->
+                throwingSupplierWrapper(() -> client.dropPartition(dbName, tableName, partName)));
+            return null;
+          }),
           null);
-    } catch (TException e) {
-      e.printStackTrace();
-      return new DescriptiveStatistics();
     } finally {
       throwingSupplierWrapper(() -> client.dropTable(dbName, tableName));
     }
diff --git a/standalone-metastore/metastore-tools/tools-common/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSClient.java b/standalone-metastore/metastore-tools/tools-common/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSClient.java
index 7da4ddb68b6..afe847281e4 100644
--- a/standalone-metastore/metastore-tools/tools-common/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSClient.java
+++ b/standalone-metastore/metastore-tools/tools-common/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSClient.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.metastore.tools;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
 import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsRequest;
 import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
@@ -291,6 +292,13 @@ final class HMSClient implements AutoCloseable {
     return client.drop_partition(dbName, tableName, arguments, true);
   }
 
+  public boolean dropPartition(@NotNull String dbName, @NotNull String tableName,
+                               @NotNull String arguments)
+          throws TException {
+    List<String> partVals = Warehouse.getPartValuesFromPartName(arguments);
+    return dropPartition(dbName, tableName, partVals);
+  }
+
   List<Partition> getPartitions(@NotNull String dbName, @NotNull String tableName) throws TException {
     return client.get_partitions(dbName, tableName, (short) -1);
   }