You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ai...@apache.org on 2017/05/15 18:11:28 UTC

hive git commit: HIVE-16143: Improve msck repair batching (Vihang Karajgaonkar, reviewed by Sahil Takiar & Aihua Xu)

Repository: hive
Updated Branches:
  refs/heads/master 86f74fdd2 -> 0efb93681


HIVE-16143: Improve msck repair batching (Vihang Karajgaonkar, reviewed by Sahil Takiar & Aihua Xu)


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

Branch: refs/heads/master
Commit: 0efb93681099af4e4b7269c72d86fe379c99da7c
Parents: 86f74fd
Author: Aihua Xu <ai...@apache.org>
Authored: Mon May 15 14:08:32 2017 -0400
Committer: Aihua Xu <ai...@apache.org>
Committed: Mon May 15 14:08:32 2017 -0400

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  13 +-
 .../apache/hive/common/util/RetryUtilities.java | 112 ++++++
 .../hive/common/util/TestRetryUtilities.java    | 150 ++++++++
 .../test/queries/clientpositive/create_like.q   |   3 +-
 .../results/clientpositive/create_like.q.out    |  15 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   3 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  89 ++---
 .../exec/TestMsckCreatePartitionsInBatches.java | 340 +++++++++++++++++++
 .../test/queries/clientpositive/msck_repair_0.q |  10 +
 .../test/queries/clientpositive/msck_repair_1.q |   8 +
 .../test/queries/clientpositive/msck_repair_2.q |   5 +
 .../test/queries/clientpositive/msck_repair_3.q |   4 +
 .../clientpositive/msck_repair_batchsize.q      |   4 +
 .../results/clientpositive/msck_repair_0.q.out  |  38 ++-
 .../results/clientpositive/msck_repair_1.q.out  |  28 +-
 .../results/clientpositive/msck_repair_2.q.out  |  28 +-
 .../results/clientpositive/msck_repair_3.q.out  |  28 +-
 .../clientpositive/msck_repair_batchsize.q.out  |  34 +-
 ql/src/test/results/clientpositive/repair.q.out |   3 +-
 19 files changed, 857 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 6068f0d..1c37b6e 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3369,9 +3369,16 @@ public class HiveConf extends Configuration {
        " others; 'ignore' will skip the validation (legacy behavior, causes bugs in many cases)"),
     HIVE_MSCK_REPAIR_BATCH_SIZE(
         "hive.msck.repair.batch.size", 0,
-        "Batch size for the msck repair command. If the value is greater than zero, "
-            + "it will execute batch wise with the configured batch size. "
-            + "The default value is zero. Zero means it will execute directly (Not batch wise)"),
+        "Batch size for the msck repair command. If the value is greater than zero,\n "
+            + "it will execute batch wise with the configured batch size. In case of errors while\n"
+            + "adding unknown partitions the batch size is automatically reduced by half in the subsequent\n"
+            + "retry attempt. The default value is zero which means it will execute directly (not batch wise)"),
+    HIVE_MSCK_REPAIR_BATCH_MAX_RETRIES("hive.msck.repair.batch.max.retries", 0,
+        "Maximum number of retries for the msck repair command when adding unknown partitions.\n "
+        + "If the value is greater than zero it will retry adding unknown partitions until the maximum\n"
+        + "number of attempts is reached or batch size is reduced to 0, whichever is earlier.\n"
+        + "In each retry attempt it will reduce the batch size by a factor of 2 until it reaches zero.\n"
+        + "If the value is set to zero it will retry until the batch size becomes zero as described above."),
     HIVE_SERVER2_LLAP_CONCURRENT_QUERIES("hive.server2.llap.concurrent.queries", -1,
         "The number of queries allowed in parallel via llap. Negative number implies 'infinite'."),
     HIVE_TEZ_ENABLE_MEMORY_MANAGER("hive.tez.enable.memory.manager", true,

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/common/src/java/org/apache/hive/common/util/RetryUtilities.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/common/util/RetryUtilities.java b/common/src/java/org/apache/hive/common/util/RetryUtilities.java
new file mode 100644
index 0000000..3a20f2c
--- /dev/null
+++ b/common/src/java/org/apache/hive/common/util/RetryUtilities.java
@@ -0,0 +1,112 @@
+/**
+ * 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.hive.common.util;
+
+import java.util.concurrent.Callable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RetryUtilities {
+  public static class RetryException extends Exception {
+    private static final long serialVersionUID = 1L;
+
+    public RetryException(Exception ex) {
+      super(ex);
+    }
+
+    public RetryException(String msg) {
+      super(msg);
+    }
+  }
+
+  /**
+   * Interface used to create a ExponentialBackOffRetry policy
+   */
+  public static interface ExponentialBackOffRetry<T> {
+    /**
+     * This method should be called by implementations of this ExponentialBackOffRetry policy
+     * It represents the actual work which needs to be done based on a given batch size
+     * @param batchSize The batch size for the work which needs to be executed
+     * @return
+     * @throws Exception
+     */
+    public T execute(int batchSize) throws Exception;
+  }
+
+  /**
+   * This class is a base implementation of a simple exponential back retry policy. The batch size
+   * and decaying factor are provided with the constructor. It reduces the batch size by dividing
+   * it by the decaying factor every time there is an exception in the execute method.
+   */
+  public static abstract class ExponentiallyDecayingBatchWork<T>
+      implements ExponentialBackOffRetry<T> {
+    private int batchSize;
+    private final int decayingFactor;
+    private int maxRetries;
+    private static final Logger LOG = LoggerFactory.getLogger(ExponentiallyDecayingBatchWork.class);
+
+    public ExponentiallyDecayingBatchWork(int batchSize, int reducingFactor, int maxRetries) {
+      if (batchSize <= 0) {
+        throw new IllegalArgumentException(String.format(
+            "Invalid batch size %d provided. Batch size must be greater than 0", batchSize));
+      }
+      this.batchSize = batchSize;
+      if (reducingFactor <= 1) {
+        throw new IllegalArgumentException(String.format(
+            "Invalid decaying factor %d provided. Decaying factor must be greater than 1",
+            batchSize));
+      }
+      if (maxRetries < 0) {
+        throw new IllegalArgumentException(String.format(
+            "Invalid number of maximum retries %d provided. It must be a non-negative integer value",
+            maxRetries));
+      }
+      //if maxRetries is 0 code retries until batch decays to zero
+      this.maxRetries = maxRetries;
+      this.decayingFactor = reducingFactor;
+    }
+
+    public T run() throws Exception {
+      int attempt = 0;
+      while (true) {
+        int size = getNextBatchSize();
+        if (size == 0) {
+          throw new RetryException("Batch size reduced to zero");
+        }
+        try {
+          return execute(size);
+        } catch (Exception ex) {
+          LOG.warn(String.format("Exception thrown while processing using a batch size %d", size),
+              ex);
+        } finally {
+          attempt++;
+          if (attempt == maxRetries) {
+            throw new RetryException(String.format("Maximum number of retry attempts %d exhausted", maxRetries));
+          }
+        }
+      }
+    }
+
+    private int getNextBatchSize() {
+      int ret = batchSize;
+      batchSize /= decayingFactor;
+      return ret;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/common/src/test/org/apache/hive/common/util/TestRetryUtilities.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hive/common/util/TestRetryUtilities.java b/common/src/test/org/apache/hive/common/util/TestRetryUtilities.java
new file mode 100644
index 0000000..4187be2
--- /dev/null
+++ b/common/src/test/org/apache/hive/common/util/TestRetryUtilities.java
@@ -0,0 +1,150 @@
+/**
+ * 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.hive.common.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hive.common.util.RetryUtilities.ExponentiallyDecayingBatchWork;
+import org.apache.hive.common.util.RetryUtilities.RetryException;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestRetryUtilities {
+
+  private class DummyExponentiallyDecayingBatchWork extends ExponentiallyDecayingBatchWork<Void> {
+    public DummyExponentiallyDecayingBatchWork(int batchSize, int reducingFactor,
+        int throwException) {
+      super(batchSize, reducingFactor, 0);
+      this.exceptionCount = throwException;
+    }
+
+    public DummyExponentiallyDecayingBatchWork(int batchSize, int reducingFactor,
+        int throwException, int maxRetries) {
+      super(batchSize, reducingFactor, maxRetries);
+      this.exceptionCount = throwException;
+    }
+
+    final List<Integer> batchSizes = new ArrayList<>();
+    int exceptionCount = 0;
+
+    @Override
+    public Void execute(int size) throws Exception {
+      batchSizes.add(size);
+      if (exceptionCount > 0) {
+        exceptionCount--;
+        throw new Exception("Dummy exception");
+      }
+      return null;
+    }
+
+    public int getCount() {
+      return batchSizes.size();
+    }
+
+    public int[] getBatchSizes() {
+      int[] ret = new int[batchSizes.size()];
+      int i = 0;
+      for (int b : batchSizes) {
+        ret[i++] = b;
+      }
+      return ret;
+    }
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testZeroBatchSize() {
+    new DummyExponentiallyDecayingBatchWork(0, 2, 0);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testNegativeBatchSize() {
+    new DummyExponentiallyDecayingBatchWork(-1, 2, 0);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testZeroDecayingFactor() {
+    new DummyExponentiallyDecayingBatchWork(5, 0, 0);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testOneDecayingFactor() {
+    new DummyExponentiallyDecayingBatchWork(10, 1, 0);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testNegativeMaxRetries() {
+    new DummyExponentiallyDecayingBatchWork(10, 2, 0, -1);
+  }
+
+  @Test
+  public void testNumberOfAttempts() throws Exception {
+    // test perfectly divisible batchsize and decaying factor
+    DummyExponentiallyDecayingBatchWork dummy = new DummyExponentiallyDecayingBatchWork(10, 2, 0);
+    dummy.run();
+    Assert.assertEquals("Unexpected number of executions of execute method", 1, dummy.getCount());
+    // there were no exception. Batchsize doesn't change until there is an exception
+    Assert.assertArrayEquals(new int[] { 10 }, dummy.getBatchSizes());
+    // test batchsize is not divisible by decaying factor
+    dummy = new DummyExponentiallyDecayingBatchWork(11, 2, 0);
+    dummy.run();
+    Assert.assertEquals("Unexpected number of executions of execute method", 1, dummy.getCount());
+    // there were no exception. Batchsize doesn't change until there is an exception
+    Assert.assertArrayEquals(new int[] { 11 }, dummy.getBatchSizes());
+
+    dummy = new DummyExponentiallyDecayingBatchWork(11, 3, 1);
+    // batches will be sized 11,3
+    dummy.run();
+    Assert.assertEquals("Unexpected number of executions of execute method", 2, dummy.getCount());
+    Assert.assertArrayEquals(new int[] { 11, 3 }, dummy.getBatchSizes());
+
+    dummy = new DummyExponentiallyDecayingBatchWork(11, 3, 2);
+    // batches will be sized 11,3,1
+    dummy.run();
+    Assert.assertEquals("Unexpected number of executions of execute method", 3, dummy.getCount());
+    Assert.assertArrayEquals(new int[] { 11, 3, 1 }, dummy.getBatchSizes());
+
+    dummy = new DummyExponentiallyDecayingBatchWork(12, 3, 2);
+    // batches will be sized 12,4,1
+    dummy.run();
+    Assert.assertEquals("Unexpected number of executions of execute method", 3, dummy.getCount());
+    Assert.assertArrayEquals(new int[] { 12, 4, 1 }, dummy.getBatchSizes());
+  }
+
+  @Test
+  public void testZeroMaxRetriesValue() throws Exception {
+    DummyExponentiallyDecayingBatchWork dummy = new DummyExponentiallyDecayingBatchWork(10, 2, 3, 0);
+    dummy.run();
+    // batches will be sized 10, 5, 2, 1
+    Assert.assertEquals("Unexpected number of executions of execute method", 4, dummy.getCount());
+    Assert.assertArrayEquals(new int[] { 10, 5, 2, 1 }, dummy.getBatchSizes());
+
+    dummy = new DummyExponentiallyDecayingBatchWork(17, 2, 4, 0);
+    // batches will be sized 17, 8, 4, 2, 1
+    dummy.run();
+    Assert.assertEquals("Unexpected number of executions of execute method", 5, dummy.getCount());
+    Assert.assertArrayEquals(new int[] { 17, 8, 4, 2, 1 }, dummy.getBatchSizes());
+  }
+
+  @Test(expected = RetryException.class)
+  public void testRetriesExhausted() throws Exception {
+    // attempts at execute will be made using batchsizes 11, 3, 1, throws retry exception
+    DummyExponentiallyDecayingBatchWork dummy = new DummyExponentiallyDecayingBatchWork(11, 3, 3);
+    dummy.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/itests/hive-blobstore/src/test/queries/clientpositive/create_like.q
----------------------------------------------------------------------
diff --git a/itests/hive-blobstore/src/test/queries/clientpositive/create_like.q b/itests/hive-blobstore/src/test/queries/clientpositive/create_like.q
index 38f384e..63ea3f6 100644
--- a/itests/hive-blobstore/src/test/queries/clientpositive/create_like.q
+++ b/itests/hive-blobstore/src/test/queries/clientpositive/create_like.q
@@ -21,10 +21,11 @@ DROP TABLE like_table;
 CREATE EXTERNAL TABLE like_table LIKE blobstore_partitioned_source_table LOCATION '${hiveconf:test.blobstore.path.unique}/create_like/data';
 
 MSCK REPAIR TABLE like_table;
+show partitions like_table;
 
 SELECT * FROM blobstore_partitioned_source_table;
 SELECT * FROM like_table;
 
 DROP TABLE like_table;
 
-SELECT * FROM blobstore_partitioned_source_table;
\ No newline at end of file
+SELECT * FROM blobstore_partitioned_source_table;

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/itests/hive-blobstore/src/test/results/clientpositive/create_like.q.out
----------------------------------------------------------------------
diff --git a/itests/hive-blobstore/src/test/results/clientpositive/create_like.q.out b/itests/hive-blobstore/src/test/results/clientpositive/create_like.q.out
index 0d362a7..8dcbb93 100644
--- a/itests/hive-blobstore/src/test/results/clientpositive/create_like.q.out
+++ b/itests/hive-blobstore/src/test/results/clientpositive/create_like.q.out
@@ -91,10 +91,17 @@ POSTHOOK: query: MSCK REPAIR TABLE like_table
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@like_table
 Partitions not in metastore:	like_table:dt=20110924/hour=1	like_table:dt=20110924/hour=2	like_table:dt=20110925/hour=1	like_table:dt=20110925/hour=2
-Repair: Added partition to metastore like_table:dt=20110924/hour=1
-Repair: Added partition to metastore like_table:dt=20110924/hour=2
-Repair: Added partition to metastore like_table:dt=20110925/hour=1
-Repair: Added partition to metastore like_table:dt=20110925/hour=2
+#### A masked pattern was here ####
+PREHOOK: query: show partitions like_table
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@like_table
+POSTHOOK: query: show partitions like_table
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@like_table
+dt=20110924/hour=1
+dt=20110924/hour=2
+dt=20110925/hour=1
+dt=20110925/hour=2
 PREHOOK: query: SELECT * FROM blobstore_partitioned_source_table
 PREHOOK: type: QUERY
 PREHOOK: Input: default@blobstore_partitioned_source_table

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index d408321..d296851 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -1699,7 +1699,8 @@ public class QTestUtil {
       ".*at com\\.sun\\.proxy.*",
       ".*at com\\.jolbox.*",
       ".*at com\\.zaxxer.*",
-      "org\\.apache\\.hadoop\\.hive\\.metastore\\.model\\.MConstraint@([0-9]|[a-z])*"
+      "org\\.apache\\.hadoop\\.hive\\.metastore\\.model\\.MConstraint@([0-9]|[a-z])*",
+      "^Repair: Added partition to metastore.*"
   });
 
   private final Pattern[] partialReservedPlanMask = toPattern(new String[] {

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index b07d6b1..44655af 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -249,10 +249,12 @@ import org.apache.hadoop.util.ToolRunner;
 import org.apache.hive.common.util.AnnotationUtils;
 import org.apache.hive.common.util.HiveStringUtils;
 import org.apache.hive.common.util.ReflectionUtil;
+import org.apache.hive.common.util.RetryUtilities;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.stringtemplate.v4.ST;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 
 /**
@@ -1848,19 +1850,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     }
   }
 
-  private void msckAddPartitionsOneByOne(Hive db, Table table,
-      Set<CheckResult.PartitionResult> partsNotInMs, List<String> repairOutput) {
-    for (CheckResult.PartitionResult part : partsNotInMs) {
-      try {
-        db.createPartition(table, Warehouse.makeSpecFromName(part.getPartitionName()));
-        repairOutput.add("Repair: Added partition to metastore "
-            + table.getTableName() + ':' + part.getPartitionName());
-      } catch (Exception e) {
-        LOG.warn("Repair error, could not add partition to metastore: ", e);
-      }
-    }
-  }
-
   private int compact(Hive db, AlterTableSimpleDesc desc) throws HiveException {
 
     Table tbl = db.getTable(desc.getTableName());
@@ -1988,34 +1977,18 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
           }
         }
         Table table = db.getTable(msckDesc.getTableName());
-        AddPartitionDesc apd = new AddPartitionDesc(
-            table.getDbName(), table.getTableName(), false);
+        int batchSize = conf.getIntVar(ConfVars.HIVE_MSCK_REPAIR_BATCH_SIZE);
+        int maxRetries = conf.getIntVar(ConfVars.HIVE_MSCK_REPAIR_BATCH_MAX_RETRIES);
+        int decayingFactor = 2;
+        if (batchSize == 0) {
+          //batching is not enabled. Try to add all the partitions in one call
+          batchSize = partsNotInMs.size();
+        }
         try {
-          int batch_size = conf.getIntVar(ConfVars.HIVE_MSCK_REPAIR_BATCH_SIZE);
-          if (batch_size > 0 && partsNotInMs.size() > batch_size) {
-            int counter = 0;
-            for (CheckResult.PartitionResult part : partsNotInMs) {
-              counter++;
-              apd.addPartition(Warehouse.makeSpecFromName(part.getPartitionName()), null);
-              repairOutput.add("Repair: Added partition to metastore " + msckDesc.getTableName()
-                  + ':' + part.getPartitionName());
-              if (counter % batch_size == 0 || counter == partsNotInMs.size()) {
-                db.createPartitions(apd);
-                apd = new AddPartitionDesc(table.getDbName(), table.getTableName(), false);
-              }
-            }
-          } else {
-            for (CheckResult.PartitionResult part : partsNotInMs) {
-              apd.addPartition(Warehouse.makeSpecFromName(part.getPartitionName()), null);
-              repairOutput.add("Repair: Added partition to metastore " + msckDesc.getTableName()
-                  + ':' + part.getPartitionName());
-            }
-            db.createPartitions(apd);
-          }
+          createPartitionsInBatches(db, repairOutput, partsNotInMs, table, batchSize,
+              decayingFactor, maxRetries);
         } catch (Exception e) {
-          LOG.info("Could not bulk-add partitions to metastore; trying one by one", e);
-          repairOutput.clear();
-          msckAddPartitionsOneByOne(db, table, partsNotInMs, repairOutput);
+          throw new HiveException(e);
         }
       }
     } catch (HiveException e) {
@@ -2067,6 +2040,44 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
+  @VisibleForTesting
+  void createPartitionsInBatches(Hive db, List<String> repairOutput,
+      Set<CheckResult.PartitionResult> partsNotInMs, Table table, int batchSize, int decayingFactor, int maxRetries)
+      throws Exception {
+    String addMsgFormat = "Repair: Added partition to metastore "
+        + table.getTableName() + ":%s";
+    Set<CheckResult.PartitionResult> batchWork = new HashSet<>(partsNotInMs);
+    new RetryUtilities.ExponentiallyDecayingBatchWork<Void>(batchSize, decayingFactor, maxRetries) {
+      @Override
+      public Void execute(int size) throws Exception {
+        while (!batchWork.isEmpty()) {
+          //get the current batch size
+          int currentBatchSize = size;
+          AddPartitionDesc apd =
+              new AddPartitionDesc(table.getDbName(), table.getTableName(), true);
+          //store the partitions temporarily until processed
+          List<CheckResult.PartitionResult> lastBatch = new ArrayList<>(currentBatchSize);
+          List<String> addMsgs = new ArrayList<>(currentBatchSize);
+          //add the number of partitions given by the current batchsize
+          for (CheckResult.PartitionResult part : batchWork) {
+            if (currentBatchSize == 0) {
+              break;
+            }
+            apd.addPartition(Warehouse.makeSpecFromName(part.getPartitionName()), null);
+            lastBatch.add(part);
+            addMsgs.add(String.format(addMsgFormat, part.getPartitionName()));
+            currentBatchSize--;
+          }
+          db.createPartitions(apd);
+          // if last batch is successful remove it from partsNotInMs
+          batchWork.removeAll(lastBatch);
+          repairOutput.addAll(addMsgs);
+        }
+        return null;
+      }
+    }.run();
+  }
+
   /**
    * Write the result of msck to a writer.
    *

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/test/org/apache/hadoop/hive/ql/exec/TestMsckCreatePartitionsInBatches.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestMsckCreatePartitionsInBatches.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestMsckCreatePartitionsInBatches.java
new file mode 100644
index 0000000..2454afb
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestMsckCreatePartitionsInBatches.java
@@ -0,0 +1,340 @@
+/**
+ * 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.exec;
+
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
+import org.apache.hadoop.hive.ql.metadata.CheckResult.PartitionResult;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.plan.AddPartitionDesc;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.RetryUtilities.RetryException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+public class TestMsckCreatePartitionsInBatches {
+  private static HiveConf hiveConf;
+  private static DDLTask ddlTask;
+  private final String tableName = "test_msck_batch";
+  private static Hive db;
+  private List<String> repairOutput;
+  private Table table;
+
+  @BeforeClass
+  public static void setupClass() throws HiveException {
+    hiveConf = new HiveConf(TestMsckCreatePartitionsInBatches.class);
+    hiveConf.setIntVar(ConfVars.HIVE_MSCK_REPAIR_BATCH_SIZE, 5);
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+        "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
+    SessionState.start(hiveConf);
+    db = Hive.get(hiveConf);
+    ddlTask = new DDLTask();
+  }
+
+  @Before
+  public void before() throws Exception {
+    createPartitionedTable("default", tableName);
+    table = db.getTable(tableName);
+    repairOutput = new ArrayList<String>();
+  }
+
+  @After
+  public void after() throws Exception {
+    cleanUpTableQuietly("default", tableName);
+  }
+
+  private Table createPartitionedTable(String dbName, String tableName) throws Exception {
+    try {
+      db.dropTable(dbName, tableName);
+      db.createTable(tableName, Arrays.asList("key", "value"), // Data columns.
+          Arrays.asList("city"), // Partition columns.
+          TextInputFormat.class, HiveIgnoreKeyTextOutputFormat.class);
+      return db.getTable(dbName, tableName);
+    } catch (Exception exception) {
+      fail("Unable to drop and create table " + dbName + "." + tableName + " because "
+          + StringUtils.stringifyException(exception));
+      throw exception;
+    }
+  }
+
+  private void cleanUpTableQuietly(String dbName, String tableName) {
+    try {
+      db.dropTable(dbName, tableName, true, true, true);
+    } catch (Exception exception) {
+      fail("Unexpected exception: " + StringUtils.stringifyException(exception));
+    }
+  }
+
+  private Set<PartitionResult> createPartsNotInMs(int numOfParts) {
+    Set<PartitionResult> partsNotInMs = new HashSet<>();
+    for (int i = 0; i < numOfParts; i++) {
+      PartitionResult result = new PartitionResult();
+      result.setPartitionName("city=dummyCity_" + String.valueOf(i));
+      partsNotInMs.add(result);
+    }
+    return partsNotInMs;
+  }
+
+  /**
+   * Tests the number of times Hive.createPartitions calls are executed with total number of
+   * partitions to be added are equally divisible by batch size
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testNumberOfCreatePartitionCalls() throws Exception {
+    // create 10 dummy partitions
+    Set<PartitionResult> partsNotInMs = createPartsNotInMs(10);
+    Hive spyDb = Mockito.spy(db);
+    // batch size of 5 and decaying factor of 2
+    ddlTask.createPartitionsInBatches(spyDb, repairOutput, partsNotInMs, table, 5, 2, 0);
+    // there should be 2 calls to create partitions with each batch size of 5
+    ArgumentCaptor<AddPartitionDesc> argument = ArgumentCaptor.forClass(AddPartitionDesc.class);
+    Mockito.verify(spyDb, Mockito.times(2)).createPartitions(argument.capture());
+    // confirm the batch sizes were 5, 5 in the two calls to create partitions
+    List<AddPartitionDesc> apds = argument.getAllValues();
+    int retryAttempt = 1;
+    Assert.assertEquals(String.format("Unexpected batch size in retry attempt %d ", retryAttempt++),
+        5, apds.get(0).getPartitionCount());
+    Assert.assertEquals(String.format("Unexpected batch size in retry attempt %d ", retryAttempt++),
+        5, apds.get(1).getPartitionCount());
+  }
+
+  /**
+   * Tests the number of times Hive.createPartitions calls are executed with total number of
+   * partitions to be added are not exactly divisible by batch size
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testUnevenNumberOfCreatePartitionCalls() throws Exception {
+    // create 9 dummy partitions
+    Set<PartitionResult> partsNotInMs = createPartsNotInMs(9);
+    Hive spyDb = Mockito.spy(db);
+    // batch size of 5 and decaying factor of 2
+    ddlTask.createPartitionsInBatches(spyDb, repairOutput, partsNotInMs, table, 5, 2, 0);
+    // there should be 2 calls to create partitions with batch sizes of 5, 4
+    ArgumentCaptor<AddPartitionDesc> argument = ArgumentCaptor.forClass(AddPartitionDesc.class);
+    Mockito.verify(spyDb, Mockito.times(2)).createPartitions(argument.capture());
+    // confirm the batch sizes were 5, 4 in the two calls to create partitions
+    List<AddPartitionDesc> apds = argument.getAllValues();
+    int retryAttempt = 1;
+    Assert.assertEquals(String.format("Unexpected batch size in retry attempt %d ", retryAttempt++),
+        5, apds.get(0).getPartitionCount());
+    Assert.assertEquals(String.format("Unexpected batch size in retry attempt %d ", retryAttempt++),
+        4, apds.get(1).getPartitionCount());
+  }
+
+  /**
+   * Tests the number of times Hive.createPartitions calls are executed with total number of
+   * partitions exactly equal to batch size
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testEqualNumberOfPartitions() throws Exception {
+    // create 13 dummy partitions
+    Set<PartitionResult> partsNotInMs = createPartsNotInMs(13);
+    Hive spyDb = Mockito.spy(db);
+    // batch size of 13 and decaying factor of 2
+    ddlTask.createPartitionsInBatches(spyDb, repairOutput, partsNotInMs, table, 13, 2, 0);
+    // there should be 1 call to create partitions with batch sizes of 13
+    ArgumentCaptor<AddPartitionDesc> argument = ArgumentCaptor.forClass(AddPartitionDesc.class);
+    Mockito.verify(spyDb, Mockito.times(1)).createPartitions(argument.capture());
+    Assert.assertEquals("Unexpected number of batch size", 13,
+        argument.getValue().getPartitionCount());
+  }
+
+  /**
+   * Tests the number of times Hive.createPartitions calls are executed with total number of
+   * partitions to is less than batch size
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testSmallNumberOfPartitions() throws Exception {
+    // create 10 dummy partitions
+    Set<PartitionResult> partsNotInMs = createPartsNotInMs(10);
+    Hive spyDb = Mockito.spy(db);
+    // batch size of 20 and decaying factor of 2
+    ddlTask.createPartitionsInBatches(spyDb, repairOutput, partsNotInMs, table, 20, 2, 0);
+    // there should be 1 call to create partitions with batch sizes of 10
+    Mockito.verify(spyDb, Mockito.times(1)).createPartitions(Mockito.anyObject());
+    ArgumentCaptor<AddPartitionDesc> argument = ArgumentCaptor.forClass(AddPartitionDesc.class);
+    Mockito.verify(spyDb).createPartitions(argument.capture());
+    Assert.assertEquals("Unexpected number of batch size", 10,
+        argument.getValue().getPartitionCount());
+  }
+
+  /**
+   * Tests the number of calls to createPartitions and the respective batch sizes when first call to
+   * createPartitions throws HiveException. The batch size should be reduced by the decayingFactor
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testBatchingWhenException() throws Exception {
+    // create 13 dummy partitions
+    Set<PartitionResult> partsNotInMs = createPartsNotInMs(23);
+    Hive spyDb = Mockito.spy(db);
+    // first call to createPartitions should throw exception
+    Mockito.doThrow(HiveException.class).doCallRealMethod().doCallRealMethod().when(spyDb)
+        .createPartitions(Mockito.any(AddPartitionDesc.class));
+
+    // test with a batch size of 30 and decaying factor of 2
+    ddlTask.createPartitionsInBatches(spyDb, repairOutput, partsNotInMs, table, 30, 2, 0);
+    // confirm the batch sizes were 23, 15, 8 in the three calls to create partitions
+    ArgumentCaptor<AddPartitionDesc> argument = ArgumentCaptor.forClass(AddPartitionDesc.class);
+    // there should be 3 calls to create partitions with batch sizes of 23, 15, 8
+    Mockito.verify(spyDb, Mockito.times(3)).createPartitions(argument.capture());
+    List<AddPartitionDesc> apds = argument.getAllValues();
+    int retryAttempt = 1;
+    Assert.assertEquals(
+        String.format("Unexpected batch size in retry attempt %d ", retryAttempt++), 23,
+        apds.get(0).getPartitionCount());
+    Assert.assertEquals(
+        String.format("Unexpected batch size in retry attempt %d ", retryAttempt++), 15,
+        apds.get(1).getPartitionCount());
+    Assert.assertEquals(
+        String.format("Unexpected batch size in retry attempt %d ", retryAttempt++), 8,
+        apds.get(2).getPartitionCount());
+  }
+
+  /**
+   * Tests the retries exhausted case when Hive.createPartitions method call always keep throwing
+   * HiveException. The batch sizes should exponentially decreased based on the decaying factor and
+   * ultimately give up when it reaches 0
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testRetriesExhaustedBatchSize() throws Exception {
+    Set<PartitionResult> partsNotInMs = createPartsNotInMs(17);
+    Hive spyDb = Mockito.spy(db);
+    Mockito.doThrow(HiveException.class).when(spyDb)
+        .createPartitions(Mockito.any(AddPartitionDesc.class));
+    // batch size of 5 and decaying factor of 2
+    Exception ex = null;
+    try {
+      ddlTask.createPartitionsInBatches(spyDb, repairOutput, partsNotInMs, table, 30, 2, 0);
+    } catch (Exception retryEx) {
+      ex = retryEx;
+    }
+    Assert.assertFalse("Exception was expected but was not thrown", ex == null);
+    Assert.assertTrue("Unexpected class of exception thrown", ex instanceof RetryException);
+    // there should be 5 calls to create partitions with batch sizes of 17, 15, 7, 3, 1
+    ArgumentCaptor<AddPartitionDesc> argument = ArgumentCaptor.forClass(AddPartitionDesc.class);
+    Mockito.verify(spyDb, Mockito.times(5)).createPartitions(argument.capture());
+    List<AddPartitionDesc> apds = argument.getAllValues();
+    int retryAttempt = 1;
+    Assert.assertEquals(
+        String.format("Unexpected batch size in retry attempt %d ", retryAttempt++), 17,
+        apds.get(0).getPartitionCount());
+    Assert.assertEquals(
+        String.format("Unexpected batch size in retry attempt %d ", retryAttempt++), 15,
+        apds.get(1).getPartitionCount());
+    Assert.assertEquals(
+        String.format("Unexpected batch size in retry attempt %d ", retryAttempt++), 7,
+        apds.get(2).getPartitionCount());
+    Assert.assertEquals(
+        String.format("Unexpected batch size in retry attempt %d ", retryAttempt++), 3,
+        apds.get(3).getPartitionCount());
+    Assert.assertEquals(
+        String.format("Unexpected batch size in retry attempt %d ", retryAttempt++), 1,
+        apds.get(4).getPartitionCount());
+  }
+
+  /**
+   * Tests the maximum retry attempts provided by configuration
+   * @throws Exception
+   */
+  @Test
+  public void testMaxRetriesReached() throws Exception {
+    Set<PartitionResult> partsNotInMs = createPartsNotInMs(17);
+    Hive spyDb = Mockito.spy(db);
+    Mockito.doThrow(HiveException.class).when(spyDb)
+        .createPartitions(Mockito.any(AddPartitionDesc.class));
+    // batch size of 5 and decaying factor of 2
+    Exception ex = null;
+    try {
+      ddlTask.createPartitionsInBatches(spyDb, repairOutput, partsNotInMs, table, 30, 2, 2);
+    } catch (Exception retryEx) {
+      ex = retryEx;
+    }
+    Assert.assertFalse("Exception was expected but was not thrown", ex == null);
+    Assert.assertTrue("Unexpected class of exception thrown", ex instanceof RetryException);
+    ArgumentCaptor<AddPartitionDesc> argument = ArgumentCaptor.forClass(AddPartitionDesc.class);
+    Mockito.verify(spyDb, Mockito.times(2)).createPartitions(argument.capture());
+    List<AddPartitionDesc> apds = argument.getAllValues();
+    int retryAttempt = 1;
+    Assert.assertEquals(
+        String.format("Unexpected batch size in retry attempt %d ", retryAttempt++), 17,
+        apds.get(0).getPartitionCount());
+    Assert.assertEquals(
+        String.format("Unexpected batch size in retry attempt %d ", retryAttempt++), 15,
+        apds.get(1).getPartitionCount());
+  }
+
+  /**
+   * Tests when max number of retries is set to 1. In this case the number of retries should
+   * be specified
+   * @throws Exception
+   */
+  @Test
+  public void testOneMaxRetries() throws Exception {
+    Set<PartitionResult> partsNotInMs = createPartsNotInMs(17);
+    Hive spyDb = Mockito.spy(db);
+    Mockito.doThrow(HiveException.class).when(spyDb)
+        .createPartitions(Mockito.any(AddPartitionDesc.class));
+    // batch size of 5 and decaying factor of 2
+    Exception ex = null;
+    try {
+      ddlTask.createPartitionsInBatches(spyDb, repairOutput, partsNotInMs, table, 30, 2, 1);
+    } catch (Exception retryEx) {
+      ex = retryEx;
+    }
+    Assert.assertFalse("Exception was expected but was not thrown", ex == null);
+    Assert.assertTrue("Unexpected class of exception thrown", ex instanceof RetryException);
+    // there should be 5 calls to create partitions with batch sizes of 17, 15, 7, 3, 1
+    ArgumentCaptor<AddPartitionDesc> argument = ArgumentCaptor.forClass(AddPartitionDesc.class);
+    Mockito.verify(spyDb, Mockito.times(1)).createPartitions(argument.capture());
+    List<AddPartitionDesc> apds = argument.getAllValues();
+    int retryAttempt = 1;
+    Assert.assertEquals(
+        String.format("Unexpected batch size in retry attempt %d ", retryAttempt++), 17,
+        apds.get(0).getPartitionCount());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/test/queries/clientpositive/msck_repair_0.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/msck_repair_0.q b/ql/src/test/queries/clientpositive/msck_repair_0.q
index 2254233..cb291fe 100644
--- a/ql/src/test/queries/clientpositive/msck_repair_0.q
+++ b/ql/src/test/queries/clientpositive/msck_repair_0.q
@@ -7,15 +7,23 @@ CREATE TABLE repairtable(col STRING) PARTITIONED BY (p1 STRING, p2 STRING);
 
 MSCK TABLE repairtable;
 
+show partitions repairtable;
+
 dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/repairtable/p1=c/p2=a/p3=b;
 dfs -touchz ${system:test.warehouse.dir}/repairtable/p1=c/p2=a/p3=b/datafile;
 
 MSCK TABLE default.repairtable;
 
+show partitions default.repairtable;
+
 MSCK REPAIR TABLE default.repairtable;
 
+show partitions default.repairtable;
+
 MSCK TABLE repairtable;
 
+show partitions repairtable;
+
 set hive.mapred.mode=strict;
 
 dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/repairtable/p1=e/p2=f/p3=g;
@@ -23,4 +31,6 @@ dfs -touchz ${system:test.warehouse.dir}/repairtable/p1=e/p2=f/p3=g/datafile;
 
 MSCK REPAIR TABLE default.repairtable;
 
+show partitions default.repairtable;
+
 DROP TABLE default.repairtable;

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/test/queries/clientpositive/msck_repair_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/msck_repair_1.q b/ql/src/test/queries/clientpositive/msck_repair_1.q
index ea596cb..21aca3b 100644
--- a/ql/src/test/queries/clientpositive/msck_repair_1.q
+++ b/ql/src/test/queries/clientpositive/msck_repair_1.q
@@ -6,13 +6,21 @@ CREATE TABLE repairtable(col STRING) PARTITIONED BY (p1 STRING, p2 STRING);
 
 MSCK TABLE repairtable;
 
+SHOW PARTITIONS repairtable;
+
 dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/repairtable/p1=c/p2=a/p3=b;
 dfs -touchz ${system:test.warehouse.dir}/repairtable/p1=c/p2=a/p3=b/datafile;
 
 MSCK TABLE default.repairtable;
 
+SHOW PARTITIONS default.repairtable;
+
 MSCK REPAIR TABLE default.repairtable;
 
+SHOW PARTITIONS default.repairtable;
+
 MSCK TABLE repairtable;
 
+SHOW PARTITIONS repairtable;
+
 DROP TABLE default.repairtable;

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/test/queries/clientpositive/msck_repair_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/msck_repair_2.q b/ql/src/test/queries/clientpositive/msck_repair_2.q
index d833821..77785e9 100644
--- a/ql/src/test/queries/clientpositive/msck_repair_2.q
+++ b/ql/src/test/queries/clientpositive/msck_repair_2.q
@@ -7,14 +7,19 @@ CREATE TABLE repairtable(col STRING) PARTITIONED BY (p1 STRING, p2 STRING);
 
 MSCK TABLE repairtable;
 
+show partitions repairtable;
+
 dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/repairtable/p1=c/p2=a/p3=b;
 dfs -touchz ${system:test.warehouse.dir}/repairtable/p1=c/p2=a/p3=b/datafile;
 dfs -touchz ${system:test.warehouse.dir}/repairtable/p1=c/datafile;
 
 MSCK TABLE default.repairtable;
+show partitions repairtable;
 
 MSCK REPAIR TABLE default.repairtable;
+show partitions repairtable;
 
 MSCK TABLE repairtable;
+show partitions repairtable;
 
 DROP TABLE default.repairtable;

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/test/queries/clientpositive/msck_repair_3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/msck_repair_3.q b/ql/src/test/queries/clientpositive/msck_repair_3.q
index fdefca1..f42443f 100644
--- a/ql/src/test/queries/clientpositive/msck_repair_3.q
+++ b/ql/src/test/queries/clientpositive/msck_repair_3.q
@@ -5,13 +5,17 @@ DROP TABLE IF EXISTS repairtable;
 CREATE TABLE repairtable(col STRING) PARTITIONED BY (p1 STRING, p2 STRING);
 
 MSCK TABLE repairtable;
+show partitions repairtable;
 
 dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/repairtable/p1=c/p2=a/p3=b;
 
 MSCK TABLE default.repairtable;
+show partitions repairtable;
 
 MSCK REPAIR TABLE default.repairtable;
+show partitions repairtable;
 
 MSCK TABLE repairtable;
+show partitions repairtable;
 
 DROP TABLE default.repairtable;

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/test/queries/clientpositive/msck_repair_batchsize.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/msck_repair_batchsize.q b/ql/src/test/queries/clientpositive/msck_repair_batchsize.q
index e56e97a..a44c00e 100644
--- a/ql/src/test/queries/clientpositive/msck_repair_batchsize.q
+++ b/ql/src/test/queries/clientpositive/msck_repair_batchsize.q
@@ -14,10 +14,13 @@ dfs -touchz ${system:test.warehouse.dir}/repairtable/p1=b/p2=a/datafile;
 dfs -touchz ${system:test.warehouse.dir}/repairtable/p1=c/p2=a/datafile;
 
 MSCK TABLE default.repairtable;
+show partitions default.repairtable;
 
 MSCK REPAIR TABLE default.repairtable;
+show partitions default.repairtable;
 
 MSCK TABLE repairtable;
+show partitions repairtable;
 
 DROP TABLE default.repairtable;
 
@@ -28,5 +31,6 @@ CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  `p2` s
 dfs -touchz ${system:test.tmp.dir}/apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
 set hive.mv.files.thread=1;
 MSCK TABLE repairtable;
+show partitions repairtable;
 
 DROP TABLE default.repairtable;

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/test/results/clientpositive/msck_repair_0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/msck_repair_0.q.out b/ql/src/test/results/clientpositive/msck_repair_0.q.out
index 2e0d9dc..f48cc78 100644
--- a/ql/src/test/results/clientpositive/msck_repair_0.q.out
+++ b/ql/src/test/results/clientpositive/msck_repair_0.q.out
@@ -16,6 +16,12 @@ PREHOOK: Output: default@repairtable
 POSTHOOK: query: MSCK TABLE repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
+PREHOOK: query: show partitions repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
 PREHOOK: query: MSCK TABLE default.repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
@@ -23,6 +29,12 @@ POSTHOOK: query: MSCK TABLE default.repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=c/p2=a
+PREHOOK: query: show partitions default.repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions default.repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
 PREHOOK: query: MSCK REPAIR TABLE default.repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
@@ -30,13 +42,27 @@ POSTHOOK: query: MSCK REPAIR TABLE default.repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=c/p2=a
-Repair: Added partition to metastore default.repairtable:p1=c/p2=a
+#### A masked pattern was here ####
+PREHOOK: query: show partitions default.repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions default.repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
+p1=c/p2=a
 PREHOOK: query: MSCK TABLE repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
 POSTHOOK: query: MSCK TABLE repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
+PREHOOK: query: show partitions repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
+p1=c/p2=a
 PREHOOK: query: MSCK REPAIR TABLE default.repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
@@ -44,7 +70,15 @@ POSTHOOK: query: MSCK REPAIR TABLE default.repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=e/p2=f
-Repair: Added partition to metastore default.repairtable:p1=e/p2=f
+#### A masked pattern was here ####
+PREHOOK: query: show partitions default.repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions default.repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
+p1=c/p2=a
+p1=e/p2=f
 PREHOOK: query: DROP TABLE default.repairtable
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@repairtable

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/test/results/clientpositive/msck_repair_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/msck_repair_1.q.out b/ql/src/test/results/clientpositive/msck_repair_1.q.out
index 3f2fe75..5f94246 100644
--- a/ql/src/test/results/clientpositive/msck_repair_1.q.out
+++ b/ql/src/test/results/clientpositive/msck_repair_1.q.out
@@ -16,6 +16,12 @@ PREHOOK: Output: default@repairtable
 POSTHOOK: query: MSCK TABLE repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
+PREHOOK: query: SHOW PARTITIONS repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: SHOW PARTITIONS repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
 PREHOOK: query: MSCK TABLE default.repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
@@ -23,6 +29,12 @@ POSTHOOK: query: MSCK TABLE default.repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=c/p2=a
+PREHOOK: query: SHOW PARTITIONS default.repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: SHOW PARTITIONS default.repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
 PREHOOK: query: MSCK REPAIR TABLE default.repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
@@ -30,13 +42,27 @@ POSTHOOK: query: MSCK REPAIR TABLE default.repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=c/p2=a
-Repair: Added partition to metastore default.repairtable:p1=c/p2=a
+#### A masked pattern was here ####
+PREHOOK: query: SHOW PARTITIONS default.repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: SHOW PARTITIONS default.repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
+p1=c/p2=a
 PREHOOK: query: MSCK TABLE repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
 POSTHOOK: query: MSCK TABLE repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
+PREHOOK: query: SHOW PARTITIONS repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: SHOW PARTITIONS repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
+p1=c/p2=a
 PREHOOK: query: DROP TABLE default.repairtable
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@repairtable

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/test/results/clientpositive/msck_repair_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/msck_repair_2.q.out b/ql/src/test/results/clientpositive/msck_repair_2.q.out
index 3f2fe75..c702f37 100644
--- a/ql/src/test/results/clientpositive/msck_repair_2.q.out
+++ b/ql/src/test/results/clientpositive/msck_repair_2.q.out
@@ -16,6 +16,12 @@ PREHOOK: Output: default@repairtable
 POSTHOOK: query: MSCK TABLE repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
+PREHOOK: query: show partitions repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
 PREHOOK: query: MSCK TABLE default.repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
@@ -23,6 +29,12 @@ POSTHOOK: query: MSCK TABLE default.repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=c/p2=a
+PREHOOK: query: show partitions repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
 PREHOOK: query: MSCK REPAIR TABLE default.repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
@@ -30,13 +42,27 @@ POSTHOOK: query: MSCK REPAIR TABLE default.repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=c/p2=a
-Repair: Added partition to metastore default.repairtable:p1=c/p2=a
+#### A masked pattern was here ####
+PREHOOK: query: show partitions repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
+p1=c/p2=a
 PREHOOK: query: MSCK TABLE repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
 POSTHOOK: query: MSCK TABLE repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
+PREHOOK: query: show partitions repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
+p1=c/p2=a
 PREHOOK: query: DROP TABLE default.repairtable
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@repairtable

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/test/results/clientpositive/msck_repair_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/msck_repair_3.q.out b/ql/src/test/results/clientpositive/msck_repair_3.q.out
index 3f2fe75..c702f37 100644
--- a/ql/src/test/results/clientpositive/msck_repair_3.q.out
+++ b/ql/src/test/results/clientpositive/msck_repair_3.q.out
@@ -16,6 +16,12 @@ PREHOOK: Output: default@repairtable
 POSTHOOK: query: MSCK TABLE repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
+PREHOOK: query: show partitions repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
 PREHOOK: query: MSCK TABLE default.repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
@@ -23,6 +29,12 @@ POSTHOOK: query: MSCK TABLE default.repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=c/p2=a
+PREHOOK: query: show partitions repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
 PREHOOK: query: MSCK REPAIR TABLE default.repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
@@ -30,13 +42,27 @@ POSTHOOK: query: MSCK REPAIR TABLE default.repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=c/p2=a
-Repair: Added partition to metastore default.repairtable:p1=c/p2=a
+#### A masked pattern was here ####
+PREHOOK: query: show partitions repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
+p1=c/p2=a
 PREHOOK: query: MSCK TABLE repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
 POSTHOOK: query: MSCK TABLE repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
+PREHOOK: query: show partitions repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
+p1=c/p2=a
 PREHOOK: query: DROP TABLE default.repairtable
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@repairtable

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/test/results/clientpositive/msck_repair_batchsize.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/msck_repair_batchsize.q.out b/ql/src/test/results/clientpositive/msck_repair_batchsize.q.out
index ba99024..aed9271 100644
--- a/ql/src/test/results/clientpositive/msck_repair_batchsize.q.out
+++ b/ql/src/test/results/clientpositive/msck_repair_batchsize.q.out
@@ -23,6 +23,12 @@ POSTHOOK: query: MSCK TABLE default.repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=a/p2=a	repairtable:p1=b/p2=a	repairtable:p1=c/p2=a
+PREHOOK: query: show partitions default.repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions default.repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
 PREHOOK: query: MSCK REPAIR TABLE default.repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
@@ -30,15 +36,31 @@ POSTHOOK: query: MSCK REPAIR TABLE default.repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=a/p2=a	repairtable:p1=b/p2=a	repairtable:p1=c/p2=a
-Repair: Added partition to metastore default.repairtable:p1=a/p2=a
-Repair: Added partition to metastore default.repairtable:p1=b/p2=a
-Repair: Added partition to metastore default.repairtable:p1=c/p2=a
+#### A masked pattern was here ####
+PREHOOK: query: show partitions default.repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions default.repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
+p1=a/p2=a
+p1=b/p2=a
+p1=c/p2=a
 PREHOOK: query: MSCK TABLE repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable
 POSTHOOK: query: MSCK TABLE repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
+PREHOOK: query: show partitions repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
+p1=a/p2=a
+p1=b/p2=a
+p1=c/p2=a
 PREHOOK: query: DROP TABLE default.repairtable
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@repairtable
@@ -64,6 +86,12 @@ POSTHOOK: query: MSCK TABLE repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=c/p2=a
+PREHOOK: query: show partitions repairtable
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@repairtable
+POSTHOOK: query: show partitions repairtable
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@repairtable
 PREHOOK: query: DROP TABLE default.repairtable
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@repairtable

http://git-wip-us.apache.org/repos/asf/hive/blob/0efb9368/ql/src/test/results/clientpositive/repair.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/repair.q.out b/ql/src/test/results/clientpositive/repair.q.out
index c183464..581afe8 100644
--- a/ql/src/test/results/clientpositive/repair.q.out
+++ b/ql/src/test/results/clientpositive/repair.q.out
@@ -30,8 +30,7 @@ POSTHOOK: query: MSCK REPAIR TABLE default.repairtable
 POSTHOOK: type: MSCK
 POSTHOOK: Output: default@repairtable
 Partitions not in metastore:	repairtable:p1=a/p2=a	repairtable:p1=b/p2=a
-Repair: Added partition to metastore default.repairtable:p1=a/p2=a
-Repair: Added partition to metastore default.repairtable:p1=b/p2=a
+#### A masked pattern was here ####
 PREHOOK: query: MSCK TABLE repairtable
 PREHOOK: type: MSCK
 PREHOOK: Output: default@repairtable