You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2015/04/15 13:09:26 UTC

[1/9] hbase git commit: HBASE-13202 Procedure v2 - core framework (addendum)

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 e994b491a -> 4788c6d1a


HBASE-13202 Procedure v2 - core framework (addendum)


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

Branch: refs/heads/hbase-12439
Commit: e75c6201c69e57416525135a397a971ad4d1b902
Parents: e994b49
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Sat Apr 11 08:53:09 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Sat Apr 11 08:53:09 2015 +0100

----------------------------------------------------------------------
 .../apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e75c6201/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
index 6e7306c..7b9fc69 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
@@ -67,8 +67,8 @@ public class ProcedureTestingUtility {
     int execThreads = procExecutor.getNumThreads();
     // stop
     procExecutor.stop();
-    procStore.stop(false);
     procExecutor.join();
+    procStore.stop(false);
     // nothing running...
     if (beforeStartAction != null) {
       beforeStartAction.run();


[2/9] hbase git commit: HBASE-13419 Thrift gateway should propagate text from exception causes (Michael Muller)

Posted by mb...@apache.org.
HBASE-13419 Thrift gateway should propagate text from exception causes (Michael Muller)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/679e0e8d
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/679e0e8d
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/679e0e8d

Branch: refs/heads/hbase-12439
Commit: 679e0e8d2efed297e5b44d17c69952372080b328
Parents: e75c620
Author: tedyu <yu...@gmail.com>
Authored: Mon Apr 13 09:58:04 2015 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Apr 13 09:58:04 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/thrift/ThriftServerRunner.java | 75 ++++++++++----------
 1 file changed, 38 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/679e0e8d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
index 617fab6..4e4ade3 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
@@ -130,6 +130,7 @@ import org.mortbay.jetty.servlet.ServletHolder;
 import org.mortbay.thread.QueuedThreadPool;
 
 import com.google.common.base.Joiner;
+import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
@@ -755,7 +756,7 @@ public class ThriftServerRunner implements Runnable {
         getAdmin().enableTable(getTableName(tableName));
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -765,7 +766,7 @@ public class ThriftServerRunner implements Runnable {
         getAdmin().disableTable(getTableName(tableName));
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -775,7 +776,7 @@ public class ThriftServerRunner implements Runnable {
         return this.connectionCache.getAdmin().isTableEnabled(getTableName(tableName));
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -788,7 +789,7 @@ public class ThriftServerRunner implements Runnable {
         ((HBaseAdmin) getAdmin()).compact(getBytes(tableNameOrRegionName));
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -801,7 +802,7 @@ public class ThriftServerRunner implements Runnable {
         ((HBaseAdmin) getAdmin()).majorCompact(getBytes(tableNameOrRegionName));
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -816,7 +817,7 @@ public class ThriftServerRunner implements Runnable {
         return list;
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -849,7 +850,7 @@ public class ThriftServerRunner implements Runnable {
         return Collections.emptyList();
       } catch (IOException e){
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -894,7 +895,7 @@ public class ThriftServerRunner implements Runnable {
         return ThriftUtilities.cellFromHBase(result.rawCells());
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -937,7 +938,7 @@ public class ThriftServerRunner implements Runnable {
         return ThriftUtilities.cellFromHBase(result.rawCells());
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -981,7 +982,7 @@ public class ThriftServerRunner implements Runnable {
         return ThriftUtilities.cellFromHBase(result.rawCells());
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1038,7 +1039,7 @@ public class ThriftServerRunner implements Runnable {
         return ThriftUtilities.rowResultFromHBase(result);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1103,7 +1104,7 @@ public class ThriftServerRunner implements Runnable {
         return ThriftUtilities.rowResultFromHBase(result);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1135,7 +1136,7 @@ public class ThriftServerRunner implements Runnable {
 
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1157,7 +1158,7 @@ public class ThriftServerRunner implements Runnable {
         table.delete(delete);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1178,10 +1179,10 @@ public class ThriftServerRunner implements Runnable {
         getAdmin().createTable(desc);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       } catch (IllegalArgumentException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IllegalArgument(e.getMessage());
+        throw new IllegalArgument(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1202,7 +1203,7 @@ public class ThriftServerRunner implements Runnable {
         getAdmin().deleteTable(tableName);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1260,10 +1261,10 @@ public class ThriftServerRunner implements Runnable {
           table.put(put);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       } catch (IllegalArgumentException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IllegalArgument(e.getMessage());
+        throw new IllegalArgument(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1331,10 +1332,10 @@ public class ThriftServerRunner implements Runnable {
 
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       } catch (IllegalArgumentException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IllegalArgument(e.getMessage());
+        throw new IllegalArgument(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1360,7 +1361,7 @@ public class ThriftServerRunner implements Runnable {
             getBytes(row), family, qualifier, amount);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1396,7 +1397,7 @@ public class ThriftServerRunner implements Runnable {
         }
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
       return ThriftUtilities.rowResultFromHBase(results, resultScannerWrapper.isColumnSorted());
     }
@@ -1450,7 +1451,7 @@ public class ThriftServerRunner implements Runnable {
         return addScanner(table.getScanner(scan), tScan.sortColumns);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1475,7 +1476,7 @@ public class ThriftServerRunner implements Runnable {
         return addScanner(table.getScanner(scan), false);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1501,7 +1502,7 @@ public class ThriftServerRunner implements Runnable {
         return addScanner(table.getScanner(scan), false);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1531,7 +1532,7 @@ public class ThriftServerRunner implements Runnable {
         return addScanner(table.getScanner(scan), false);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1557,7 +1558,7 @@ public class ThriftServerRunner implements Runnable {
         return addScanner(table.getScanner(scan), false);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1585,7 +1586,7 @@ public class ThriftServerRunner implements Runnable {
         return addScanner(table.getScanner(scan), false);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1606,7 +1607,7 @@ public class ThriftServerRunner implements Runnable {
         return columns;
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1619,7 +1620,7 @@ public class ThriftServerRunner implements Runnable {
         return ThriftUtilities.cellFromHBase(result.rawCells());
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1658,7 +1659,7 @@ public class ThriftServerRunner implements Runnable {
         return region;
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1696,7 +1697,7 @@ public class ThriftServerRunner implements Runnable {
         table.increment(inc);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1724,7 +1725,7 @@ public class ThriftServerRunner implements Runnable {
         return ThriftUtilities.cellFromHBase(result.rawCells());
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       }
     }
 
@@ -1745,7 +1746,7 @@ public class ThriftServerRunner implements Runnable {
         put.setDurability(mput.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
       } catch (IllegalArgumentException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IllegalArgument(e.getMessage());
+        throw new IllegalArgument(Throwables.getStackTraceAsString(e));
       }
 
       Table table = null;
@@ -1756,10 +1757,10 @@ public class ThriftServerRunner implements Runnable {
           value != null ? getBytes(value) : HConstants.EMPTY_BYTE_ARRAY, put);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IOError(e.getMessage());
+        throw new IOError(Throwables.getStackTraceAsString(e));
       } catch (IllegalArgumentException e) {
         LOG.warn(e.getMessage(), e);
-        throw new IllegalArgument(e.getMessage());
+        throw new IllegalArgument(Throwables.getStackTraceAsString(e));
       }
     }
   }


[3/9] hbase git commit: HBASE-13457 SnapshotExistsException doesn't honor the DoNotRetry

Posted by mb...@apache.org.
HBASE-13457 SnapshotExistsException doesn't honor the DoNotRetry


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

Branch: refs/heads/hbase-12439
Commit: e9da064ccd8dd3c64d99e0863e36cdc1236779f1
Parents: 679e0e8
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Mon Apr 13 23:21:50 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Mon Apr 13 23:21:50 2015 +0100

----------------------------------------------------------------------
 .../hbase/snapshot/SnapshotExistsException.java |   3 +
 .../snapshot/TestSnapshotClientRetries.java     | 125 +++++++++++++++++++
 2 files changed, 128 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e9da064c/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
index 2c609d9..172c89e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
@@ -28,6 +28,9 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptio
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class SnapshotExistsException extends HBaseSnapshotException {
+  public SnapshotExistsException(String msg) {
+    super(msg);
+  }
 
   /**
    * Failure due to the snapshot already existing

http://git-wip-us.apache.org/repos/asf/hbase/blob/e9da064c/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java
new file mode 100644
index 0000000..5168b85
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java
@@ -0,0 +1,125 @@
+/**
+ *
+ * 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.hbase.snapshot;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.snapshot.SnapshotExistsException;
+import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.TestTableName;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@Category({ MediumTests.class })
+public class TestSnapshotClientRetries {
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final Log LOG = LogFactory.getLog(TestSnapshotClientRetries.class);
+
+  @Rule public TestTableName TEST_TABLE = new TestTableName();
+
+  @Before
+  public void setUp() throws Exception {
+    TEST_UTIL.getConfiguration().set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+      MasterSyncObserver.class.getName());
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test(timeout = 60000, expected=SnapshotExistsException.class)
+  public void testSnapshotAlreadyExist() throws Exception {
+    final String snapshotName = "testSnapshotAlreadyExist";
+    TEST_UTIL.createTable(TEST_TABLE.getTableName(), "f");
+    TEST_UTIL.getHBaseAdmin().snapshot(snapshotName, TEST_TABLE.getTableName());
+    snapshotAndAssertOneRetry(snapshotName, TEST_TABLE.getTableName());
+  }
+
+  @Test(timeout = 60000, expected=SnapshotDoesNotExistException.class)
+  public void testCloneNonExistentSnapshot() throws Exception {
+    final String snapshotName = "testCloneNonExistentSnapshot";
+    cloneAndAssertOneRetry(snapshotName, TEST_TABLE.getTableName());
+  }
+
+  public static class MasterSyncObserver extends BaseMasterObserver {
+    volatile AtomicInteger snapshotCount = null;
+    volatile AtomicInteger cloneCount = null;
+
+    @Override
+    public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+        throws IOException {
+      if (snapshotCount != null) {
+        snapshotCount.incrementAndGet();
+      }
+    }
+
+    @Override
+    public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+        throws IOException {
+      if (cloneCount != null) {
+        cloneCount.incrementAndGet();
+      }
+    }
+  }
+
+  public void snapshotAndAssertOneRetry(final String snapshotName, final TableName tableName)
+      throws Exception {
+    MasterSyncObserver observer = getMasterSyncObserver();
+    observer.snapshotCount = new AtomicInteger(0);
+    TEST_UTIL.getHBaseAdmin().snapshot(snapshotName, tableName);
+    assertEquals(1, observer.snapshotCount.get());
+  }
+
+  public void cloneAndAssertOneRetry(final String snapshotName, final TableName tableName)
+      throws Exception {
+    MasterSyncObserver observer = getMasterSyncObserver();
+    observer.cloneCount = new AtomicInteger(0);
+    TEST_UTIL.getHBaseAdmin().cloneSnapshot(snapshotName, tableName);
+    assertEquals(1, observer.cloneCount.get());
+  }
+
+  private MasterSyncObserver getMasterSyncObserver() {
+    return (MasterSyncObserver)TEST_UTIL.getHBaseCluster().getMaster()
+      .getMasterCoprocessorHost().findCoprocessor(MasterSyncObserver.class.getName());
+  }
+}


[9/9] hbase git commit: HBASE-13455 Procedure V2 - master truncate table

Posted by mb...@apache.org.
HBASE-13455 Procedure V2 - master truncate table


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/4788c6d1
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4788c6d1
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4788c6d1

Branch: refs/heads/hbase-12439
Commit: 4788c6d1a8cd4b6f31416f57e757bcd8738b4772
Parents: d75326a
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Wed Apr 15 09:40:18 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Wed Apr 15 10:35:41 2015 +0100

----------------------------------------------------------------------
 .../generated/MasterProcedureProtos.java        | 2016 ++++++++++++++++--
 .../src/main/protobuf/MasterProcedure.proto     |   18 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   10 +-
 .../master/procedure/DeleteTableProcedure.java  |    5 +-
 .../procedure/TruncateTableProcedure.java       |  291 +++
 .../hadoop/hbase/HBaseTestingUtility.java       |   12 +
 .../MasterProcedureTestingUtility.java          |   43 +
 .../TestMasterFailoverWithProcedures.java       |   62 +
 .../procedure/TestTruncateTableProcedure.java   |  246 +++
 9 files changed, 2496 insertions(+), 207 deletions(-)
----------------------------------------------------------------------



[8/9] hbase git commit: HBASE-13455 Procedure V2 - master truncate table

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/4788c6d1/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
index d83ee19..e0a4775 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
@@ -254,6 +254,133 @@ public final class MasterProcedureProtos {
   }
 
   /**
+   * Protobuf enum {@code TruncateTableState}
+   */
+  public enum TruncateTableState
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>TRUNCATE_TABLE_PRE_OPERATION = 1;</code>
+     */
+    TRUNCATE_TABLE_PRE_OPERATION(0, 1),
+    /**
+     * <code>TRUNCATE_TABLE_REMOVE_FROM_META = 2;</code>
+     */
+    TRUNCATE_TABLE_REMOVE_FROM_META(1, 2),
+    /**
+     * <code>TRUNCATE_TABLE_CLEAR_FS_LAYOUT = 3;</code>
+     */
+    TRUNCATE_TABLE_CLEAR_FS_LAYOUT(2, 3),
+    /**
+     * <code>TRUNCATE_TABLE_CREATE_FS_LAYOUT = 4;</code>
+     */
+    TRUNCATE_TABLE_CREATE_FS_LAYOUT(3, 4),
+    /**
+     * <code>TRUNCATE_TABLE_ADD_TO_META = 5;</code>
+     */
+    TRUNCATE_TABLE_ADD_TO_META(4, 5),
+    /**
+     * <code>TRUNCATE_TABLE_ASSIGN_REGIONS = 6;</code>
+     */
+    TRUNCATE_TABLE_ASSIGN_REGIONS(5, 6),
+    /**
+     * <code>TRUNCATE_TABLE_POST_OPERATION = 7;</code>
+     */
+    TRUNCATE_TABLE_POST_OPERATION(6, 7),
+    ;
+
+    /**
+     * <code>TRUNCATE_TABLE_PRE_OPERATION = 1;</code>
+     */
+    public static final int TRUNCATE_TABLE_PRE_OPERATION_VALUE = 1;
+    /**
+     * <code>TRUNCATE_TABLE_REMOVE_FROM_META = 2;</code>
+     */
+    public static final int TRUNCATE_TABLE_REMOVE_FROM_META_VALUE = 2;
+    /**
+     * <code>TRUNCATE_TABLE_CLEAR_FS_LAYOUT = 3;</code>
+     */
+    public static final int TRUNCATE_TABLE_CLEAR_FS_LAYOUT_VALUE = 3;
+    /**
+     * <code>TRUNCATE_TABLE_CREATE_FS_LAYOUT = 4;</code>
+     */
+    public static final int TRUNCATE_TABLE_CREATE_FS_LAYOUT_VALUE = 4;
+    /**
+     * <code>TRUNCATE_TABLE_ADD_TO_META = 5;</code>
+     */
+    public static final int TRUNCATE_TABLE_ADD_TO_META_VALUE = 5;
+    /**
+     * <code>TRUNCATE_TABLE_ASSIGN_REGIONS = 6;</code>
+     */
+    public static final int TRUNCATE_TABLE_ASSIGN_REGIONS_VALUE = 6;
+    /**
+     * <code>TRUNCATE_TABLE_POST_OPERATION = 7;</code>
+     */
+    public static final int TRUNCATE_TABLE_POST_OPERATION_VALUE = 7;
+
+
+    public final int getNumber() { return value; }
+
+    public static TruncateTableState valueOf(int value) {
+      switch (value) {
+        case 1: return TRUNCATE_TABLE_PRE_OPERATION;
+        case 2: return TRUNCATE_TABLE_REMOVE_FROM_META;
+        case 3: return TRUNCATE_TABLE_CLEAR_FS_LAYOUT;
+        case 4: return TRUNCATE_TABLE_CREATE_FS_LAYOUT;
+        case 5: return TRUNCATE_TABLE_ADD_TO_META;
+        case 6: return TRUNCATE_TABLE_ASSIGN_REGIONS;
+        case 7: return TRUNCATE_TABLE_POST_OPERATION;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<TruncateTableState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<TruncateTableState>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<TruncateTableState>() {
+            public TruncateTableState findValueByNumber(int number) {
+              return TruncateTableState.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(2);
+    }
+
+    private static final TruncateTableState[] VALUES = values();
+
+    public static TruncateTableState valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private TruncateTableState(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:TruncateTableState)
+  }
+
+  /**
    * Protobuf enum {@code DeleteTableState}
    */
   public enum DeleteTableState
@@ -346,7 +473,7 @@ public final class MasterProcedureProtos {
     }
     public static final com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(2);
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(3);
     }
 
     private static final DeleteTableState[] VALUES = values();
@@ -455,7 +582,7 @@ public final class MasterProcedureProtos {
     }
     public static final com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(3);
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(4);
     }
 
     private static final AddColumnFamilyState[] VALUES = values();
@@ -564,7 +691,7 @@ public final class MasterProcedureProtos {
     }
     public static final com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(4);
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(5);
     }
 
     private static final ModifyColumnFamilyState[] VALUES = values();
@@ -682,7 +809,7 @@ public final class MasterProcedureProtos {
     }
     public static final com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(5);
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(6);
     }
 
     private static final DeleteColumnFamilyState[] VALUES = values();
@@ -800,7 +927,7 @@ public final class MasterProcedureProtos {
     }
     public static final com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(6);
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(7);
     }
 
     private static final EnableTableState[] VALUES = values();
@@ -918,7 +1045,7 @@ public final class MasterProcedureProtos {
     }
     public static final com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(7);
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(8);
     }
 
     private static final DisableTableState[] VALUES = values();
@@ -2989,205 +3116,1670 @@ public final class MasterProcedureProtos {
           unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
           onChanged();
         } else {
-          unmodifiedTableSchemaBuilder_.clear();
+          unmodifiedTableSchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getUnmodifiedTableSchemaBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getUnmodifiedTableSchemaFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder() {
+        if (unmodifiedTableSchemaBuilder_ != null) {
+          return unmodifiedTableSchemaBuilder_.getMessageOrBuilder();
+        } else {
+          return unmodifiedTableSchema_;
+        }
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
+          getUnmodifiedTableSchemaFieldBuilder() {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
+                  unmodifiedTableSchema_,
+                  getParentForChildren(),
+                  isClean());
+          unmodifiedTableSchema_ = null;
+        }
+        return unmodifiedTableSchemaBuilder_;
+      }
+
+      // required .TableSchema modified_table_schema = 3;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema modifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> modifiedTableSchemaBuilder_;
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public boolean hasModifiedTableSchema() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getModifiedTableSchema() {
+        if (modifiedTableSchemaBuilder_ == null) {
+          return modifiedTableSchema_;
+        } else {
+          return modifiedTableSchemaBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public Builder setModifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (modifiedTableSchemaBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          modifiedTableSchema_ = value;
+          onChanged();
+        } else {
+          modifiedTableSchemaBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public Builder setModifiedTableSchema(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder builderForValue) {
+        if (modifiedTableSchemaBuilder_ == null) {
+          modifiedTableSchema_ = builderForValue.build();
+          onChanged();
+        } else {
+          modifiedTableSchemaBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public Builder mergeModifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (modifiedTableSchemaBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              modifiedTableSchema_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance()) {
+            modifiedTableSchema_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.newBuilder(modifiedTableSchema_).mergeFrom(value).buildPartial();
+          } else {
+            modifiedTableSchema_ = value;
+          }
+          onChanged();
+        } else {
+          modifiedTableSchemaBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public Builder clearModifiedTableSchema() {
+        if (modifiedTableSchemaBuilder_ == null) {
+          modifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+          onChanged();
+        } else {
+          modifiedTableSchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getModifiedTableSchemaBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getModifiedTableSchemaFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getModifiedTableSchemaOrBuilder() {
+        if (modifiedTableSchemaBuilder_ != null) {
+          return modifiedTableSchemaBuilder_.getMessageOrBuilder();
+        } else {
+          return modifiedTableSchema_;
+        }
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
+          getModifiedTableSchemaFieldBuilder() {
+        if (modifiedTableSchemaBuilder_ == null) {
+          modifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
+                  modifiedTableSchema_,
+                  getParentForChildren(),
+                  isClean());
+          modifiedTableSchema_ = null;
+        }
+        return modifiedTableSchemaBuilder_;
+      }
+
+      // required bool delete_column_family_in_modify = 4;
+      private boolean deleteColumnFamilyInModify_ ;
+      /**
+       * <code>required bool delete_column_family_in_modify = 4;</code>
+       */
+      public boolean hasDeleteColumnFamilyInModify() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>required bool delete_column_family_in_modify = 4;</code>
+       */
+      public boolean getDeleteColumnFamilyInModify() {
+        return deleteColumnFamilyInModify_;
+      }
+      /**
+       * <code>required bool delete_column_family_in_modify = 4;</code>
+       */
+      public Builder setDeleteColumnFamilyInModify(boolean value) {
+        bitField0_ |= 0x00000008;
+        deleteColumnFamilyInModify_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bool delete_column_family_in_modify = 4;</code>
+       */
+      public Builder clearDeleteColumnFamilyInModify() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        deleteColumnFamilyInModify_ = false;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:ModifyTableStateData)
+    }
+
+    static {
+      defaultInstance = new ModifyTableStateData(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:ModifyTableStateData)
+  }
+
+  public interface TruncateTableStateDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required .UserInformation user_info = 1;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    boolean hasUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
+
+    // required bool preserve_splits = 2;
+    /**
+     * <code>required bool preserve_splits = 2;</code>
+     */
+    boolean hasPreserveSplits();
+    /**
+     * <code>required bool preserve_splits = 2;</code>
+     */
+    boolean getPreserveSplits();
+
+    // optional .TableName table_name = 3;
+    /**
+     * <code>optional .TableName table_name = 3;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>optional .TableName table_name = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>optional .TableName table_name = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    // optional .TableSchema table_schema = 4;
+    /**
+     * <code>optional .TableSchema table_schema = 4;</code>
+     */
+    boolean hasTableSchema();
+    /**
+     * <code>optional .TableSchema table_schema = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema();
+    /**
+     * <code>optional .TableSchema table_schema = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder();
+
+    // repeated .RegionInfo region_info = 5;
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> 
+        getRegionInfoList();
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    int getRegionInfoCount();
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList();
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code TruncateTableStateData}
+   */
+  public static final class TruncateTableStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements TruncateTableStateDataOrBuilder {
+    // Use TruncateTableStateData.newBuilder() to construct.
+    private TruncateTableStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private TruncateTableStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final TruncateTableStateData defaultInstance;
+    public static TruncateTableStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public TruncateTableStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private TruncateTableStateData(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = userInfo_.toBuilder();
+              }
+              userInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(userInfo_);
+                userInfo_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              preserveSplits_ = input.readBool();
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000004;
+              break;
+            }
+            case 34: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000008) == 0x00000008)) {
+                subBuilder = tableSchema_.toBuilder();
+              }
+              tableSchema_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableSchema_);
+                tableSchema_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000008;
+              break;
+            }
+            case 42: {
+              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+                regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>();
+                mutable_bitField0_ |= 0x00000010;
+              }
+              regionInfo_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+          regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_TruncateTableStateData_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_TruncateTableStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<TruncateTableStateData> PARSER =
+        new com.google.protobuf.AbstractParser<TruncateTableStateData>() {
+      public TruncateTableStateData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new TruncateTableStateData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<TruncateTableStateData> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required .UserInformation user_info = 1;
+    public static final int USER_INFO_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public boolean hasUserInfo() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+      return userInfo_;
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+      return userInfo_;
+    }
+
+    // required bool preserve_splits = 2;
+    public static final int PRESERVE_SPLITS_FIELD_NUMBER = 2;
+    private boolean preserveSplits_;
+    /**
+     * <code>required bool preserve_splits = 2;</code>
+     */
+    public boolean hasPreserveSplits() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required bool preserve_splits = 2;</code>
+     */
+    public boolean getPreserveSplits() {
+      return preserveSplits_;
+    }
+
+    // optional .TableName table_name = 3;
+    public static final int TABLE_NAME_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>optional .TableName table_name = 3;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional .TableName table_name = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_;
+    }
+    /**
+     * <code>optional .TableName table_name = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_;
+    }
+
+    // optional .TableSchema table_schema = 4;
+    public static final int TABLE_SCHEMA_FIELD_NUMBER = 4;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema tableSchema_;
+    /**
+     * <code>optional .TableSchema table_schema = 4;</code>
+     */
+    public boolean hasTableSchema() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional .TableSchema table_schema = 4;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema() {
+      return tableSchema_;
+    }
+    /**
+     * <code>optional .TableSchema table_schema = 4;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder() {
+      return tableSchema_;
+    }
+
+    // repeated .RegionInfo region_info = 5;
+    public static final int REGION_INFO_FIELD_NUMBER = 5;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_;
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+      return regionInfo_;
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList() {
+      return regionInfo_;
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    public int getRegionInfoCount() {
+      return regionInfo_.size();
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
+      return regionInfo_.get(index);
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index) {
+      return regionInfo_.get(index);
+    }
+
+    private void initFields() {
+      userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      preserveSplits_ = false;
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      regionInfo_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasUserInfo()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasPreserveSplits()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getUserInfo().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (hasTableName()) {
+        if (!getTableName().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      if (hasTableSchema()) {
+        if (!getTableSchema().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getRegionInfoCount(); i++) {
+        if (!getRegionInfo(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBool(2, preserveSplits_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, tableName_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeMessage(4, tableSchema_);
+      }
+      for (int i = 0; i < regionInfo_.size(); i++) {
+        output.writeMessage(5, regionInfo_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(2, preserveSplits_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, tableName_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, tableSchema_);
+      }
+      for (int i = 0; i < regionInfo_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(5, regionInfo_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData) obj;
+
+      boolean result = true;
+      result = result && (hasUserInfo() == other.hasUserInfo());
+      if (hasUserInfo()) {
+        result = result && getUserInfo()
+            .equals(other.getUserInfo());
+      }
+      result = result && (hasPreserveSplits() == other.hasPreserveSplits());
+      if (hasPreserveSplits()) {
+        result = result && (getPreserveSplits()
+            == other.getPreserveSplits());
+      }
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && (hasTableSchema() == other.hasTableSchema());
+      if (hasTableSchema()) {
+        result = result && getTableSchema()
+            .equals(other.getTableSchema());
+      }
+      result = result && getRegionInfoList()
+          .equals(other.getRegionInfoList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasUserInfo()) {
+        hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getUserInfo().hashCode();
+      }
+      if (hasPreserveSplits()) {
+        hash = (37 * hash) + PRESERVE_SPLITS_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getPreserveSplits());
+      }
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (hasTableSchema()) {
+        hash = (37 * hash) + TABLE_SCHEMA_FIELD_NUMBER;
+        hash = (53 * hash) + getTableSchema().hashCode();
+      }
+      if (getRegionInfoCount() > 0) {
+        hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getRegionInfoList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code TruncateTableStateData}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateDataOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_TruncateTableStateData_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_TruncateTableStateData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getUserInfoFieldBuilder();
+          getTableNameFieldBuilder();
+          getTableSchemaFieldBuilder();
+          getRegionInfoFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        preserveSplits_ = false;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        if (tableSchemaBuilder_ == null) {
+          tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+        } else {
+          tableSchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000010);
+        } else {
+          regionInfoBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_TruncateTableStateData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (userInfoBuilder_ == null) {
+          result.userInfo_ = userInfo_;
+        } else {
+          result.userInfo_ = userInfoBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.preserveSplits_ = preserveSplits_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        if (tableSchemaBuilder_ == null) {
+          result.tableSchema_ = tableSchema_;
+        } else {
+          result.tableSchema_ = tableSchemaBuilder_.build();
+        }
+        if (regionInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000010) == 0x00000010)) {
+            regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
+            bitField0_ = (bitField0_ & ~0x00000010);
+          }
+          result.regionInfo_ = regionInfo_;
+        } else {
+          result.regionInfo_ = regionInfoBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData.getDefaultInstance()) return this;
+        if (other.hasUserInfo()) {
+          mergeUserInfo(other.getUserInfo());
+        }
+        if (other.hasPreserveSplits()) {
+          setPreserveSplits(other.getPreserveSplits());
+        }
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (other.hasTableSchema()) {
+          mergeTableSchema(other.getTableSchema());
+        }
+        if (regionInfoBuilder_ == null) {
+          if (!other.regionInfo_.isEmpty()) {
+            if (regionInfo_.isEmpty()) {
+              regionInfo_ = other.regionInfo_;
+              bitField0_ = (bitField0_ & ~0x00000010);
+            } else {
+              ensureRegionInfoIsMutable();
+              regionInfo_.addAll(other.regionInfo_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.regionInfo_.isEmpty()) {
+            if (regionInfoBuilder_.isEmpty()) {
+              regionInfoBuilder_.dispose();
+              regionInfoBuilder_ = null;
+              regionInfo_ = other.regionInfo_;
+              bitField0_ = (bitField0_ & ~0x00000010);
+              regionInfoBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getRegionInfoFieldBuilder() : null;
+            } else {
+              regionInfoBuilder_.addAllMessages(other.regionInfo_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasUserInfo()) {
+          
+          return false;
+        }
+        if (!hasPreserveSplits()) {
+          
+          return false;
+        }
+        if (!getUserInfo().isInitialized()) {
+          
+          return false;
+        }
+        if (hasTableName()) {
+          if (!getTableName().isInitialized()) {
+            
+            return false;
+          }
+        }
+        if (hasTableSchema()) {
+          if (!getTableSchema().isInitialized()) {
+            
+            return false;
+          }
+        }
+        for (int i = 0; i < getRegionInfoCount(); i++) {
+          if (!getRegionInfo(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableStateData) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required .UserInformation user_info = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public boolean hasUserInfo() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+        if (userInfoBuilder_ == null) {
+          return userInfo_;
+        } else {
+          return userInfoBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          userInfo_ = value;
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder builderForValue) {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = builderForValue.build();
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder mergeUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              userInfo_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance()) {
+            userInfo_ =
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.newBuilder(userInfo_).mergeFrom(value).buildPartial();
+          } else {
+            userInfo_ = value;
+          }
+          onChanged();
+        } else {
+          userInfoBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder clearUserInfo() {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+          onChanged();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder getUserInfoBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getUserInfoFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+        if (userInfoBuilder_ != null) {
+          return userInfoBuilder_.getMessageOrBuilder();
+        } else {
+          return userInfo_;
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
+          getUserInfoFieldBuilder() {
+        if (userInfoBuilder_ == null) {
+          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
+                  userInfo_,
+                  getParentForChildren(),
+                  isClean());
+          userInfo_ = null;
+        }
+        return userInfoBuilder_;
+      }
+
+      // required bool preserve_splits = 2;
+      private boolean preserveSplits_ ;
+      /**
+       * <code>required bool preserve_splits = 2;</code>
+       */
+      public boolean hasPreserveSplits() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required bool preserve_splits = 2;</code>
+       */
+      public boolean getPreserveSplits() {
+        return preserveSplits_;
+      }
+      /**
+       * <code>required bool preserve_splits = 2;</code>
+       */
+      public Builder setPreserveSplits(boolean value) {
+        bitField0_ |= 0x00000002;
+        preserveSplits_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bool preserve_splits = 2;</code>
+       */
+      public Builder clearPreserveSplits() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        preserveSplits_ = false;
+        onChanged();
+        return this;
+      }
+
+      // optional .TableName table_name = 3;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>optional .TableName table_name = 3;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional .TableName table_name = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .TableName table_name = 3;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .TableName table_name = 3;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .TableName table_name = 3;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .TableName table_name = 3;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      /**
+       * <code>optional .TableName table_name = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .TableName table_name = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_;
+        }
+      }
+      /**
+       * <code>optional .TableName table_name = 3;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tableName_,
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      // optional .TableSchema table_schema = 4;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
+      /**
+       * <code>optional .TableSchema table_schema = 4;</code>
+       */
+      public boolean hasTableSchema() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional .TableSchema table_schema = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema() {
+        if (tableSchemaBuilder_ == null) {
+          return tableSchema_;
+        } else {
+          return tableSchemaBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .TableSchema table_schema = 4;</code>
+       */
+      public Builder setTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (tableSchemaBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableSchema_ = value;
+          onChanged();
+        } else {
+          tableSchemaBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema table_schema = 4;</code>
+       */
+      public Builder setTableSchema(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder builderForValue) {
+        if (tableSchemaBuilder_ == null) {
+          tableSchema_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableSchemaBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema table_schema = 4;</code>
+       */
+      public Builder mergeTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (tableSchemaBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008) &&
+              tableSchema_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance()) {
+            tableSchema_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.newBuilder(tableSchema_).mergeFrom(value).buildPartial();
+          } else {
+            tableSchema_ = value;
+          }
+          onChanged();
+        } else {
+          tableSchemaBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema table_schema = 4;</code>
+       */
+      public Builder clearTableSchema() {
+        if (tableSchemaBuilder_ == null) {
+          tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+          onChanged();
+        } else {
+          tableSchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema table_schema = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getTableSchemaBuilder() {
+        bitField0_ |= 0x00000008;
+        onChanged();
+        return getTableSchemaFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .TableSchema table_schema = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder() {
+        if (tableSchemaBuilder_ != null) {
+          return tableSchemaBuilder_.getMessageOrBuilder();
+        } else {
+          return tableSchema_;
+        }
+      }
+      /**
+       * <code>optional .TableSchema table_schema = 4;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
+          getTableSchemaFieldBuilder() {
+        if (tableSchemaBuilder_ == null) {
+          tableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
+                  tableSchema_,
+                  getParentForChildren(),
+                  isClean());
+          tableSchema_ = null;
+        }
+        return tableSchemaBuilder_;
+      }
+
+      // repeated .RegionInfo region_info = 5;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_ =
+        java.util.Collections.emptyList();
+      private void ensureRegionInfoIsMutable() {
+        if (!((bitField0_ & 0x00000010) == 0x00000010)) {
+          regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>(regionInfo_);
+          bitField0_ |= 0x00000010;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
+
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+        if (regionInfoBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(regionInfo_);
+        } else {
+          return regionInfoBuilder_.getMessageList();
         }
-        bitField0_ = (bitField0_ & ~0x00000002);
-        return this;
       }
       /**
-       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getUnmodifiedTableSchemaBuilder() {
-        bitField0_ |= 0x00000002;
-        onChanged();
-        return getUnmodifiedTableSchemaFieldBuilder().getBuilder();
+      public int getRegionInfoCount() {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_.size();
+        } else {
+          return regionInfoBuilder_.getCount();
+        }
       }
       /**
-       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder() {
-        if (unmodifiedTableSchemaBuilder_ != null) {
-          return unmodifiedTableSchemaBuilder_.getMessageOrBuilder();
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_.get(index);
         } else {
-          return unmodifiedTableSchema_;
+          return regionInfoBuilder_.getMessage(index);
         }
       }
       /**
-       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
-          getUnmodifiedTableSchemaFieldBuilder() {
-        if (unmodifiedTableSchemaBuilder_ == null) {
-          unmodifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
-                  unmodifiedTableSchema_,
-                  getParentForChildren(),
-                  isClean());
-          unmodifiedTableSchema_ = null;
+      public Builder setRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureRegionInfoIsMutable();
+          regionInfo_.set(index, value);
+          onChanged();
+        } else {
+          regionInfoBuilder_.setMessage(index, value);
         }
-        return unmodifiedTableSchemaBuilder_;
+        return this;
       }
-
-      // required .TableSchema modified_table_schema = 3;
-      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema modifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> modifiedTableSchemaBuilder_;
       /**
-       * <code>required .TableSchema modified_table_schema = 3;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public boolean hasModifiedTableSchema() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
+      public Builder setRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          regionInfoBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
       }
       /**
-       * <code>required .TableSchema modified_table_schema = 3;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getModifiedTableSchema() {
-        if (modifiedTableSchemaBuilder_ == null) {
-          return modifiedTableSchema_;
+      public Builder addRegionInfo(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(value);
+          onChanged();
         } else {
-          return modifiedTableSchemaBuilder_.getMessage();
+          regionInfoBuilder_.addMessage(value);
         }
+        return this;
       }
       /**
-       * <code>required .TableSchema modified_table_schema = 3;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public Builder setModifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
-        if (modifiedTableSchemaBuilder_ == null) {
+      public Builder addRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          modifiedTableSchema_ = value;
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(index, value);
           onChanged();
         } else {
-          modifiedTableSchemaBuilder_.setMessage(value);
+          regionInfoBuilder_.addMessage(index, value);
         }
-        bitField0_ |= 0x00000004;
         return this;
       }
       /**
-       * <code>required .TableSchema modified_table_schema = 3;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public Builder setModifiedTableSchema(
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder builderForValue) {
-        if (modifiedTableSchemaBuilder_ == null) {
-          modifiedTableSchema_ = builderForValue.build();
+      public Builder addRegionInfo(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(builderForValue.build());
           onChanged();
         } else {
-          modifiedTableSchemaBuilder_.setMessage(builderForValue.build());
+          regionInfoBuilder_.addMessage(builderForValue.build());
         }
-        bitField0_ |= 0x00000004;
         return this;
       }
       /**
-       * <code>required .TableSchema modified_table_schema = 3;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public Builder mergeModifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
-        if (modifiedTableSchemaBuilder_ == null) {
-          if (((bitField0_ & 0x00000004) == 0x00000004) &&
-              modifiedTableSchema_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance()) {
-            modifiedTableSchema_ =
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.newBuilder(modifiedTableSchema_).mergeFrom(value).buildPartial();
-          } else {
-            modifiedTableSchema_ = value;
-          }
+      public Builder addRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(index, builderForValue.build());
           onChanged();
         } else {
-          modifiedTableSchemaBuilder_.mergeFrom(value);
+          regionInfoBuilder_.addMessage(index, builderForValue.build());
         }
-        bitField0_ |= 0x00000004;
         return this;
       }
       /**
-       * <code>required .TableSchema modified_table_schema = 3;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public Builder clearModifiedTableSchema() {
-        if (modifiedTableSchemaBuilder_ == null) {
-          modifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      public Builder addAllRegionInfo(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> values) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          super.addAll(values, regionInfo_);
           onChanged();
         } else {
-          modifiedTableSchemaBuilder_.clear();
+          regionInfoBuilder_.addAllMessages(values);
         }
-        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
       /**
-       * <code>required .TableSchema modified_table_schema = 3;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getModifiedTableSchemaBuilder() {
-        bitField0_ |= 0x00000004;
-        onChanged();
-        return getModifiedTableSchemaFieldBuilder().getBuilder();
+      public Builder clearRegionInfo() {
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000010);
+          onChanged();
+        } else {
+          regionInfoBuilder_.clear();
+        }
+        return this;
       }
       /**
-       * <code>required .TableSchema modified_table_schema = 3;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getModifiedTableSchemaOrBuilder() {
-        if (modifiedTableSchemaBuilder_ != null) {
-          return modifiedTableSchemaBuilder_.getMessageOrBuilder();
+      public Builder removeRegionInfo(int index) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.remove(index);
+          onChanged();
         } else {
-          return modifiedTableSchema_;
+          regionInfoBuilder_.remove(index);
         }
+        return this;
       }
       /**
-       * <code>required .TableSchema modified_table_schema = 3;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
-          getModifiedTableSchemaFieldBuilder() {
-        if (modifiedTableSchemaBuilder_ == null) {
-          modifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
-                  modifiedTableSchema_,
-                  getParentForChildren(),
-                  isClean());
-          modifiedTableSchema_ = null;
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder(
+          int index) {
+        return getRegionInfoFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+          int index) {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_.get(index);  } else {
+          return regionInfoBuilder_.getMessageOrBuilder(index);
         }
-        return modifiedTableSchemaBuilder_;
       }
-
-      // required bool delete_column_family_in_modify = 4;
-      private boolean deleteColumnFamilyInModify_ ;
       /**
-       * <code>required bool delete_column_family_in_modify = 4;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public boolean hasDeleteColumnFamilyInModify() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+           getRegionInfoOrBuilderList() {
+        if (regionInfoBuilder_ != null) {
+          return regionInfoBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(regionInfo_);
+        }
       }
       /**
-       * <code>required bool delete_column_family_in_modify = 4;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public boolean getDeleteColumnFamilyInModify() {
-        return deleteColumnFamilyInModify_;
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder() {
+        return getRegionInfoFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
       }
       /**
-       * <code>required bool delete_column_family_in_modify = 4;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public Builder setDeleteColumnFamilyInModify(boolean value) {
-        bitField0_ |= 0x00000008;
-        deleteColumnFamilyInModify_ = value;
-        onChanged();
-        return this;
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder(
+          int index) {
+        return getRegionInfoFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
       }
       /**
-       * <code>required bool delete_column_family_in_modify = 4;</code>
+       * <code>repeated .RegionInfo region_info = 5;</code>
        */
-      public Builder clearDeleteColumnFamilyInModify() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        deleteColumnFamilyInModify_ = false;
-        onChanged();
-        return this;
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder> 
+           getRegionInfoBuilderList() {
+        return getRegionInfoFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+          getRegionInfoFieldBuilder() {
+        if (regionInfoBuilder_ == null) {
+          regionInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
+                  regionInfo_,
+                  ((bitField0_ & 0x00000010) == 0x00000010),
+                  getParentForChildren(),
+                  isClean());
+          regionInfo_ = null;
+        }
+        return regionInfoBuilder_;
       }
 
-      // @@protoc_insertion_point(builder_scope:ModifyTableStateData)
+      // @@protoc_insertion_point(builder_scope:TruncateTableStateData)
     }
 
     static {
-      defaultInstance = new ModifyTableStateData(true);
+      defaultInstance = new TruncateTableStateData(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:ModifyTableStateData)
+    // @@protoc_insertion_point(class_scope:TruncateTableStateData)
   }
 
   public interface DeleteTableStateDataOrBuilder
@@ -9619,6 +11211,11 @@ public final class MasterProcedureProtos {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_ModifyTableStateData_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_TruncateTableStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_TruncateTableStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_DeleteTableStateData_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -9666,81 +11263,92 @@ public final class MasterProcedureProtos {
       "n\022-\n\027unmodified_table_schema\030\002 \001(\0132\014.Tab" +
       "leSchema\022+\n\025modified_table_schema\030\003 \002(\0132" +
       "\014.TableSchema\022&\n\036delete_column_family_in" +
-      "_modify\030\004 \002(\010\"}\n\024DeleteTableStateData\022#\n",
-      "\tuser_info\030\001 \002(\0132\020.UserInformation\022\036\n\nta" +
-      "ble_name\030\002 \002(\0132\n.TableName\022 \n\013region_inf" +
-      "o\030\003 \003(\0132\013.RegionInfo\"\300\001\n\030AddColumnFamily" +
+      "_modify\030\004 \002(\010\"\274\001\n\026TruncateTableStateData",
+      "\022#\n\tuser_info\030\001 \002(\0132\020.UserInformation\022\027\n" +
+      "\017preserve_splits\030\002 \002(\010\022\036\n\ntable_name\030\003 \001" +
+      "(\0132\n.TableName\022\"\n\014table_schema\030\004 \001(\0132\014.T" +
+      "ableSchema\022 \n\013region_info\030\005 \003(\0132\013.Region" +
+      "Info\"}\n\024DeleteTableStateData\022#\n\tuser_inf" +
+      "o\030\001 \002(\0132\020.UserInformation\022\036\n\ntable_name\030" +
+      "\002 \002(\0132\n.TableName\022 \n\013region_info\030\003 \003(\0132\013" +
+      ".RegionInfo\"\300\001\n\030AddColumnFamilyStateData" +
+      "\022#\n\tuser_info\030\001 \002(\0132\020.UserInformation\022\036\n" +
+      "\ntable_name\030\002 \002(\0132\n.TableName\0220\n\023columnf",
+      "amily_schema\030\003 \002(\0132\023.ColumnFamilySchema\022" +
+      "-\n\027unmodified_table_schema\030\004 \001(\0132\014.Table" +
+      "Schema\"\303\001\n\033ModifyColumnFamilyStateData\022#" +
+      "\n\tuser_info\030\001 \002(\0132\020.UserInformation\022\036\n\nt" +
+      "able_name\030\002 \002(\0132\n.TableName\0220\n\023columnfam" +
+      "ily_schema\030\003 \002(\0132\023.ColumnFamilySchema\022-\n" +
+      "\027unmodified_table_schema\030\004 \001(\0132\014.TableSc" +
+      "hema\"\254\001\n\033DeleteColumnFamilyStateData\022#\n\t" +
+      "user_info\030\001 \002(\0132\020.UserInformation\022\036\n\ntab" +
+      "le_name\030\002 \002(\0132\n.TableName\022\031\n\021columnfamil",
+      "y_name\030\003 \002(\014\022-\n\027unmodified_table_schema\030" +
+      "\004 \001(\0132\014.TableSchema\"{\n\024EnableTableStateD" +
+      "ata\022#\n\tuser_info\030\001 \002(\0132\020.UserInformation" +
+      "\022\036\n\ntable_name\030\002 \002(\0132\n.TableName\022\036\n\026skip" +
+      "_table_state_check\030\003 \002(\010\"|\n\025DisableTable" +
       "StateData\022#\n\tuser_info\030\001 \002(\0132\020.UserInfor" +
-      "mation\022\036\n\ntable_name\030\002 \002(\0132\n.TableName\0220" +
-      "\n\023columnfamily_schema\030\003 \002(\0132\023.ColumnFami" +
-      "lySchema\022-\n\027unmodified_table_schema\030\004 \001(" +
-      "\0132\014.TableSchema\"\303\001\n\033ModifyColumnFamilySt" +
-      "ateData\022#\n\tuser_info\030\001 \002(\0132\020.UserInforma" +
-      "tion\022\036\n\ntable_name\030\002 \002(\0132\n.TableName\0220\n\023",
-      "columnfamily_schema\030\003 \002(\0132\023.ColumnFamily" +
-      "Schema\022-\n\027unmodified_table_schema\030\004 \001(\0132" +
-      "\014.TableSchema\"\254\001\n\033DeleteColumnFamilyStat" +
-      "eData\022#\n\tuser_info\030\001 \002(\0132\020.UserInformati" +
-      "on\022\036\n\ntable_name\030\002 \002(\0132\n.TableName\022\031\n\021co" +
-      "lumnfamily_name\030\003 \002(\014\022-\n\027unmodified_tabl" +
-      "e_schema\030\004 \001(\0132\014.TableSchema\"{\n\024EnableTa" +
-      "bleStateData\022#\n\tuser_info\030\001 \002(\0132\020.UserIn" +
-      "formation\022\036\n\ntable_name\030\002 \002(\0132\n.TableNam" +
-      "e\022\036\n\026skip_table_state_check\030\003 \002(\010\"|\n\025Dis",
-      "ableTableStateData\022#\n\tuser_info\030\001 \002(\0132\020." +
-      "UserInformation\022\036\n\ntable_name\030\002 \002(\0132\n.Ta" +
-      "bleName\022\036\n\026skip_table_state_check\030\003 \002(\010*" +
-      "\330\001\n\020CreateTableState\022\036\n\032CREATE_TABLE_PRE" +
-      "_OPERATION\020\001\022 \n\034CREATE_TABLE_WRITE_FS_LA" +
-      "YOUT\020\002\022\034\n\030CREATE_TABLE_ADD_TO_META\020\003\022\037\n\033" +
-      "CREATE_TABLE_ASSIGN_REGIONS\020\004\022\"\n\036CREATE_" +
-      "TABLE_UPDATE_DESC_CACHE\020\005\022\037\n\033CREATE_TABL" +
-      "E_POST_OPERATION\020\006*\207\002\n\020ModifyTableState\022" +
-      "\030\n\024MODIFY_TABLE_PREPARE\020\001\022\036\n\032MODIFY_TABL",
-      "E_PRE_OPERATION\020\002\022(\n$MODIFY_TABLE_UPDATE" +
-      "_TABLE_DESCRIPTOR\020\003\022&\n\"MODIFY_TABLE_REMO" +
-      "VE_REPLICA_COLUMN\020\004\022!\n\035

<TRUNCATED>

[7/9] hbase git commit: HBASE-13455 Procedure V2 - master truncate table

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/4788c6d1/hbase-protocol/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/MasterProcedure.proto b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
index a9ad0e0..e1c6880 100644
--- a/hbase-protocol/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
@@ -75,6 +75,24 @@ message ModifyTableStateData {
   required bool delete_column_family_in_modify = 4;
 }
 
+enum TruncateTableState {
+  TRUNCATE_TABLE_PRE_OPERATION = 1;
+  TRUNCATE_TABLE_REMOVE_FROM_META = 2;
+  TRUNCATE_TABLE_CLEAR_FS_LAYOUT = 3;
+  TRUNCATE_TABLE_CREATE_FS_LAYOUT = 4;
+  TRUNCATE_TABLE_ADD_TO_META = 5;
+  TRUNCATE_TABLE_ASSIGN_REGIONS = 6;
+  TRUNCATE_TABLE_POST_OPERATION = 7;
+}
+
+message TruncateTableStateData {
+  required UserInformation user_info = 1;
+  required bool preserve_splits = 2;
+  optional TableName table_name = 3;
+  optional TableSchema table_schema = 4;
+  repeated RegionInfo region_info = 5;
+}
+
 enum DeleteTableState {
   DELETE_TABLE_PRE_OPERATION = 1;
   DELETE_TABLE_REMOVE_FROM_META = 2;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4788c6d1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index fdbc31c..45bcdcb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -88,7 +88,6 @@ import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
 import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
-import org.apache.hadoop.hbase.master.handler.TruncateTableHandler;
 import org.apache.hadoop.hbase.master.procedure.AddColumnFamilyProcedure;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
@@ -101,6 +100,7 @@ import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
 import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
+import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@@ -1599,9 +1599,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       cpHost.preTruncateTable(tableName);
     }
     LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
-    TruncateTableHandler handler = new TruncateTableHandler(tableName, this, this, preserveSplits);
-    handler.prepare();
-    handler.process();
+
+    long procId = this.procedureExecutor.submitProcedure(
+        new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName, preserveSplits));
+    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+
     if (cpHost != null) {
       cpHost.postTruncateTable(tableName);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4788c6d1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
index 84e9bef..2582a1e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
@@ -106,14 +106,15 @@ public class DeleteTableProcedure
             return Flow.NO_MORE_STATE;
           }
 
-          preDelete(env);
-
           // TODO: Move out... in the acquireLock()
           LOG.debug("waiting for '" + getTableName() + "' regions in transition");
           regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
           assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
           ProcedureSyncWait.waitRegionInTransition(env, regions);
 
+          // Call coprocessors
+          preDelete(env);
+
           setNextState(DeleteTableState.DELETE_TABLE_REMOVE_FROM_META);
           break;
         case DELETE_TABLE_REMOVE_FROM_META:

http://git-wip-us.apache.org/repos/asf/hbase/blob/4788c6d1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
new file mode 100644
index 0000000..5ef0a19
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
@@ -0,0 +1,291 @@
+/**
+ * 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.hbase.master.procedure;
+
+import java.io.InputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotDisabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.exceptions.HBaseException;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+
+@InterfaceAudience.Private
+public class TruncateTableProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, TruncateTableState>
+    implements TableProcedureInterface {
+  private static final Log LOG = LogFactory.getLog(TruncateTableProcedure.class);
+
+  private boolean preserveSplits;
+  private List<HRegionInfo> regions;
+  private UserGroupInformation user;
+  private HTableDescriptor hTableDescriptor;
+  private TableName tableName;
+
+  public TruncateTableProcedure() {
+    // Required by the Procedure framework to create the procedure on replay
+  }
+
+  public TruncateTableProcedure(final MasterProcedureEnv env, final TableName tableName,
+      boolean preserveSplits) throws IOException {
+    this.tableName = tableName;
+    this.preserveSplits = preserveSplits;
+    this.user = env.getRequestUser().getUGI();
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, TruncateTableState state) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+    try {
+      switch (state) {
+        case TRUNCATE_TABLE_PRE_OPERATION:
+          // Verify if we can truncate the table
+          if (!prepareTruncate(env)) {
+            assert isFailed() : "the truncate should have an exception here";
+            return Flow.NO_MORE_STATE;
+          }
+
+          // TODO: Move out... in the acquireLock()
+          LOG.debug("waiting for '" + getTableName() + "' regions in transition");
+          regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+          assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
+          ProcedureSyncWait.waitRegionInTransition(env, regions);
+
+          // Call coprocessors
+          preTruncate(env);
+
+          setNextState(TruncateTableState.TRUNCATE_TABLE_REMOVE_FROM_META);
+          break;
+        case TRUNCATE_TABLE_REMOVE_FROM_META:
+          hTableDescriptor = env.getMasterServices().getTableDescriptors()
+              .getDescriptor(tableName).getHTableDescriptor();
+          DeleteTableProcedure.deleteFromMeta(env, getTableName(), regions);
+          DeleteTableProcedure.deleteAssignmentState(env, getTableName());
+          setNextState(TruncateTableState.TRUNCATE_TABLE_CLEAR_FS_LAYOUT);
+          break;
+        case TRUNCATE_TABLE_CLEAR_FS_LAYOUT:
+          DeleteTableProcedure.deleteFromFs(env, getTableName(), regions, true);
+          if (!preserveSplits) {
+            // if we are not preserving splits, generate a new single region
+            regions = Arrays.asList(ModifyRegionUtils.createHRegionInfos(hTableDescriptor, null));
+          }
+          setNextState(TruncateTableState.TRUNCATE_TABLE_CREATE_FS_LAYOUT);
+          break;
+        case TRUNCATE_TABLE_CREATE_FS_LAYOUT:
+          regions = CreateTableProcedure.createFsLayout(env, hTableDescriptor, regions);
+          CreateTableProcedure.updateTableDescCache(env, getTableName());
+          setNextState(TruncateTableState.TRUNCATE_TABLE_ADD_TO_META);
+          break;
+        case TRUNCATE_TABLE_ADD_TO_META:
+          regions = CreateTableProcedure.addTableToMeta(env, hTableDescriptor, regions);
+          setNextState(TruncateTableState.TRUNCATE_TABLE_ASSIGN_REGIONS);
+          break;
+        case TRUNCATE_TABLE_ASSIGN_REGIONS:
+          CreateTableProcedure.assignRegions(env, getTableName(), regions);
+          setNextState(TruncateTableState.TRUNCATE_TABLE_POST_OPERATION);
+          hTableDescriptor = null;
+          regions = null;
+          break;
+        case TRUNCATE_TABLE_POST_OPERATION:
+          postTruncate(env);
+          LOG.debug("truncate '" + getTableName() + "' completed");
+          return Flow.NO_MORE_STATE;
+        default:
+          throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (HBaseException|IOException e) {
+      LOG.warn("Retriable error trying to truncate table=" + getTableName() + " state=" + state, e);
+    } catch (InterruptedException e) {
+      // if the interrupt is real, the executor will be stopped.
+      LOG.warn("Interrupted trying to truncate table=" + getTableName() + " state=" + state, e);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final TruncateTableState state) {
+    if (state == TruncateTableState.TRUNCATE_TABLE_PRE_OPERATION) {
+      // nothing to rollback, pre-truncate is just table-state checks.
+      // We can fail if the table does not exist or is not disabled.
+      return;
+    }
+
+    // The truncate doesn't have a rollback. The execution will succeed, at some point.
+    throw new UnsupportedOperationException("unhandled state=" + state);
+  }
+
+  @Override
+  protected TruncateTableState getState(final int stateId) {
+    return TruncateTableState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final TruncateTableState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected TruncateTableState getInitialState() {
+    return TruncateTableState.TRUNCATE_TABLE_PRE_OPERATION;
+  }
+
+  @Override
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.EDIT;
+  }
+
+  @Override
+  public boolean abort(final MasterProcedureEnv env) {
+    // TODO: We may be able to abort if the procedure is not started yet.
+    return false;
+  }
+
+  @Override
+  protected boolean acquireLock(final MasterProcedureEnv env) {
+    if (!env.isInitialized()) return false;
+    return env.getProcedureQueue().tryAcquireTableWrite(getTableName(), "truncate table");
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureQueue().releaseTableWrite(getTableName());
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append(" (table=");
+    sb.append(getTableName());
+    sb.append(" preserveSplits=");
+    sb.append(preserveSplits);
+    sb.append(") user=");
+    sb.append(user);
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    MasterProcedureProtos.TruncateTableStateData.Builder state =
+      MasterProcedureProtos.TruncateTableStateData.newBuilder()
+        .setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
+        .setPreserveSplits(preserveSplits);
+    if (hTableDescriptor != null) {
+      state.setTableSchema(hTableDescriptor.convert());
+    } else {
+      state.setTableName(ProtobufUtil.toProtoTableName(tableName));
+    }
+    if (regions != null) {
+      for (HRegionInfo hri: regions) {
+        state.addRegionInfo(HRegionInfo.convert(hri));
+      }
+    }
+    state.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    MasterProcedureProtos.TruncateTableStateData state =
+      MasterProcedureProtos.TruncateTableStateData.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
+    if (state.hasTableSchema()) {
+      hTableDescriptor = HTableDescriptor.convert(state.getTableSchema());
+      tableName = hTableDescriptor.getTableName();
+    } else {
+      tableName = ProtobufUtil.toTableName(state.getTableName());
+    }
+    preserveSplits = state.getPreserveSplits();
+    if (state.getRegionInfoCount() == 0) {
+      regions = null;
+    } else {
+      regions = new ArrayList<HRegionInfo>(state.getRegionInfoCount());
+      for (HBaseProtos.RegionInfo hri: state.getRegionInfoList()) {
+        regions.add(HRegionInfo.convert(hri));
+      }
+    }
+  }
+
+  private boolean prepareTruncate(final MasterProcedureEnv env) throws IOException {
+    try {
+      env.getMasterServices().checkTableModifiable(getTableName());
+    } catch (TableNotFoundException|TableNotDisabledException e) {
+      setFailure("master-truncate-table", e);
+      return false;
+    }
+    return true;
+  }
+
+  private boolean preTruncate(final MasterProcedureEnv env)
+      throws IOException, InterruptedException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      final TableName tableName = getTableName();
+      user.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          cpHost.preTruncateTableHandler(tableName);
+          return null;
+        }
+      });
+    }
+    return true;
+  }
+
+  private void postTruncate(final MasterProcedureEnv env)
+      throws IOException, InterruptedException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      final TableName tableName = getTableName();
+      user.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          cpHost.postTruncateTableHandler(tableName);
+          return null;
+        }
+      });
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4788c6d1/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index ff79569..9cb0d57c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -2291,6 +2291,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   }
 
   /**
+   * Return the number of rows in the given table.
+   */
+  public int countRows(final TableName tableName) throws IOException {
+    Table table = getConnection().getTable(tableName);
+    try {
+      return countRows(table);
+    } finally {
+      table.close();
+    }
+  }
+
+  /**
    * Return an md5 digest of the entire contents of a table.
    */
   public String checksumRows(final Table table) throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4788c6d1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index 9bb436e..57a15e8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -35,6 +35,10 @@ import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableDescriptor;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -44,6 +48,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.MD5Hash;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -85,6 +90,7 @@ public class MasterProcedureTestingUtility {
     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
     final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
     assertTrue(fs.exists(tableDir));
+    FSUtils.logFileSystemState(fs, tableDir, LOG);
     List<Path> allRegionDirs = FSUtils.getRegionDirs(fs, tableDir);
     for (int i = 0; i < regions.length; ++i) {
       Path regionDir = new Path(tableDir, regions[i].getEncodedName());
@@ -343,6 +349,43 @@ public class MasterProcedureTestingUtility {
     assertTrue(hcfd.equals(columnDescriptor));
   }
 
+  public static void loadData(final Connection connection, final TableName tableName,
+      int rows, final byte[][] splitKeys,  final String... sfamilies) throws IOException {
+    byte[][] families = new byte[sfamilies.length][];
+    for (int i = 0; i < families.length; ++i) {
+      families[i] = Bytes.toBytes(sfamilies[i]);
+    }
+
+    BufferedMutator mutator = connection.getBufferedMutator(tableName);
+
+    // Ensure one row per region
+    assertTrue(rows >= splitKeys.length);
+    for (byte[] k: splitKeys) {
+      byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), k);
+      byte[] key = Bytes.add(k, Bytes.toBytes(MD5Hash.getMD5AsHex(value)));
+      mutator.mutate(createPut(families, key, value));
+      rows--;
+    }
+
+    // Add other extra rows. more rows, more files
+    while (rows-- > 0) {
+      byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), Bytes.toBytes(rows));
+      byte[] key = Bytes.toBytes(MD5Hash.getMD5AsHex(value));
+      mutator.mutate(createPut(families, key, value));
+    }
+    mutator.flush();
+  }
+
+  private static Put createPut(final byte[][] families, final byte[] key, final byte[] value) {
+    byte[] q = Bytes.toBytes("q");
+    Put put = new Put(key);
+    put.setDurability(Durability.SKIP_WAL);
+    for (byte[] family: families) {
+      put.add(family, q, value);
+    }
+    return put;
+  }
+
   public static class InjectAbortOnLoadListener
       implements ProcedureExecutor.ProcedureExecutorListener {
     private final ProcedureExecutor<MasterProcedureEnv> procExec;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4788c6d1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
index 0f6c910..2576302 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTa
 import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -215,6 +216,67 @@ public class TestMasterFailoverWithProcedures {
   }
 
   // ==========================================================================
+  //  Test Truncate Table
+  // ==========================================================================
+  @Test(timeout=90000)
+  public void testTruncateWithFailover() throws Exception {
+    // TODO: Should we try every step? (master failover takes long time)
+    // It is already covered by TestTruncateTableProcedure
+    // but without the master restart, only the executor/store is restarted.
+    // Without Master restart we may not find bug in the procedure code
+    // like missing "wait" for resources to be available (e.g. RS)
+    testTruncateWithFailoverAtStep(true, TruncateTableState.TRUNCATE_TABLE_ADD_TO_META.ordinal());
+  }
+
+  private void testTruncateWithFailoverAtStep(final boolean preserveSplits, final int step)
+      throws Exception {
+    final TableName tableName = TableName.valueOf("testTruncateWithFailoverAtStep" + step);
+
+    // create the table
+    final String[] families = new String[] { "f1", "f2" };
+    final byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      getMasterProcedureExecutor(), tableName, splitKeys, families);
+    // load and verify that there are rows in the table
+    MasterProcedureTestingUtility.loadData(
+      UTIL.getConnection(), tableName, 100, splitKeys, families);
+    assertEquals(100, UTIL.countRows(tableName));
+    // disable the table
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Truncate procedure && kill the executor
+    long procId = procExec.submitProcedure(
+      new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits));
+    testRecoveryAndDoubleExecution(UTIL, procId, step, TruncateTableState.values());
+
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+    UTIL.waitUntilAllRegionsAssigned(tableName);
+
+    // validate the table regions and layout
+    if (preserveSplits) {
+      assertEquals(1 + splitKeys.length, UTIL.getHBaseAdmin().getTableRegions(tableName).size());
+    } else {
+      regions = UTIL.getHBaseAdmin().getTableRegions(tableName).toArray(new HRegionInfo[1]);
+      assertEquals(1, regions.length);
+    }
+    MasterProcedureTestingUtility.validateTableCreation(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, families);
+
+    // verify that there are no rows in the table
+    assertEquals(0, UTIL.countRows(tableName));
+
+    // verify that the table is read/writable
+    MasterProcedureTestingUtility.loadData(
+      UTIL.getConnection(), tableName, 50, splitKeys, families);
+    assertEquals(50, UTIL.countRows(tableName));
+  }
+
+  // ==========================================================================
   //  Test Disable Table
   // ==========================================================================
   @Test(timeout=60000)

http://git-wip-us.apache.org/repos/asf/hbase/blob/4788c6d1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
new file mode 100644
index 0000000..58acbae
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
@@ -0,0 +1,246 @@
+/**
+ * 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.hbase.master.procedure;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotDisabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureResult;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestTruncateTableProcedure {
+  private static final Log LOG = LogFactory.getLog(TestTruncateTableProcedure.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static void setupConf(Configuration conf) {
+    conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  @Before
+  public void setup() throws Exception {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+    assertTrue("expected executor to be running", procExec.isRunning());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
+    for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
+      LOG.info("Tear down, remove table=" + htd.getTableName());
+      UTIL.deleteTable(htd.getTableName());
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testTruncateNotExistentTable() throws Exception {
+    final TableName tableName = TableName.valueOf("testTruncateNotExistentTable");
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    long procId = ProcedureTestingUtility.submitAndWait(procExec,
+        new TruncateTableProcedure(procExec.getEnvironment(), tableName, true));
+
+    // Second delete should fail with TableNotFound
+    ProcedureResult result = procExec.getResult(procId);
+    assertTrue(result.isFailed());
+    LOG.debug("Truncate failed with exception: " + result.getException());
+    assertTrue(result.getException().getCause() instanceof TableNotFoundException);
+  }
+
+  @Test(timeout=60000)
+  public void testTruncateNotDisabledTable() throws Exception {
+    final TableName tableName = TableName.valueOf("testTruncateNotDisabledTable");
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f");
+
+    long procId = ProcedureTestingUtility.submitAndWait(procExec,
+        new TruncateTableProcedure(procExec.getEnvironment(), tableName, false));
+
+    // Second delete should fail with TableNotDisabled
+    ProcedureResult result = procExec.getResult(procId);
+    assertTrue(result.isFailed());
+    LOG.debug("Truncate failed with exception: " + result.getException());
+    assertTrue(result.getException().getCause() instanceof TableNotDisabledException);
+  }
+
+  @Test(timeout=60000)
+  public void testSimpleTruncatePreserveSplits() throws Exception {
+    final TableName tableName = TableName.valueOf("testSimpleTruncatePreserveSplits");
+    testSimpleTruncate(tableName, true);
+  }
+
+  @Test(timeout=60000)
+  public void testSimpleTruncateNoPreserveSplits() throws Exception {
+    final TableName tableName = TableName.valueOf("testSimpleTruncateNoPreserveSplits");
+    testSimpleTruncate(tableName, false);
+  }
+
+  private void testSimpleTruncate(final TableName tableName, final boolean preserveSplits)
+      throws Exception {
+    final String[] families = new String[] { "f1", "f2" };
+    final byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      getMasterProcedureExecutor(), tableName, splitKeys, families);
+    // load and verify that there are rows in the table
+    MasterProcedureTestingUtility.loadData(
+      UTIL.getConnection(), tableName, 100, splitKeys, families);
+    assertEquals(100, UTIL.countRows(tableName));
+    // disable the table
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    // truncate the table
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    long procId = ProcedureTestingUtility.submitAndWait(procExec,
+      new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits));
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+
+    UTIL.waitUntilAllRegionsAssigned(tableName);
+
+    // validate the table regions and layout
+    if (preserveSplits) {
+      assertEquals(1 + splitKeys.length, UTIL.getHBaseAdmin().getTableRegions(tableName).size());
+    } else {
+      regions = UTIL.getHBaseAdmin().getTableRegions(tableName).toArray(new HRegionInfo[1]);
+      assertEquals(1, regions.length);
+    }
+    MasterProcedureTestingUtility.validateTableCreation(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, families);
+
+    // verify that there are no rows in the table
+    assertEquals(0, UTIL.countRows(tableName));
+
+    // verify that the table is read/writable
+    MasterProcedureTestingUtility.loadData(
+      UTIL.getConnection(), tableName, 50, splitKeys, families);
+    assertEquals(50, UTIL.countRows(tableName));
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecutionPreserveSplits() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionPreserveSplits");
+    testRecoveryAndDoubleExecution(tableName, true);
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecutionNoPreserveSplits() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionNoPreserveSplits");
+    testRecoveryAndDoubleExecution(tableName, false);
+  }
+
+  private void testRecoveryAndDoubleExecution(final TableName tableName,
+      final boolean preserveSplits) throws Exception {
+    final String[] families = new String[] { "f1", "f2" };
+
+    // create the table
+    final byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      getMasterProcedureExecutor(), tableName, splitKeys, families);
+    // load and verify that there are rows in the table
+    MasterProcedureTestingUtility.loadData(
+      UTIL.getConnection(), tableName, 100, splitKeys, families);
+    assertEquals(100, UTIL.countRows(tableName));
+    // disable the table
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Truncate procedure && kill the executor
+    long procId = procExec.submitProcedure(
+      new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits));
+
+    // Restart the executor and execute the step twice
+    // NOTE: the 7 (number of TruncateTableState steps) is hardcoded,
+    //       so you have to look at this test at least once when you add a new step.
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+      procExec, procId, 7, TruncateTableState.values());
+
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+    UTIL.waitUntilAllRegionsAssigned(tableName);
+
+    // validate the table regions and layout
+    if (preserveSplits) {
+      assertEquals(1 + splitKeys.length, UTIL.getHBaseAdmin().getTableRegions(tableName).size());
+    } else {
+      regions = UTIL.getHBaseAdmin().getTableRegions(tableName).toArray(new HRegionInfo[1]);
+      assertEquals(1, regions.length);
+    }
+    MasterProcedureTestingUtility.validateTableCreation(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, families);
+
+    // verify that there are no rows in the table
+    assertEquals(0, UTIL.countRows(tableName));
+
+    // verify that the table is read/writable
+    MasterProcedureTestingUtility.loadData(
+      UTIL.getConnection(), tableName, 50, splitKeys, families);
+    assertEquals(50, UTIL.countRows(tableName));
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+}


[6/9] hbase git commit: HBASE-13202 Procedure v2 - core framework (addendum)

Posted by mb...@apache.org.
HBASE-13202 Procedure v2 - core framework (addendum)


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

Branch: refs/heads/hbase-12439
Commit: d75326a7974881a41993e210b9c5b7d4b0fe5b8b
Parents: 4f15144
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Wed Apr 15 09:39:25 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Wed Apr 15 09:50:47 2015 +0100

----------------------------------------------------------------------
 .../procedure2/ProcedureFairRunQueues.java      |  1 +
 .../procedure2/store/ProcedureStoreTracker.java | 12 ++++++++--
 .../procedure2/store/wal/WALProcedureStore.java |  6 ++---
 .../store/TestProcedureStoreTracker.java        | 25 ++++++++++++++++++++
 4 files changed, 39 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d75326a7/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFairRunQueues.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFairRunQueues.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFairRunQueues.java
index 03d007a..242ae86 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFairRunQueues.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFairRunQueues.java
@@ -95,6 +95,7 @@ public class ProcedureFairRunQueues<TKey, TQueue extends ProcedureFairRunQueues.
   public void clear() {
     lock.lock();
     try {
+      currentQuantum = 0;
       current = null;
       objMap.clear();
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d75326a7/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java
index 4e4653a..a4711f1 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java
@@ -195,11 +195,12 @@ public class ProcedureStoreTracker {
     //  Grow/Merge Helpers
     // ========================================================================
     public boolean canGrow(final long procId) {
-      return (procId - start) < MAX_NODE_SIZE;
+      return Math.abs(procId - start) < MAX_NODE_SIZE;
     }
 
     public boolean canMerge(final BitSetNode rightNode) {
-      return (start + rightNode.getEnd()) < MAX_NODE_SIZE;
+      assert start < rightNode.getEnd();
+      return (rightNode.getEnd() - start) < MAX_NODE_SIZE;
     }
 
     public void grow(final long procId) {
@@ -258,6 +259,11 @@ public class ProcedureStoreTracker {
       }
     }
 
+    @Override
+    public String toString() {
+      return "BitSetNode(" + getStart() + "-" + getEnd() + ")";
+    }
+
     // ========================================================================
     //  Min/Max Helpers
     // ========================================================================
@@ -377,6 +383,7 @@ public class ProcedureStoreTracker {
   @InterfaceAudience.Private
   public void setDeleted(final long procId, final boolean isDeleted) {
     BitSetNode node = getOrCreateNode(procId);
+    assert node.contains(procId) : "expected procId in the node";
     node.updateState(procId, isDeleted);
   }
 
@@ -507,6 +514,7 @@ public class ProcedureStoreTracker {
   }
 
   private BitSetNode mergeNodes(BitSetNode leftNode, BitSetNode rightNode) {
+    assert leftNode.getStart() < rightNode.getStart();
     leftNode.merge(rightNode);
     map.remove(rightNode.getStart());
     return leftNode;

http://git-wip-us.apache.org/repos/asf/hbase/blob/d75326a7/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
index 13f7bfa..09d2f7a 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
@@ -389,7 +389,7 @@ public class WALProcedureStore implements ProcedureStore {
   }
 
   private long pushData(final ByteSlot slot) {
-    assert !logs.isEmpty() : "recoverLease() must be called before inserting data";
+    assert isRunning() && !logs.isEmpty() : "recoverLease() must be called before inserting data";
     long logId = -1;
 
     lock.lock();
@@ -677,7 +677,7 @@ public class WALProcedureStore implements ProcedureStore {
       try {
         log.readTracker(storeTracker);
       } catch (IOException e) {
-        LOG.error("Unable to read tracker for " + log, e);
+        LOG.warn("Unable to read tracker for " + log + " - " + e.getMessage());
         // try the next one...
         storeTracker.clear();
         storeTracker.setPartialFlag(true);
@@ -718,4 +718,4 @@ public class WALProcedureStore implements ProcedureStore {
     }
     return log;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d75326a7/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java
index 0669549..be759dc 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.procedure2.store;
 
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -165,4 +166,28 @@ public class TestProcedureStoreTracker {
     tracker.delete(procs[5].getProcId());
     assertTrue(tracker.isEmpty());
   }
+
+  @Test
+  public void testRandLoad() {
+    final int NPROCEDURES = 2500;
+    final int NRUNS = 5000;
+
+    final ProcedureStoreTracker tracker = new ProcedureStoreTracker();
+
+    Random rand = new Random(1);
+    for (int i = 0; i < NRUNS; ++i) {
+      assertTrue(tracker.isEmpty());
+
+      int count = 0;
+      while (count < NPROCEDURES) {
+        long procId = rand.nextLong();
+        if (procId < 1) continue;
+
+        tracker.setDeleted(procId, i % 2 == 0);
+        count++;
+      }
+
+      tracker.clear();
+    }
+  }
 }


[5/9] hbase git commit: HBASE-13301 Possible memory leak in BucketCache

Posted by mb...@apache.org.
HBASE-13301 Possible memory leak in BucketCache


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/4f151444
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4f151444
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4f151444

Branch: refs/heads/hbase-12439
Commit: 4f151444b58ae85b93f76254961358932e0ffb9b
Parents: 71536bd
Author: zhangduo <zh...@wandoujia.com>
Authored: Sat Apr 11 10:43:43 2015 +0800
Committer: zhangduo <zh...@wandoujia.com>
Committed: Tue Apr 14 17:41:46 2015 +0800

----------------------------------------------------------------------
 .../hbase/io/hfile/bucket/BucketCache.java      | 182 +++++++++++--------
 .../hbase/io/hfile/bucket/CachedEntryQueue.java |  20 +-
 .../org/apache/hadoop/hbase/util/IdLock.java    |  16 ++
 .../hadoop/hbase/io/hfile/CacheTestUtils.java   |   6 +-
 .../hbase/io/hfile/bucket/TestBucketCache.java  |  87 ++++++---
 5 files changed, 196 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4f151444/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index 7dda0e6..6a5c884 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -39,6 +39,7 @@ import java.util.Set;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
@@ -109,13 +110,14 @@ public class BucketCache implements BlockCache, HeapSize {
   final static int DEFAULT_WRITER_QUEUE_ITEMS = 64;
 
   // Store/read block data
-  IOEngine ioEngine;
+  final IOEngine ioEngine;
 
   // Store the block in this map before writing it to cache
   @VisibleForTesting
-  Map<BlockCacheKey, RAMQueueEntry> ramCache;
+  final ConcurrentMap<BlockCacheKey, RAMQueueEntry> ramCache;
   // In this map, store the block's meta data like offset, length
-  private Map<BlockCacheKey, BucketEntry> backingMap;
+  @VisibleForTesting
+  ConcurrentMap<BlockCacheKey, BucketEntry> backingMap;
 
   /**
    * Flag if the cache is enabled or not... We shut it off if there are IO
@@ -132,14 +134,14 @@ public class BucketCache implements BlockCache, HeapSize {
    * to the BucketCache.  It then updates the ramCache and backingMap accordingly.
    */
   @VisibleForTesting
-  ArrayList<BlockingQueue<RAMQueueEntry>> writerQueues =
+  final ArrayList<BlockingQueue<RAMQueueEntry>> writerQueues =
       new ArrayList<BlockingQueue<RAMQueueEntry>>();
   @VisibleForTesting
-  WriterThread writerThreads[];
+  final WriterThread[] writerThreads;
 
   /** Volatile boolean to track if free space is in process or not */
   private volatile boolean freeInProgress = false;
-  private Lock freeSpaceLock = new ReentrantLock();
+  private final Lock freeSpaceLock = new ReentrantLock();
 
   private UniqueIndexMap<Integer> deserialiserMap = new UniqueIndexMap<Integer>();
 
@@ -152,17 +154,16 @@ public class BucketCache implements BlockCache, HeapSize {
   /** Cache access count (sequential ID) */
   private final AtomicLong accessCount = new AtomicLong(0);
 
-  private final Object[] cacheWaitSignals;
   private static final int DEFAULT_CACHE_WAIT_TIME = 50;
   // Used in test now. If the flag is false and the cache speed is very fast,
   // bucket cache will skip some blocks when caching. If the flag is true, we
   // will wait blocks flushed to IOEngine for some time when caching
   boolean wait_when_cache = false;
 
-  private BucketCacheStats cacheStats = new BucketCacheStats();
+  private final BucketCacheStats cacheStats = new BucketCacheStats();
 
-  private String persistencePath;
-  private long cacheCapacity;
+  private final String persistencePath;
+  private final long cacheCapacity;
   /** Approximate block size */
   private final long blockSize;
 
@@ -182,7 +183,8 @@ public class BucketCache implements BlockCache, HeapSize {
    *
    * TODO:We could extend the IdLock to IdReadWriteLock for better.
    */
-  private IdLock offsetLock = new IdLock();
+  @VisibleForTesting
+  final IdLock offsetLock = new IdLock();
 
   private final ConcurrentIndex<String, BlockCacheKey> blocksByHFile =
       new ConcurrentIndex<String, BlockCacheKey>(new Comparator<BlockCacheKey>() {
@@ -216,7 +218,6 @@ public class BucketCache implements BlockCache, HeapSize {
       throws FileNotFoundException, IOException {
     this.ioEngine = getIOEngineFromName(ioEngineName, capacity);
     this.writerThreads = new WriterThread[writerThreadNum];
-    this.cacheWaitSignals = new Object[writerThreadNum];
     long blockNumCapacity = capacity / blockSize;
     if (blockNumCapacity >= Integer.MAX_VALUE) {
       // Enough for about 32TB of cache!
@@ -231,7 +232,6 @@ public class BucketCache implements BlockCache, HeapSize {
     bucketAllocator = new BucketAllocator(capacity, bucketSizes);
     for (int i = 0; i < writerThreads.length; ++i) {
       writerQueues.add(new ArrayBlockingQueue<RAMQueueEntry>(writerQLen));
-      this.cacheWaitSignals[i] = new Object();
     }
 
     assert writerQueues.size() == writerThreads.length;
@@ -252,7 +252,7 @@ public class BucketCache implements BlockCache, HeapSize {
     final String threadName = Thread.currentThread().getName();
     this.cacheEnabled = true;
     for (int i = 0; i < writerThreads.length; ++i) {
-      writerThreads[i] = new WriterThread(writerQueues.get(i), i);
+      writerThreads[i] = new WriterThread(writerQueues.get(i));
       writerThreads[i].setName(threadName + "-BucketCacheWriter-" + i);
       writerThreads[i].setDaemon(true);
     }
@@ -344,38 +344,39 @@ public class BucketCache implements BlockCache, HeapSize {
    * @param inMemory if block is in-memory
    * @param wait if true, blocking wait when queue is full
    */
-  public void cacheBlockWithWait(BlockCacheKey cacheKey, Cacheable cachedItem,
-      boolean inMemory, boolean wait) {
-    if (!cacheEnabled)
+  public void cacheBlockWithWait(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory,
+      boolean wait) {
+    if (!cacheEnabled) {
       return;
+    }
 
-    if (backingMap.containsKey(cacheKey) || ramCache.containsKey(cacheKey))
+    if (backingMap.containsKey(cacheKey)) {
       return;
+    }
 
     /*
-     * Stuff the entry into the RAM cache so it can get drained to the
-     * persistent store
+     * Stuff the entry into the RAM cache so it can get drained to the persistent store
      */
-    RAMQueueEntry re = new RAMQueueEntry(cacheKey, cachedItem,
-        accessCount.incrementAndGet(), inMemory);
-    ramCache.put(cacheKey, re);
+    RAMQueueEntry re =
+        new RAMQueueEntry(cacheKey, cachedItem, accessCount.incrementAndGet(), inMemory);
+    if (ramCache.putIfAbsent(cacheKey, re) != null) {
+      return;
+    }
     int queueNum = (cacheKey.hashCode() & 0x7FFFFFFF) % writerQueues.size();
     BlockingQueue<RAMQueueEntry> bq = writerQueues.get(queueNum);
-    boolean successfulAddition = bq.offer(re);
-    if (!successfulAddition && wait) {
-      synchronized (cacheWaitSignals[queueNum]) {
-        try {
-          successfulAddition = bq.offer(re);
-          if (!successfulAddition) cacheWaitSignals[queueNum].wait(DEFAULT_CACHE_WAIT_TIME);
-        } catch (InterruptedException ie) {
-          Thread.currentThread().interrupt();
-        }
+    boolean successfulAddition = false;
+    if (wait) {
+      try {
+        successfulAddition = bq.offer(re, DEFAULT_CACHE_WAIT_TIME, TimeUnit.MILLISECONDS);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
       }
+    } else {
       successfulAddition = bq.offer(re);
     }
     if (!successfulAddition) {
-        ramCache.remove(cacheKey);
-        failedBlockAdditions.incrementAndGet();
+      ramCache.remove(cacheKey);
+      failedBlockAdditions.incrementAndGet();
     } else {
       this.blockNumber.incrementAndGet();
       this.heapSize.addAndGet(cachedItem.heapSize());
@@ -394,11 +395,14 @@ public class BucketCache implements BlockCache, HeapSize {
   @Override
   public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat,
       boolean updateCacheMetrics) {
-    if (!cacheEnabled)
+    if (!cacheEnabled) {
       return null;
+    }
     RAMQueueEntry re = ramCache.get(key);
     if (re != null) {
-      if (updateCacheMetrics) cacheStats.hit(caching);
+      if (updateCacheMetrics) {
+        cacheStats.hit(caching);
+      }
       re.access(accessCount.incrementAndGet());
       return re.getData();
     }
@@ -408,6 +412,9 @@ public class BucketCache implements BlockCache, HeapSize {
       IdLock.Entry lockEntry = null;
       try {
         lockEntry = offsetLock.getLockEntry(bucketEntry.offset());
+        // We can not read here even if backingMap does contain the given key because its offset
+        // maybe changed. If we lock BlockCacheKey instead of offset, then we can only check
+        // existence here.
         if (bucketEntry.equals(backingMap.get(key))) {
           int len = bucketEntry.getLength();
           ByteBuffer bb = ByteBuffer.allocate(len);
@@ -438,13 +445,27 @@ public class BucketCache implements BlockCache, HeapSize {
         }
       }
     }
-    if (!repeat && updateCacheMetrics) cacheStats.miss(caching);
+    if (!repeat && updateCacheMetrics) {
+      cacheStats.miss(caching);
+    }
     return null;
   }
 
+  @VisibleForTesting
+  void blockEvicted(BlockCacheKey cacheKey, BucketEntry bucketEntry, boolean decrementBlockNumber) {
+    bucketAllocator.freeBlock(bucketEntry.offset());
+    realCacheSize.addAndGet(-1 * bucketEntry.getLength());
+    blocksByHFile.remove(cacheKey.getHfileName(), cacheKey);
+    if (decrementBlockNumber) {
+      this.blockNumber.decrementAndGet();
+    }
+  }
+
   @Override
   public boolean evictBlock(BlockCacheKey cacheKey) {
-    if (!cacheEnabled) return false;
+    if (!cacheEnabled) {
+      return false;
+    }
     RAMQueueEntry removedBlock = ramCache.remove(cacheKey);
     if (removedBlock != null) {
       this.blockNumber.decrementAndGet();
@@ -462,13 +483,8 @@ public class BucketCache implements BlockCache, HeapSize {
     IdLock.Entry lockEntry = null;
     try {
       lockEntry = offsetLock.getLockEntry(bucketEntry.offset());
-      if (bucketEntry.equals(backingMap.remove(cacheKey))) {
-        bucketAllocator.freeBlock(bucketEntry.offset());
-        realCacheSize.addAndGet(-1 * bucketEntry.getLength());
-        blocksByHFile.remove(cacheKey.getHfileName(), cacheKey);
-        if (removedBlock == null) {
-          this.blockNumber.decrementAndGet();
-        }
+      if (backingMap.remove(cacheKey, bucketEntry)) {
+        blockEvicted(cacheKey, bucketEntry, removedBlock == null);
       } else {
         return false;
       }
@@ -705,13 +721,10 @@ public class BucketCache implements BlockCache, HeapSize {
   @VisibleForTesting
   class WriterThread extends HasThread {
     private final BlockingQueue<RAMQueueEntry> inputQueue;
-    private final int threadNO;
     private volatile boolean writerEnabled = true;
 
-    WriterThread(BlockingQueue<RAMQueueEntry> queue, int threadNO) {
-      super();
+    WriterThread(BlockingQueue<RAMQueueEntry> queue) {
       this.inputQueue = queue;
-      this.threadNO = threadNO;
     }
 
     // Used for test
@@ -728,9 +741,6 @@ public class BucketCache implements BlockCache, HeapSize {
             try {
               // Blocks
               entries = getRAMQueueEntries(inputQueue, entries);
-              synchronized (cacheWaitSignals[threadNO]) {
-                cacheWaitSignals[threadNO].notifyAll();
-              }
             } catch (InterruptedException ie) {
               if (!cacheEnabled) break;
             }
@@ -755,7 +765,9 @@ public class BucketCache implements BlockCache, HeapSize {
      */
     @VisibleForTesting
     void doDrain(final List<RAMQueueEntry> entries) throws InterruptedException {
-      if (entries.isEmpty()) return;
+      if (entries.isEmpty()) {
+        return;
+      }
       // This method is a little hard to follow. We run through the passed in entries and for each
       // successful add, we add a non-null BucketEntry to the below bucketEntries.  Later we must
       // do cleanup making sure we've cleared ramCache of all entries regardless of whether we
@@ -830,6 +842,21 @@ public class BucketCache implements BlockCache, HeapSize {
         RAMQueueEntry ramCacheEntry = ramCache.remove(key);
         if (ramCacheEntry != null) {
           heapSize.addAndGet(-1 * entries.get(i).getData().heapSize());
+        } else if (bucketEntries[i] != null){
+          // Block should have already been evicted. Remove it and free space.
+          IdLock.Entry lockEntry = null;
+          try {
+            lockEntry = offsetLock.getLockEntry(bucketEntries[i].offset());
+            if (backingMap.remove(key, bucketEntries[i])) {
+              blockEvicted(key, bucketEntries[i], false);
+            }
+          } catch (IOException e) {
+            LOG.warn("failed to free space for " + key, e);
+          } finally {
+            if (lockEntry != null) {
+              offsetLock.releaseLockEntry(lockEntry);
+            }
+          }
         }
       }
 
@@ -1055,23 +1082,35 @@ public class BucketCache implements BlockCache, HeapSize {
    * up the long. Doubt we'll see devices this big for ages. Offsets are divided
    * by 256. So 5 bytes gives us 256TB or so.
    */
-  static class BucketEntry implements Serializable, Comparable<BucketEntry> {
+  static class BucketEntry implements Serializable {
     private static final long serialVersionUID = -6741504807982257534L;
+
+    // access counter comparator, descending order
+    static final Comparator<BucketEntry> COMPARATOR = new Comparator<BucketCache.BucketEntry>() {
+
+      @Override
+      public int compare(BucketEntry o1, BucketEntry o2) {
+        long accessCounter1 = o1.accessCounter;
+        long accessCounter2 = o2.accessCounter;
+        return accessCounter1 < accessCounter2 ? 1 : accessCounter1 == accessCounter2 ? 0 : -1;
+      }
+    };
+
     private int offsetBase;
     private int length;
     private byte offset1;
     byte deserialiserIndex;
-    private volatile long accessTime;
+    private volatile long accessCounter;
     private BlockPriority priority;
     /**
      * Time this block was cached.  Presumes we are created just before we are added to the cache.
      */
     private final long cachedTime = System.nanoTime();
 
-    BucketEntry(long offset, int length, long accessTime, boolean inMemory) {
+    BucketEntry(long offset, int length, long accessCounter, boolean inMemory) {
       setOffset(offset);
       this.length = length;
-      this.accessTime = accessTime;
+      this.accessCounter = accessCounter;
       if (inMemory) {
         this.priority = BlockPriority.MEMORY;
       } else {
@@ -1110,10 +1149,10 @@ public class BucketCache implements BlockCache, HeapSize {
     }
 
     /**
-     * Block has been accessed. Update its local access time.
+     * Block has been accessed. Update its local access counter.
      */
-    public void access(long accessTime) {
-      this.accessTime = accessTime;
+    public void access(long accessCounter) {
+      this.accessCounter = accessCounter;
       if (this.priority == BlockPriority.SINGLE) {
         this.priority = BlockPriority.MULTI;
       }
@@ -1123,17 +1162,6 @@ public class BucketCache implements BlockCache, HeapSize {
       return this.priority;
     }
 
-    @Override
-    public int compareTo(BucketEntry that) {
-      if(this.accessTime == that.accessTime) return 0;
-      return this.accessTime < that.accessTime ? 1 : -1;
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      return this == that;
-    }
-
     public long getCachedTime() {
       return cachedTime;
     }
@@ -1204,14 +1232,14 @@ public class BucketCache implements BlockCache, HeapSize {
   static class RAMQueueEntry {
     private BlockCacheKey key;
     private Cacheable data;
-    private long accessTime;
+    private long accessCounter;
     private boolean inMemory;
 
-    public RAMQueueEntry(BlockCacheKey bck, Cacheable data, long accessTime,
+    public RAMQueueEntry(BlockCacheKey bck, Cacheable data, long accessCounter,
         boolean inMemory) {
       this.key = bck;
       this.data = data;
-      this.accessTime = accessTime;
+      this.accessCounter = accessCounter;
       this.inMemory = inMemory;
     }
 
@@ -1223,8 +1251,8 @@ public class BucketCache implements BlockCache, HeapSize {
       return key;
     }
 
-    public void access(long accessTime) {
-      this.accessTime = accessTime;
+    public void access(long accessCounter) {
+      this.accessCounter = accessCounter;
     }
 
     public BucketEntry writeToCache(final IOEngine ioEngine,
@@ -1236,7 +1264,7 @@ public class BucketCache implements BlockCache, HeapSize {
       // This cacheable thing can't be serialized...
       if (len == 0) return null;
       long offset = bucketAllocator.allocateBlock(len);
-      BucketEntry bucketEntry = new BucketEntry(offset, len, accessTime, inMemory);
+      BucketEntry bucketEntry = new BucketEntry(offset, len, accessCounter, inMemory);
       bucketEntry.setDeserialiserReference(data.getDeserializer(), deserialiserMap);
       try {
         if (data instanceof HFileBlock) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4f151444/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CachedEntryQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CachedEntryQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CachedEntryQueue.java
index b6954bb..0e33a56 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CachedEntryQueue.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CachedEntryQueue.java
@@ -54,23 +54,23 @@ public class CachedEntryQueue {
    */
   public CachedEntryQueue(long maxSize, long blockSize) {
     int initialSize = (int) (maxSize / blockSize);
-    if (initialSize == 0)
+    if (initialSize == 0) {
       initialSize++;
-    queue = MinMaxPriorityQueue
-        .orderedBy(new Comparator<Map.Entry<BlockCacheKey, BucketEntry>>() {
-          public int compare(Entry<BlockCacheKey, BucketEntry> entry1,
-              Entry<BlockCacheKey, BucketEntry> entry2) {
-            return entry1.getValue().compareTo(entry2.getValue());
-          }
+    }
+    queue = MinMaxPriorityQueue.orderedBy(new Comparator<Map.Entry<BlockCacheKey, BucketEntry>>() {
+
+      public int compare(Entry<BlockCacheKey, BucketEntry> entry1,
+          Entry<BlockCacheKey, BucketEntry> entry2) {
+        return BucketEntry.COMPARATOR.compare(entry1.getValue(), entry2.getValue());
+      }
 
-        }).expectedSize(initialSize).create();
+    }).expectedSize(initialSize).create();
     cacheSize = 0;
     this.maxSize = maxSize;
   }
 
   /**
    * Attempt to add the specified entry to this queue.
-   * 
    * <p>
    * If the queue is smaller than the max size, or if the specified element is
    * ordered after the smallest element in the queue, the element will be added
@@ -83,7 +83,7 @@ public class CachedEntryQueue {
       cacheSize += entry.getValue().getLength();
     } else {
       BucketEntry head = queue.peek().getValue();
-      if (entry.getValue().compareTo(head) > 0) {
+      if (BucketEntry.COMPARATOR.compare(entry.getValue(), head) > 0) {
         cacheSize += entry.getValue().getLength();
         cacheSize -= head.getLength();
         if (cacheSize > maxSize) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4f151444/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdLock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdLock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdLock.java
index b9d0983..fedf951 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdLock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdLock.java
@@ -25,6 +25,8 @@ import java.util.concurrent.ConcurrentMap;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Allows multiple concurrent clients to lock on a numeric id with a minimal
  * memory overhead. The intended usage is as follows:
@@ -119,4 +121,18 @@ public class IdLock {
     assert map.size() == 0;
   }
 
+  @VisibleForTesting
+  public void waitForWaiters(long id, int numWaiters) throws InterruptedException {
+    for (Entry entry;;) {
+      entry = map.get(id);
+      if (entry != null) {
+        synchronized (entry) {
+          if (entry.numWaiters >= numWaiters) {
+            return;
+          }
+        }
+      }
+      Thread.sleep(100);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4f151444/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
index 5ef8cf0..b0a2ba2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
@@ -247,11 +247,11 @@ public class CacheTestUtils {
     assertTrue(toBeTested.getStats().getEvictedCount() > 0);
   }
 
-  private static class ByteArrayCacheable implements Cacheable {
+  public static class ByteArrayCacheable implements Cacheable {
 
-    static final CacheableDeserializer<Cacheable> blockDeserializer = 
+    static final CacheableDeserializer<Cacheable> blockDeserializer =
       new CacheableDeserializer<Cacheable>() {
-      
+
       @Override
       public Cacheable deserialize(ByteBuffer b) throws IOException {
         int len = b.getInt();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4f151444/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
index d29be01..99f5657 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.io.hfile.bucket;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -27,13 +28,14 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.hadoop.hbase.testclassification.IOTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
 import org.apache.hadoop.hbase.io.hfile.CacheTestUtils;
 import org.apache.hadoop.hbase.io.hfile.Cacheable;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.BucketSizeInfo;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.IndexStatistics;
+import org.apache.hadoop.hbase.testclassification.IOTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.IdLock;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -44,24 +46,23 @@ import org.junit.runners.Parameterized;
 /**
  * Basic test of BucketCache.Puts and gets.
  * <p>
- * Tests will ensure that blocks' data correctness under several threads
- * concurrency
+ * Tests will ensure that blocks' data correctness under several threads concurrency
  */
 @RunWith(Parameterized.class)
-@Category({IOTests.class, SmallTests.class})
+@Category({ IOTests.class, SmallTests.class })
 public class TestBucketCache {
 
   private static final Random RAND = new Random();
 
-  @Parameterized.Parameters(name="{index}: blockSize={0}, bucketSizes={1}")
+  @Parameterized.Parameters(name = "{index}: blockSize={0}, bucketSizes={1}")
   public static Iterable<Object[]> data() {
     return Arrays.asList(new Object[][] {
-      { 8192, null }, // TODO: why is 8k the default blocksize for these tests?
-      { 16 * 1024, new int[] {
-        2 * 1024 + 1024, 4 * 1024 + 1024, 8 * 1024 + 1024, 16 * 1024 + 1024,
-        28 * 1024 + 1024, 32 * 1024 + 1024, 64 * 1024 + 1024, 96 * 1024 + 1024,
-        128 * 1024 + 1024 } }
-    });
+        { 8192, null }, // TODO: why is 8k the default blocksize for these tests?
+        {
+            16 * 1024,
+            new int[] { 2 * 1024 + 1024, 4 * 1024 + 1024, 8 * 1024 + 1024, 16 * 1024 + 1024,
+                28 * 1024 + 1024, 32 * 1024 + 1024, 64 * 1024 + 1024, 96 * 1024 + 1024,
+                128 * 1024 + 1024 } } });
   }
 
   @Parameterized.Parameter(0)
@@ -76,7 +77,7 @@ public class TestBucketCache {
   final int BLOCK_SIZE = CACHE_SIZE / NUM_BLOCKS;
   final int NUM_THREADS = 1000;
   final int NUM_QUERIES = 10000;
-  
+
   final long capacitySize = 32 * 1024 * 1024;
   final int writeThreads = BucketCache.DEFAULT_WRITER_THREADS;
   final int writerQLen = BucketCache.DEFAULT_WRITER_QUEUE_ITEMS;
@@ -86,16 +87,16 @@ public class TestBucketCache {
   private class MockedBucketCache extends BucketCache {
 
     public MockedBucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
-      int writerThreads, int writerQLen, String persistencePath)
-        throws FileNotFoundException, IOException {
+        int writerThreads, int writerQLen, String persistencePath) throws FileNotFoundException,
+        IOException {
       super(ioEngineName, capacity, blockSize, bucketSizes, writerThreads, writerQLen,
-        persistencePath);
+          persistencePath);
       super.wait_when_cache = true;
     }
 
     @Override
-    public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf,
-        boolean inMemory, boolean cacheDataInL1) {
+    public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory,
+        boolean cacheDataInL1) {
       if (super.getBlock(cacheKey, true, false, true) != null) {
         throw new RuntimeException("Cached an already cached block");
       }
@@ -113,8 +114,9 @@ public class TestBucketCache {
 
   @Before
   public void setup() throws FileNotFoundException, IOException {
-    cache = new MockedBucketCache(ioEngineName, capacitySize, constructedBlockSize,
-      constructedBlockSizes, writeThreads, writerQLen, persistencePath);
+    cache =
+        new MockedBucketCache(ioEngineName, capacitySize, constructedBlockSize,
+            constructedBlockSizes, writeThreads, writerQLen, persistencePath);
   }
 
   @After
@@ -142,7 +144,7 @@ public class TestBucketCache {
     // Fill the allocated extents by choosing a random blocksize. Continues selecting blocks until
     // the cache is completely filled.
     List<Integer> tmp = new ArrayList<Integer>(BLOCKSIZES);
-    for (int i = 0; !full; i++) {
+    while (!full) {
       Integer blockSize = null;
       try {
         blockSize = randFrom(tmp);
@@ -156,9 +158,7 @@ public class TestBucketCache {
     for (Integer blockSize : BLOCKSIZES) {
       BucketSizeInfo bucketSizeInfo = mAllocator.roundUpToBucketSizeInfo(blockSize);
       IndexStatistics indexStatistics = bucketSizeInfo.statistics();
-      assertEquals(
-        "unexpected freeCount for " + bucketSizeInfo,
-        0, indexStatistics.freeCount());
+      assertEquals("unexpected freeCount for " + bucketSizeInfo, 0, indexStatistics.freeCount());
     }
 
     for (long offset : allocations) {
@@ -182,4 +182,41 @@ public class TestBucketCache {
     cache.stopWriterThreads();
     CacheTestUtils.testHeapSizeChanges(cache, BLOCK_SIZE);
   }
-}
\ No newline at end of file
+
+  // BucketCache.cacheBlock is async, it first adds block to ramCache and writeQueue, then writer
+  // threads will flush it to the bucket and put reference entry in backingMap.
+  private void cacheAndWaitUntilFlushedToBucket(BucketCache cache, BlockCacheKey cacheKey,
+      Cacheable block) throws InterruptedException {
+    cache.cacheBlock(cacheKey, block);
+    while (!cache.backingMap.containsKey(cacheKey)) {
+      Thread.sleep(100);
+    }
+  }
+
+  @Test
+  public void testMemoryLeak() throws Exception {
+    final BlockCacheKey cacheKey = new BlockCacheKey("dummy", 1L);
+    cacheAndWaitUntilFlushedToBucket(cache, cacheKey, new CacheTestUtils.ByteArrayCacheable(
+        new byte[10]));
+    long lockId = cache.backingMap.get(cacheKey).offset();
+    IdLock.Entry lockEntry = cache.offsetLock.getLockEntry(lockId);
+    Thread evictThread = new Thread("evict-block") {
+
+      @Override
+      public void run() {
+        cache.evictBlock(cacheKey);
+      }
+
+    };
+    evictThread.start();
+    cache.offsetLock.waitForWaiters(lockId, 1);
+    cache.blockEvicted(cacheKey, cache.backingMap.remove(cacheKey), true);
+    cacheAndWaitUntilFlushedToBucket(cache, cacheKey, new CacheTestUtils.ByteArrayCacheable(
+        new byte[10]));
+    cache.offsetLock.releaseLockEntry(lockEntry);
+    evictThread.join();
+    assertEquals(1L, cache.getBlockCount());
+    assertTrue(cache.getCurrentSize() > 0L);
+    assertTrue("We should have a block!", cache.iterator().hasNext());
+  }
+}


[4/9] hbase git commit: HBASE-13447 Bypass logic in TimeRange.compare.

Posted by mb...@apache.org.
HBASE-13447 Bypass logic in TimeRange.compare.


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/71536bdc
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/71536bdc
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/71536bdc

Branch: refs/heads/hbase-12439
Commit: 71536bdcc7d2d4b7a36a24ab9fa304cea4a8dda2
Parents: e9da064
Author: anoopsjohn <an...@gmail.com>
Authored: Tue Apr 14 11:37:48 2015 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Tue Apr 14 11:37:48 2015 +0530

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/io/TimeRange.java         | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/71536bdc/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
index 8c16389..8352e4e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
@@ -166,6 +166,7 @@ public class TimeRange {
    * 1 if timestamp is greater than timerange
    */
   public int compare(long timestamp) {
+    if (allTime) return 0;
     if (timestamp < minStamp) {
       return -1;
     } else if (timestamp >= maxStamp) {