You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pv...@apache.org on 2019/09/18 12:02:02 UTC

[hive] branch master updated: HIVE-21875: Implement drop partition related methods on temporary tables (Laszlo Pinter, via Peter Vary)

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

pvary 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 c05a32c  HIVE-21875: Implement drop partition related methods on temporary tables (Laszlo Pinter, via Peter Vary)
c05a32c is described below

commit c05a32cdbf7872b2e428ccb7ae1187aee5b74d43
Author: Laszlo Pinter <lp...@cloudera.com>
AuthorDate: Wed Sep 18 14:01:10 2019 +0200

    HIVE-21875: Implement drop partition related methods on temporary tables (Laszlo Pinter, via Peter Vary)
---
 .../ql/metadata/SessionHiveMetaStoreClient.java    | 153 ++++++++++++++++++++-
 ...HiveMetastoreClientDropPartitionsTempTable.java | 137 ++++++++++++++++++
 .../hive/metastore/client/TestDropPartitions.java  |  44 ++++--
 3 files changed, 316 insertions(+), 18 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
index a2c84b4..506bf5d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
@@ -33,6 +33,8 @@ import java.util.Map.Entry;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.commons.lang3.tuple.Pair;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -43,6 +45,7 @@ import org.apache.hadoop.hive.io.HdfsUtils;
 import org.apache.hadoop.hive.metastore.HiveMetaHookLoader;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.PartitionDropOptions;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
@@ -78,9 +81,11 @@ import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-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.Warehouse.makePartName;
+import static org.apache.hadoop.hive.metastore.Warehouse.makeSpecFromName;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
 
 /**
  * todo: This need review re: thread safety.  Various places (see callsers of
@@ -922,6 +927,9 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
   public static final class TempTable {
     private final org.apache.hadoop.hive.metastore.api.Table tTable;
     private final PartitionTree pTree;
+
+    private static final String EXTERNAL_PARAM = "EXTERNAL";
+
     TempTable(org.apache.hadoop.hive.metastore.api.Table t) {
       assert t != null;
       this.tTable = t;
@@ -934,6 +942,10 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
       return partition == null ? pTree.getPartition(partName) : partition;
     }
 
+    private boolean isExternal() {
+      return tTable.getParameters() != null && "true".equals(tTable.getParameters().get(EXTERNAL_PARAM));
+    }
+
     private Partition getPartition(String partName) throws MetaException {
       if (partName == null || partName.isEmpty()) {
         throw new MetaException("Partition name cannot be null or empty");
@@ -1007,6 +1019,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
       return result;
     }
 
+
     private boolean checkPrivilegesForPartition(Partition partition, String userName, List<String> groupNames) {
       if ((userName == null || userName.isEmpty()) && (groupNames == null || groupNames.isEmpty())) {
         return true;
@@ -1033,6 +1046,28 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
       return true;
     }
 
+    private Partition dropPartition(List<String> partVals) throws MetaException, NoSuchObjectException {
+      return pTree.dropPartition(partVals);
+    }
+
+    private Partition dropPartition(String partitionName) throws MetaException, NoSuchObjectException {
+      Map<String, String> specFromName = makeSpecFromName(partitionName);
+      if (specFromName == null || specFromName.isEmpty()) {
+        throw new NoSuchObjectException("Invalid partition name " + partitionName);
+      }
+      List<String> pVals = new ArrayList<>();
+      for (FieldSchema field : tTable.getPartitionKeys()) {
+        String val = specFromName.get(field.getName());
+        if (val == null) {
+          throw new NoSuchObjectException("Partition name " + partitionName + " and table partition keys " + Arrays
+              .toString(tTable.getPartitionKeys().toArray()) + " does not match");
+        }
+        pVals.add(val);
+      }
+      return pTree.dropPartition(pVals);
+    }
+
+
     /**
      * Always clone objects before adding or returning them so that callers don't modify them
      * via references.
@@ -1132,6 +1167,21 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
       private List<Partition> listPartitions() {
         return new ArrayList<>(parts.values());
       }
+
+      /**
+       * Remove a partition from the table.
+       * @param partVals partition values, must be not null
+       * @return the instance of the dropped partition, if the remove was successful, otherwise false
+       * @throws MetaException
+       */
+      private Partition dropPartition(List<String> partVals) throws MetaException, NoSuchObjectException {
+        String partName = makePartName(tTable.getPartitionKeys(), partVals);
+        if (!parts.containsKey(partName)) {
+          throw new NoSuchObjectException(
+              "Partition with partition values " + Arrays.toString(partVals.toArray()) + " is not found.");
+        }
+        return parts.remove(partName);
+      }
     }
   }
 
@@ -1449,7 +1499,74 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     return deepCopy(partition);
   }
 
-  private  TempTable getPartitionedTempTable(org.apache.hadoop.hive.metastore.api.Table t) throws MetaException {
+  @Override
+  public boolean dropPartition(String dbName, String tableName, List<String> partVals) throws TException {
+    return dropPartition(getDefaultCatalog(conf), dbName, tableName, partVals,
+        PartitionDropOptions.instance().deleteData(true));
+  }
+
+  @Override
+  public boolean dropPartition(String catName, String dbName, String tblName, List<String> partVals,
+      PartitionDropOptions options) throws TException {
+    org.apache.hadoop.hive.metastore.api.Table table = getTempTable(dbName, tblName);
+    if (table == null) {
+      return super.dropPartition(catName, dbName, tblName, partVals, options);
+    }
+    assertTempTablePartitioned(table);
+    if (partVals == null || partVals.isEmpty() || partVals.contains(null)) {
+      throw new MetaException("Partition values cannot be null, empty or contain null values");
+    }
+    TempTable tt = getPartitionedTempTable(table);
+    if (tt == null) {
+      throw new IllegalStateException("TempTable not found for " + getCatalogQualifiedTableName(table));
+    }
+    Partition droppedPartition = tt.dropPartition(partVals);
+    boolean result = droppedPartition != null ? true : false;
+    boolean purgeData = true;
+    boolean deleteData = true;
+    if (options != null) {
+      deleteData = options.deleteData;
+      purgeData = options.purgeData;
+    }
+
+    if (deleteData && !tt.isExternal()) {
+      result &= deletePartitionLocation(droppedPartition, purgeData);
+    }
+
+    return result;
+  }
+
+  @Override
+  public boolean dropPartition(String catName, String dbName, String tableName, String partitionName,
+      boolean deleteData) throws TException {
+    org.apache.hadoop.hive.metastore.api.Table table = getTempTable(dbName, tableName);
+    if (table == null) {
+      return super.dropPartition(catName, dbName, tableName, partitionName, deleteData);
+    }
+    TempTable tt = getPartitionedTempTable(table);
+    if (tt == null) {
+      throw new IllegalStateException("TempTable not found for " + getCatalogQualifiedTableName(table));
+    }
+    Partition droppedPartition = tt.dropPartition(partitionName);
+    boolean result = droppedPartition != null ? true : false;
+    if (deleteData && !tt.isExternal()) {
+      result &= deletePartitionLocation(droppedPartition, true);
+    }
+    return result;
+  }
+
+  @Override
+  public List<Partition> dropPartitions(String catName, String dbName, String tblName,
+      List<Pair<Integer, byte[]>> partExprs, PartitionDropOptions options) throws TException {
+    org.apache.hadoop.hive.metastore.api.Table table = getTempTable(dbName, tblName);
+    if (table == null) {
+      return super.dropPartitions(catName, dbName, tblName, partExprs, options);
+    }
+    throw new UnsupportedOperationException("Dropping partitions for temporary tables, using an expression is not"
+        + "supported");
+  }
+
+  private TempTable getPartitionedTempTable(org.apache.hadoop.hive.metastore.api.Table t) throws MetaException {
     String qualifiedTableName = Warehouse.
         getQualifiedName(t.getDbName().toLowerCase(), t.getTableName().toLowerCase());
     SessionState ss = SessionState.get();
@@ -1509,8 +1626,8 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
       // 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(partitionLocation)) {
-        if (!wh.mkdirs(partitionLocation)) {
+      if (!getWh().isDir(partitionLocation)) {
+        if (!getWh().mkdirs(partitionLocation)) {
           throw new MetaException(partitionLocation
               + " is not a directory or unable to create one");
         }
@@ -1582,7 +1699,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
         throw new MetaException("Cannot specify location for a view partition");
       }
       try {
-        partLocation = wh.getDnsPath(new Path(partLocationStr));
+        partLocation = getWh().getDnsPath(new Path(partLocationStr));
       } catch (IllegalArgumentException e) {
         throw new MetaException("Partition path is invalid. " + e.getLocalizedMessage());
       }
@@ -1685,4 +1802,30 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
       throw new MetaException(getCatalogQualifiedTableName(table) + " is not partitioned");
     }
   }
+
+  /**
+   * Delete the directory where the partition resides.
+   * @param partition instance of partition, must be not null
+   * @param purgeData purge the data
+   * @return true if delete was successful
+   * @throws MetaException if delete fails
+   */
+  private boolean deletePartitionLocation(Partition partition, boolean purgeData) throws MetaException {
+    String location = partition.getSd().getLocation();
+    if (location != null) {
+      Path path = getWh().getDnsPath(new Path(location));
+      try {
+        do {
+          if (!getWh().deleteDir(path, true, purgeData, false)) {
+            throw new MetaException("Unable to delete partition at " + location);
+          }
+          path = path.getParent();
+        } while (getWh().isEmptyDir(path));
+      } catch (IOException e) {
+        throw new MetaException("Unable to delete partition at " + path.toString());
+      }
+      return true;
+    }
+    return false;
+  }
 }
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestSessionHiveMetastoreClientDropPartitionsTempTable.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestSessionHiveMetastoreClientDropPartitionsTempTable.java
new file mode 100644
index 0000000..c18dadc
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestSessionHiveMetastoreClientDropPartitionsTempTable.java
@@ -0,0 +1,137 @@
+/*
+ * 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.metadata;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.CustomIgnoreRule;
+import org.apache.hadoop.hive.metastore.client.TestDropPartitions;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Test class for delete partitions related methods on temporary tables.
+ */
+@RunWith(Parameterized.class)
+@Category(MetastoreCheckinTest.class)
+public class TestSessionHiveMetastoreClientDropPartitionsTempTable
+    extends TestDropPartitions {
+
+  private HiveConf conf;
+
+  public TestSessionHiveMetastoreClientDropPartitionsTempTable(String name, AbstractMetaStoreService metaStore) {
+    super(name, metaStore);
+    ignoreRule = new CustomIgnoreRule();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    initHiveConf();
+    SessionState.start(conf);
+    setClient(Hive.get(conf).getMSC());
+    getClient().dropDatabase(DB_NAME, true, true, true);
+    getMetaStore().cleanWarehouseDirs();
+    new DatabaseBuilder().
+        setName(DB_NAME).
+        create(getClient(), conf);
+
+    // Create test tables with 3 partitions
+    createTable(TABLE_NAME, getYearAndMonthPartCols(), null);
+    createPartitions();
+  }
+
+  private void initHiveConf() throws HiveException {
+    conf = Hive.get().getConf();
+    conf.setBoolVar(HiveConf.ConfVars.METASTORE_FASTPATH, true);
+  }
+
+  @Override
+  protected Table createTable(String tableName, List<FieldSchema> partCols, Map<String, String> tableParams)
+      throws Exception {
+    TableBuilder builder =
+        new TableBuilder().setDbName(DB_NAME).setTableName(tableName).addCol("test_id", "int", "test col id")
+            .addCol("test_value", "string", "test col value").setPartCols(partCols)
+            .setLocation(getMetaStore().getWarehouseRoot() + "/" + tableName).setTemporary(true);
+    if (tableParams != null) {
+      builder.setTableParams(tableParams);
+    }
+    return builder.create(getClient(), conf);
+  }
+
+  @Override
+  protected Partition createPartition(List<String> values, List<FieldSchema> partCols) throws Exception {
+    new PartitionBuilder().setDbName(DB_NAME).setTableName(TABLE_NAME).setValues(values).setCols(partCols)
+        .addToTable(getClient(), conf);
+    org.apache.hadoop.hive.metastore.api.Partition partition = getClient().getPartition(DB_NAME, TABLE_NAME, values);
+    return partition;
+  }
+
+  @Override
+  protected Partition createPartition(String tableName, String location, List<String> values,
+      List<FieldSchema> partCols, Map<String, String> partParams) throws Exception {
+    new PartitionBuilder().setDbName(DB_NAME).setTableName(tableName).setValues(values).setCols(partCols)
+        .setLocation(location).setPartParams(partParams).addToTable(getClient(), conf);
+    Partition partition = getClient().getPartition(DB_NAME, tableName, values);
+    return partition;
+  }
+
+  @Override
+  protected void checkPartitionsAfterDelete(String tableName, List<Partition> droppedPartitions,
+      List<Partition> existingPartitions, boolean deleteData, boolean purge) throws Exception {
+
+    List<Partition> partitions = getClient().listPartitions(DB_NAME, tableName, MAX);
+    Assert.assertEquals(
+        "The table " + tableName + " has " + partitions.size() + " partitions, but it should have " + existingPartitions
+            .size(), existingPartitions.size(), partitions.size());
+    for (Partition droppedPartition : droppedPartitions) {
+      Assert.assertFalse(partitions.contains(droppedPartition));
+      Path partitionPath = new Path(droppedPartition.getSd().getLocation());
+      if (deleteData) {
+        Assert.assertFalse("The location '" + partitionPath.toString() + "' should not exist.",
+            getMetaStore().isPathExists(partitionPath));
+      } else {
+        Assert.assertTrue("The location '" + partitionPath.toString() + "' should exist.",
+            getMetaStore().isPathExists(partitionPath));
+      }
+    }
+
+    for (Partition existingPartition : existingPartitions) {
+      Assert.assertTrue(partitions.contains(existingPartition));
+      Path partitionPath = new Path(existingPartition.getSd().getLocation());
+      Assert.assertTrue("The location '" + partitionPath.toString() + "' should exist.",
+          getMetaStore().isPathExists(partitionPath));
+    }
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestDropPartitions.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestDropPartitions.java
index 91c9eda..d496294 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestDropPartitions.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestDropPartitions.java
@@ -54,8 +54,6 @@ import org.junit.runners.Parameterized;
 
 import com.google.common.collect.Lists;
 
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
-
 /**
  * Tests for dropping partitions.
  */
@@ -65,12 +63,12 @@ public class TestDropPartitions extends MetaStoreClientTest {
   private AbstractMetaStoreService metaStore;
   private IMetaStoreClient client;
 
-  private static final String DB_NAME = "test_drop_part_db";
-  private static final String TABLE_NAME = "test_drop_part_table";
+  protected static final String DB_NAME = "test_drop_part_db";
+  protected static final String TABLE_NAME = "test_drop_part_table";
   private static final String DEFAULT_COL_TYPE = "string";
   private static final String YEAR_COL_NAME = "year";
   private static final String MONTH_COL_NAME = "month";
-  private static final short MAX = -1;
+  protected static final short MAX = -1;
   private static final Partition[] PARTITIONS = new Partition[3];
 
   @BeforeClass
@@ -101,9 +99,7 @@ public class TestDropPartitions extends MetaStoreClientTest {
 
     // Create test tables with 3 partitions
     createTable(TABLE_NAME, getYearAndMonthPartCols(), null);
-    PARTITIONS[0] = createPartition(Lists.newArrayList("2017", "march"), getYearAndMonthPartCols());
-    PARTITIONS[1] = createPartition(Lists.newArrayList("2017", "april"), getYearAndMonthPartCols());
-    PARTITIONS[2] = createPartition(Lists.newArrayList("2018", "march"), getYearAndMonthPartCols());
+    createPartitions();
   }
 
   @After
@@ -121,6 +117,18 @@ public class TestDropPartitions extends MetaStoreClientTest {
     }
   }
 
+  public AbstractMetaStoreService getMetaStore() {
+    return metaStore;
+  }
+
+  public IMetaStoreClient getClient() {
+    return client;
+  }
+
+  public void setClient(IMetaStoreClient client) {
+    this.client = client;
+  }
+
   // Tests for dropPartition(String db_name, String tbl_name, List<String> part_vals,
   // boolean deleteData) method
 
@@ -178,6 +186,7 @@ public class TestDropPartitions extends MetaStoreClientTest {
   }
 
   @Test
+  @ConditionalIgnoreOnSessionHiveMetastoreClient
   public void testDropPartitionArchivedPartition() throws Exception {
 
     String originalLocation = metaStore.getWarehouseRoot() + "/" + TABLE_NAME + "/2016_may";
@@ -501,6 +510,7 @@ public class TestDropPartitions extends MetaStoreClientTest {
   }
 
   @Test
+  @ConditionalIgnoreOnSessionHiveMetastoreClient
   public void otherCatalog() throws TException {
     String catName = "drop_partition_catalog";
     Catalog cat = new CatalogBuilder()
@@ -554,11 +564,13 @@ public class TestDropPartitions extends MetaStoreClientTest {
   }
 
   @Test(expected = NoSuchObjectException.class)
+  @ConditionalIgnoreOnSessionHiveMetastoreClient
   public void testDropPartitionBogusCatalog() throws Exception {
     client.dropPartition("nosuch", DB_NAME, TABLE_NAME, Lists.newArrayList("2017"), false);
   }
 
   @Test(expected = NoSuchObjectException.class)
+  @ConditionalIgnoreOnSessionHiveMetastoreClient
   public void testDropPartitionByNameBogusCatalog() throws Exception {
     client.dropPartition("nosuch", DB_NAME, TABLE_NAME, "year=2017", false);
   }
@@ -566,7 +578,13 @@ public class TestDropPartitions extends MetaStoreClientTest {
 
   // Helper methods
 
-  private Table createTable(String tableName, List<FieldSchema> partCols,
+  protected void createPartitions() throws Exception {
+    PARTITIONS[0] = createPartition(Lists.newArrayList("2017", "march"), getYearAndMonthPartCols());
+    PARTITIONS[1] = createPartition(Lists.newArrayList("2017", "april"), getYearAndMonthPartCols());
+    PARTITIONS[2] = createPartition(Lists.newArrayList("2018", "march"), getYearAndMonthPartCols());
+  }
+
+  protected Table createTable(String tableName, List<FieldSchema> partCols,
       Map<String, String> tableParams) throws Exception {
     String type = "MANAGED_TABLE";
     if (tableParams != null)
@@ -585,7 +603,7 @@ public class TestDropPartitions extends MetaStoreClientTest {
     return table;
   }
 
-  private Partition createPartition(List<String> values,
+  protected Partition createPartition(List<String> values,
       List<FieldSchema> partCols) throws Exception {
     new PartitionBuilder()
         .setDbName(DB_NAME)
@@ -597,7 +615,7 @@ public class TestDropPartitions extends MetaStoreClientTest {
     return partition;
   }
 
-  private Partition createPartition(String tableName, String location, List<String> values,
+  protected Partition createPartition(String tableName, String location, List<String> values,
       List<FieldSchema> partCols, Map<String, String> partParams) throws Exception {
     new PartitionBuilder()
         .setDbName(DB_NAME)
@@ -611,7 +629,7 @@ public class TestDropPartitions extends MetaStoreClientTest {
     return partition;
   }
 
-  private static List<FieldSchema> getYearAndMonthPartCols() {
+  protected static List<FieldSchema> getYearAndMonthPartCols() {
     List<FieldSchema> cols = new ArrayList<>();
     cols.add(new FieldSchema(YEAR_COL_NAME, DEFAULT_COL_TYPE, "year part col"));
     cols.add(new FieldSchema(MONTH_COL_NAME, DEFAULT_COL_TYPE, "month part col"));
@@ -624,7 +642,7 @@ public class TestDropPartitions extends MetaStoreClientTest {
     return cols;
   }
 
-  private void checkPartitionsAfterDelete(String tableName, List<Partition> droppedPartitions,
+  protected void checkPartitionsAfterDelete(String tableName, List<Partition> droppedPartitions,
       List<Partition> existingPartitions, boolean deleteData, boolean purge) throws Exception {
 
     List<Partition> partitions = client.listPartitions(DB_NAME, tableName, MAX);