You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by md...@apache.org on 2018/07/27 20:02:09 UTC

[01/24] hbase git commit: HBASE-20908 Infinite loop on regionserver if region replica are reduced [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-20749 c7a2bdd39 -> c8a7ced12 (forced update)


HBASE-20908 Infinite loop on regionserver if region replica are reduced

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-20749
Commit: eb906e20eeee76e3544ccd403f1d3a264c82a1e9
Parents: 067388b
Author: Ankit Singhal <an...@gmail.com>
Authored: Thu Jul 19 14:58:59 2018 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Fri Jul 20 13:05:11 2018 -0700

----------------------------------------------------------------------
 .../RegionReplicaReplicationEndpoint.java       | 34 ++++++++++---
 .../TestRegionReplicaReplicationEndpoint.java   | 53 ++++++++++++++++----
 2 files changed, 68 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eb906e20/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
index cb755fe..dc83eb7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
@@ -31,7 +31,6 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellScanner;
@@ -70,8 +69,10 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
 import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
 
@@ -276,7 +277,8 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
         EntryBuffers entryBuffers, ClusterConnection connection, ExecutorService pool,
         int numWriters, int operationTimeout) {
       super(controller, entryBuffers, numWriters);
-      this.sinkWriter = new RegionReplicaSinkWriter(this, connection, pool, operationTimeout);
+      this.sinkWriter =
+          new RegionReplicaSinkWriter(this, connection, pool, operationTimeout, tableDescriptors);
       this.tableDescriptors = tableDescriptors;
 
       // A cache for the table "memstore replication enabled" flag.
@@ -390,9 +392,10 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
     int operationTimeout;
     ExecutorService pool;
     Cache<TableName, Boolean> disabledAndDroppedTables;
+    TableDescriptors tableDescriptors;
 
     public RegionReplicaSinkWriter(RegionReplicaOutputSink sink, ClusterConnection connection,
-        ExecutorService pool, int operationTimeout) {
+        ExecutorService pool, int operationTimeout, TableDescriptors tableDescriptors) {
       this.sink = sink;
       this.connection = connection;
       this.operationTimeout = operationTimeout;
@@ -400,6 +403,7 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
         = RpcRetryingCallerFactory.instantiate(connection.getConfiguration());
       this.rpcControllerFactory = RpcControllerFactory.instantiate(connection.getConfiguration());
       this.pool = pool;
+      this.tableDescriptors = tableDescriptors;
 
       int nonExistentTableCacheExpiryMs = connection.getConfiguration()
         .getInt("hbase.region.replica.replication.cache.disabledAndDroppedTables.expiryMs", 5000);
@@ -506,13 +510,14 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
       }
 
       boolean tasksCancelled = false;
-      for (Future<ReplicateWALEntryResponse> task : tasks) {
+      for (int replicaId = 0; replicaId < tasks.size(); replicaId++) {
         try {
-          task.get();
+          tasks.get(replicaId).get();
         } catch (InterruptedException e) {
           throw new InterruptedIOException(e.getMessage());
         } catch (ExecutionException e) {
           Throwable cause = e.getCause();
+          boolean canBeSkipped = false;
           if (cause instanceof IOException) {
             // The table can be disabled or dropped at this time. For disabled tables, we have no
             // cheap mechanism to detect this case because meta does not contain this information.
@@ -520,21 +525,34 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
             // RPC. So instead we start the replay RPC with retries and check whether the table is
             // dropped or disabled which might cause SocketTimeoutException, or
             // RetriesExhaustedException or similar if we get IOE.
-            if (cause instanceof TableNotFoundException || connection.isTableDisabled(tableName)) {
+            if (cause instanceof TableNotFoundException
+                || connection.isTableDisabled(tableName)) {
+              disabledAndDroppedTables.put(tableName, Boolean.TRUE); // put to cache for later.
+              canBeSkipped = true;
+            } else if (tableDescriptors != null) {
+              TableDescriptor tableDescriptor = tableDescriptors.get(tableName);
+              if (tableDescriptor != null
+                  //(replicaId + 1) as no task is added for primary replica for replication
+                  && tableDescriptor.getRegionReplication() <= (replicaId + 1)) {
+                canBeSkipped = true;
+              }
+            }
+            if (canBeSkipped) {
               if (LOG.isTraceEnabled()) {
                 LOG.trace("Skipping " + entries.size() + " entries in table " + tableName
-                  + " because received exception for dropped or disabled table", cause);
+                    + " because received exception for dropped or disabled table",
+                  cause);
                 for (Entry entry : entries) {
                   LOG.trace("Skipping : " + entry);
                 }
               }
-              disabledAndDroppedTables.put(tableName, Boolean.TRUE); // put to cache for later.
               if (!tasksCancelled) {
                 sink.getSkippedEditsCounter().addAndGet(entries.size());
                 tasksCancelled = true; // so that we do not add to skipped counter again
               }
               continue;
             }
+
             // otherwise rethrow
             throw (IOException)cause;
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb906e20/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
index 61a1fbf..04db81a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
@@ -29,6 +29,11 @@ import java.util.List;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.CellBuilderFactory;
+import org.apache.hadoop.hbase.CellBuilderType;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -50,6 +55,8 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.testclassification.FlakeyTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -263,7 +270,7 @@ public class TestRegionReplicaReplicationEndpoint {
     for (int i = 1; i < regionReplication; i++) {
       final Region region = regions[i];
       // wait until all the data is replicated to all secondary regions
-      Waiter.waitFor(HTU.getConfiguration(), 90000, new Waiter.Predicate<Exception>() {
+      Waiter.waitFor(HTU.getConfiguration(), 90000, 1000, new Waiter.Predicate<Exception>() {
         @Override
         public boolean evaluate() throws Exception {
           LOG.info("verifying replication for region replica:" + region.getRegionInfo());
@@ -342,7 +349,6 @@ public class TestRegionReplicaReplicationEndpoint {
 
     Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration());
     Table table = connection.getTable(tableName);
-
     try {
       // load the data to the table
 
@@ -364,26 +370,35 @@ public class TestRegionReplicaReplicationEndpoint {
 
   @Test
   public void testRegionReplicaReplicationIgnoresDisabledTables() throws Exception {
-    testRegionReplicaReplicationIgnoresDisabledTables(false);
+    testRegionReplicaReplicationIgnores(false, false);
   }
 
   @Test
   public void testRegionReplicaReplicationIgnoresDroppedTables() throws Exception {
-    testRegionReplicaReplicationIgnoresDisabledTables(true);
+    testRegionReplicaReplicationIgnores(true, false);
   }
 
-  public void testRegionReplicaReplicationIgnoresDisabledTables(boolean dropTable)
+  @Test
+  public void testRegionReplicaReplicationIgnoresNonReplicatedTables() throws Exception {
+    testRegionReplicaReplicationIgnores(false, true);
+  }
+
+  public void testRegionReplicaReplicationIgnores(boolean dropTable, boolean disableReplication)
       throws Exception {
+
     // tests having edits from a disabled or dropped table is handled correctly by skipping those
     // entries and further edits after the edits from dropped/disabled table can be replicated
     // without problems.
-    final TableName tableName = TableName.valueOf(name.getMethodName() + dropTable);
+    final TableName tableName = TableName.valueOf(
+      name.getMethodName() + "_drop_" + dropTable + "_disabledReplication_" + disableReplication);
     HTableDescriptor htd = HTU.createTableDescriptor(tableName);
     int regionReplication = 3;
     htd.setRegionReplication(regionReplication);
     HTU.deleteTableIfAny(tableName);
+
     HTU.getAdmin().createTable(htd);
-    TableName toBeDisabledTable = TableName.valueOf(dropTable ? "droppedTable" : "disabledTable");
+    TableName toBeDisabledTable = TableName.valueOf(
+      dropTable ? "droppedTable" : (disableReplication ? "disableReplication" : "disabledTable"));
     HTU.deleteTableIfAny(toBeDisabledTable);
     htd = HTU.createTableDescriptor(toBeDisabledTable.toString());
     htd.setRegionReplication(regionReplication);
@@ -405,28 +420,44 @@ public class TestRegionReplicaReplicationEndpoint {
     RegionReplicaReplicationEndpoint.RegionReplicaOutputSink sink =
         mock(RegionReplicaReplicationEndpoint.RegionReplicaOutputSink.class);
     when(sink.getSkippedEditsCounter()).thenReturn(skippedEdits);
+    FSTableDescriptors fstd = new FSTableDescriptors(HTU.getConfiguration(),
+        FileSystem.get(HTU.getConfiguration()), HTU.getDefaultRootDirPath());
     RegionReplicaReplicationEndpoint.RegionReplicaSinkWriter sinkWriter =
         new RegionReplicaReplicationEndpoint.RegionReplicaSinkWriter(sink,
-          (ClusterConnection) connection,
-          Executors.newSingleThreadExecutor(), Integer.MAX_VALUE);
+            (ClusterConnection) connection, Executors.newSingleThreadExecutor(), Integer.MAX_VALUE,
+            fstd);
     RegionLocator rl = connection.getRegionLocator(toBeDisabledTable);
     HRegionLocation hrl = rl.getRegionLocation(HConstants.EMPTY_BYTE_ARRAY);
     byte[] encodedRegionName = hrl.getRegionInfo().getEncodedNameAsBytes();
 
+    Cell cell = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(Bytes.toBytes("A"))
+        .setFamily(HTU.fam1).setValue(Bytes.toBytes("VAL")).setType(Type.Put).build();
     Entry entry = new Entry(
       new WALKeyImpl(encodedRegionName, toBeDisabledTable, 1),
-      new WALEdit());
+        new WALEdit()
+            .add(cell));
 
     HTU.getAdmin().disableTable(toBeDisabledTable); // disable the table
     if (dropTable) {
       HTU.getAdmin().deleteTable(toBeDisabledTable);
+    } else if (disableReplication) {
+      htd.setRegionReplication(regionReplication - 2);
+      HTU.getAdmin().modifyTable(toBeDisabledTable, htd);
+      HTU.getAdmin().enableTable(toBeDisabledTable);
     }
-
     sinkWriter.append(toBeDisabledTable, encodedRegionName,
       HConstants.EMPTY_BYTE_ARRAY, Lists.newArrayList(entry, entry));
 
     assertEquals(2, skippedEdits.get());
 
+    if (disableReplication) {
+      // enable replication again so that we can verify replication
+      HTU.getAdmin().disableTable(toBeDisabledTable); // disable the table
+      htd.setRegionReplication(regionReplication);
+      HTU.getAdmin().modifyTable(toBeDisabledTable, htd);
+      HTU.getAdmin().enableTable(toBeDisabledTable);
+    }
+
     try {
       // load some data to the to-be-dropped table
 


[23/24] hbase git commit: HBASE-20966 RestoreTool#getTableInfoPath should look for completed snapshot only

Posted by md...@apache.org.
HBASE-20966 RestoreTool#getTableInfoPath should look for completed snapshot only


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

Branch: refs/heads/HBASE-20749
Commit: cf481d3b514450c024a2b0febfaef6dd19a83281
Parents: 7178a98
Author: tedyu <yu...@gmail.com>
Authored: Fri Jul 27 11:12:26 2018 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Fri Jul 27 11:12:26 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/backup/util/RestoreTool.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cf481d3b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
index 5c43722..13b183d 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
@@ -236,7 +236,8 @@ public class RestoreTool {
     Path tableInfoPath = null;
 
     // can't build the path directly as the timestamp values are different
-    FileStatus[] snapshots = fs.listStatus(tableSnapShotPath);
+    FileStatus[] snapshots = fs.listStatus(tableSnapShotPath,
+        new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
     for (FileStatus snapshot : snapshots) {
       tableInfoPath = snapshot.getPath();
       // SnapshotManifest.DATA_MANIFEST_NAME = "data.manifest";


[18/24] hbase git commit: HBASE-20949 Add logs for debugging

Posted by md...@apache.org.
HBASE-20949 Add logs for debugging


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

Branch: refs/heads/HBASE-20749
Commit: 8b8de1f8a77b5b9f6d4b8cfb7eeb3d545a69d0f2
Parents: a392c01
Author: zhangduo <zh...@apache.org>
Authored: Thu Jul 26 22:42:44 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Jul 26 22:43:14 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/procedure2/LockAndQueue.java  | 13 ++++++++++---
 .../master/procedure/MasterProcedureScheduler.java    | 14 +++++++-------
 2 files changed, 17 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8b8de1f8/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
index ae8daa2..f86c7c8 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.hbase.procedure2;
 import java.util.function.Predicate;
 import java.util.stream.Stream;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Locking for mutual exclusion between procedures. Used only by procedure framework internally.
@@ -48,6 +50,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public class LockAndQueue implements LockStatus {
+  private static final Logger LOG = LoggerFactory.getLogger(LockAndQueue.class);
   private final ProcedureDeque queue = new ProcedureDeque();
   private Procedure<?> exclusiveLockOwnerProcedure = null;
   private int sharedLock = 0;
@@ -111,11 +114,13 @@ public class LockAndQueue implements LockStatus {
    */
   public boolean trySharedLock(Procedure<?> proc) {
     if (hasExclusiveLock() && !hasLockAccess(proc)) {
+      LOG.debug("{} acquire shared lock {} failed", proc, this, new Exception());
       return false;
     }
     // If no one holds the xlock, then we are free to hold the sharedLock
     // If the parent proc or we have already held the xlock, then we return true here as
     // xlock is more powerful then shared lock.
+    LOG.debug("{} acquire shared lock {} succeeded", proc, this, new Exception());
     sharedLock++;
     return true;
   }
@@ -123,7 +128,8 @@ public class LockAndQueue implements LockStatus {
   /**
    * @return whether we should wake the procedures waiting on the lock here.
    */
-  public boolean releaseSharedLock() {
+  public boolean releaseSharedLock(Procedure<?> proc) {
+    LOG.debug("{} release shared lock {}", proc, this, new Exception());
     // hasExclusiveLock could be true, it usually means we acquire shared lock while we or our
     // parent have held the xlock. And since there is still an exclusive lock, we do not need to
     // wake any procedures.
@@ -186,7 +192,8 @@ public class LockAndQueue implements LockStatus {
 
   @Override
   public String toString() {
-    return "exclusiveLockOwner=" + (hasExclusiveLock() ? getExclusiveLockProcIdOwner() : "NONE") +
-      ", sharedLockCount=" + getSharedLockCount() + ", waitingProcCount=" + queue.size();
+    return String.format("%08x", hashCode()) + ": exclusiveLockOwner=" +
+      (hasExclusiveLock() ? getExclusiveLockProcIdOwner() : "NONE") + ", sharedLockCount=" +
+      getSharedLockCount() + ", waitingProcCount=" + queue.size();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8b8de1f8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index 2a29ee1..4180a96 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -526,7 +526,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         return true;
       }
       if (!tableLock.tryExclusiveLock(procedure)) {
-        namespaceLock.releaseSharedLock();
+        namespaceLock.releaseSharedLock(procedure);
         waitProcedure(tableLock, procedure);
         logLockedResource(LockedResourceType.TABLE, table.getNameAsString());
         return true;
@@ -552,7 +552,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       if (tableLock.releaseExclusiveLock(procedure)) {
         waitingCount += wakeWaitingProcedures(tableLock);
       }
-      if (namespaceLock.releaseSharedLock()) {
+      if (namespaceLock.releaseSharedLock(procedure)) {
         waitingCount += wakeWaitingProcedures(namespaceLock);
       }
       addToRunQueue(tableRunQueue, getTableQueue(table));
@@ -584,7 +584,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       }
 
       if (!tableLock.trySharedLock(procedure)) {
-        namespaceLock.releaseSharedLock();
+        namespaceLock.releaseSharedLock(procedure);
         waitProcedure(tableLock, procedure);
         return null;
       }
@@ -606,11 +606,11 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       final LockAndQueue namespaceLock = locking.getNamespaceLock(table.getNamespaceAsString());
       final LockAndQueue tableLock = locking.getTableLock(table);
       int waitingCount = 0;
-      if (tableLock.releaseSharedLock()) {
+      if (tableLock.releaseSharedLock(procedure)) {
         addToRunQueue(tableRunQueue, getTableQueue(table));
         waitingCount += wakeWaitingProcedures(tableLock);
       }
-      if (namespaceLock.releaseSharedLock()) {
+      if (namespaceLock.releaseSharedLock(procedure)) {
         waitingCount += wakeWaitingProcedures(namespaceLock);
       }
       wakePollIfNeeded(waitingCount);
@@ -784,7 +784,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
 
       final LockAndQueue namespaceLock = locking.getNamespaceLock(namespace);
       if (!namespaceLock.tryExclusiveLock(procedure)) {
-        systemNamespaceTableLock.releaseSharedLock();
+        systemNamespaceTableLock.releaseSharedLock(procedure);
         waitProcedure(namespaceLock, procedure);
         logLockedResource(LockedResourceType.NAMESPACE, namespace);
         return true;
@@ -811,7 +811,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       if (namespaceLock.releaseExclusiveLock(procedure)) {
         waitingCount += wakeWaitingProcedures(namespaceLock);
       }
-      if (systemNamespaceTableLock.releaseSharedLock()) {
+      if (systemNamespaceTableLock.releaseSharedLock(procedure)) {
         addToRunQueue(tableRunQueue, getTableQueue(TableName.NAMESPACE_TABLE_NAME));
         waitingCount += wakeWaitingProcedures(systemNamespaceTableLock);
       }


[14/24] hbase git commit: HBASE-20867 RS may get killed while master restarts

Posted by md...@apache.org.
HBASE-20867 RS may get killed while master restarts


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

Branch: refs/heads/HBASE-20749
Commit: 44f6ef1c90548304402103d6210c9edc39ebc246
Parents: f3f17fa
Author: Allan Yang <al...@apache.org>
Authored: Wed Jul 25 18:16:03 2018 +0800
Committer: Allan Yang <al...@apache.org>
Committed: Wed Jul 25 18:16:28 2018 +0800

----------------------------------------------------------------------
 .../hbase/exceptions/ClientExceptionsUtil.java  |   2 +-
 .../exceptions/ConnectionClosedException.java   |  36 +++++
 .../org/apache/hadoop/hbase/ipc/IPCUtil.java    |   5 +
 .../hadoop/hbase/ipc/NettyRpcDuplexHandler.java |   3 +-
 .../hbase/security/CryptoAESWrapHandler.java    |   4 +-
 .../NettyHBaseSaslRpcClientHandler.java         |   3 +-
 .../hadoop/hbase/security/SaslWrapHandler.java  |  12 +-
 .../master/assignment/AssignmentManager.java    |   2 +-
 .../assignment/RegionTransitionProcedure.java   |   2 +-
 .../master/procedure/RSProcedureDispatcher.java |  11 +-
 .../master/TestMasterAbortAndRSGotKilled.java   | 138 +++++++++++++++++++
 11 files changed, 199 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/44f6ef1c/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
index 126571b..5402a0e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
@@ -149,7 +149,7 @@ public final class ClientExceptionsUtil {
       || e instanceof ClosedChannelException || e instanceof SyncFailedException
       || e instanceof EOFException || e instanceof TimeoutException
       || e instanceof CallTimeoutException || e instanceof ConnectionClosingException
-      || e instanceof FailedServerException);
+      || e instanceof FailedServerException || e instanceof ConnectionClosedException);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/44f6ef1c/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosedException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosedException.java
new file mode 100644
index 0000000..c595c14
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosedException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.exceptions;
+
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Thrown when the connection is closed
+ */
+
+@InterfaceAudience.Public
+public class ConnectionClosedException extends HBaseIOException {
+
+  private static final long serialVersionUID = -8938225073412971497L;
+
+  public ConnectionClosedException(String string) {
+    super(string);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/44f6ef1c/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
index 974994e..5e38732 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import org.apache.hadoop.hbase.exceptions.ConnectionClosedException;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream;
 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
@@ -176,6 +177,10 @@ class IPCUtil {
     } else if (exception instanceof DoNotRetryIOException) {
       return (IOException) new DoNotRetryIOException(
           "Call to " + addr + " failed on local exception: " + exception).initCause(exception);
+    } else if (exception instanceof ConnectionClosedException) {
+      return (ConnectionClosedException) exception;
+    } else if (exception instanceof StoppedRpcClientException) {
+      return (StoppedRpcClientException) exception;
     } else {
       return (IOException) new IOException(
           "Call to " + addr + " failed on local exception: " + exception).initCause(exception);

http://git-wip-us.apache.org/repos/asf/hbase/blob/44f6ef1c/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
index f6d338b..649375a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import org.apache.hadoop.hbase.exceptions.ConnectionClosedException;
 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
 import org.apache.hbase.thirdparty.com.google.protobuf.Message.Builder;
 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
@@ -207,7 +208,7 @@ class NettyRpcDuplexHandler extends ChannelDuplexHandler {
   @Override
   public void channelInactive(ChannelHandlerContext ctx) throws Exception {
     if (!id2Call.isEmpty()) {
-      cleanupCalls(ctx, new IOException("Connection closed"));
+      cleanupCalls(ctx, new ConnectionClosedException("Connection closed"));
     }
     conn.shutdown();
     ctx.fireChannelInactive();

http://git-wip-us.apache.org/repos/asf/hbase/blob/44f6ef1c/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java
index ad1aa69..ceb3f35 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hbase.security;
 
+import org.apache.hadoop.hbase.exceptions.ConnectionClosedException;
 import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;
 import org.apache.hbase.thirdparty.io.netty.buffer.Unpooled;
 import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
@@ -29,7 +30,6 @@ import org.apache.hbase.thirdparty.io.netty.util.concurrent.PromiseCombiner;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES;
 
-import java.io.IOException;
 
 /**
  * wrap messages with Crypto AES.
@@ -91,7 +91,7 @@ public class CryptoAESWrapHandler extends ChannelOutboundHandlerAdapter {
   @Override
   public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception {
     if (!queue.isEmpty()) {
-      queue.releaseAndFailAll(new IOException("Connection closed"));
+      queue.releaseAndFailAll(new ConnectionClosedException("Connection closed"));
     }
     ctx.close(promise);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/44f6ef1c/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java
index 8da3fde..eb4f205 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.security;
 
+import org.apache.hadoop.hbase.exceptions.ConnectionClosedException;
 import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;
 import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
 import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler;
@@ -150,7 +151,7 @@ public class NettyHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler<
   @Override
   public void channelInactive(ChannelHandlerContext ctx) throws Exception {
     saslRpcClient.dispose();
-    saslPromise.tryFailure(new IOException("Connection closed"));
+    saslPromise.tryFailure(new ConnectionClosedException("Connection closed"));
     ctx.fireChannelInactive();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/44f6ef1c/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java
index 949d8bb..62c127e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java
@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.hbase.security;
 
+import javax.security.sasl.SaslClient;
+
+import org.apache.hadoop.hbase.exceptions.ConnectionClosedException;
+import org.apache.yetus.audience.InterfaceAudience;
+
 import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;
 import org.apache.hbase.thirdparty.io.netty.buffer.Unpooled;
 import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
@@ -26,11 +31,6 @@ import org.apache.hbase.thirdparty.io.netty.channel.CoalescingBufferQueue;
 import org.apache.hbase.thirdparty.io.netty.util.ReferenceCountUtil;
 import org.apache.hbase.thirdparty.io.netty.util.concurrent.PromiseCombiner;
 
-import java.io.IOException;
-
-import javax.security.sasl.SaslClient;
-
-import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * wrap sasl messages.
@@ -92,7 +92,7 @@ public class SaslWrapHandler extends ChannelOutboundHandlerAdapter {
   @Override
   public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception {
     if (!queue.isEmpty()) {
-      queue.releaseAndFailAll(new IOException("Connection closed"));
+      queue.releaseAndFailAll(new ConnectionClosedException("Connection closed"));
     }
     ctx.close(promise);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/44f6ef1c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index f5cd0a3..60a2349 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -1016,7 +1016,7 @@ public class AssignmentManager implements ServerListener {
           if (regionNode.isInState(State.OPENING, State.OPEN)) {
             if (!regionNode.getRegionLocation().equals(serverName)) {
               throw new UnexpectedStateException(regionNode.toString() +
-                "reported OPEN on server=" + serverName +
+                " reported OPEN on server=" + serverName +
                 " but state has otherwise.");
             } else if (regionNode.isInState(State.OPENING)) {
               try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/44f6ef1c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index 4054778..0db8676 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -217,7 +217,7 @@ public abstract class RegionTransitionProcedure
       final ServerName serverName, final IOException exception) {
     final RegionStateNode regionNode = getRegionState(env);
     LOG.warn("Remote call failed {}; {}; {}; exception={}", serverName,
-        this, regionNode.toShortString(), exception.getClass().getSimpleName());
+        this, regionNode.toShortString(), exception.getClass().getSimpleName(), exception);
     if (remoteCallFailed(env, regionNode, exception)) {
       // NOTE: This call to wakeEvent puts this Procedure back on the scheduler.
       // Thereafter, another Worker can be in here so DO NOT MESS WITH STATE beyond

http://git-wip-us.apache.org/repos/asf/hbase/blob/44f6ef1c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index 3959af7..638f9d3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
-import java.net.SocketTimeoutException;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.Callable;
@@ -26,6 +25,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.ServerListener;
@@ -192,11 +192,11 @@ public class RSProcedureDispatcher
         return false;
       }
 
-      // In case socket is timed out and the region server is still online,
+      // In case it is a connection exception and the region server is still online,
       // the openRegion RPC could have been accepted by the server and
-      // just the response didn't go through.  So we will retry to
+      // just the response didn't go through. So we will retry to
       // open the region on the same server.
-      final boolean retry = !hold && (e instanceof SocketTimeoutException
+      final boolean retry = !hold && (ClientExceptionsUtil.isConnectionException(e)
           && master.getServerManager().isServerOnline(serverName));
       if (retry) {
         // we want to retry as many times as needed as long as the RS is not dead.
@@ -204,10 +204,9 @@ public class RSProcedureDispatcher
           LOG.debug(String.format("Retrying to same RegionServer %s because: %s",
               serverName, e.getMessage()), e);
         }
-        submitTask(this);
+        submitTask(this, 100, TimeUnit.MILLISECONDS);
         return true;
       }
-
       // trying to send the request elsewhere instead
       LOG.warn(String.format("Failed dispatch to server=%s try=%d",
                   serverName, numberOfAttemptsSoFar), e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/44f6ef1c/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterAbortAndRSGotKilled.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterAbortAndRSGotKilled.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterAbortAndRSGotKilled.java
new file mode 100644
index 0000000..41a8001
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterAbortAndRSGotKilled.java
@@ -0,0 +1,138 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.master.assignment.MoveRegionProcedure;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@Category({ MasterTests.class, MediumTests.class })
+public class TestMasterAbortAndRSGotKilled {
+  private static Logger LOG = LoggerFactory
+      .getLogger(TestMasterAbortAndRSGotKilled.class.getName());
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterAbortAndRSGotKilled.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static TableName TABLE_NAME = TableName.valueOf("test");
+
+  private static CountDownLatch countDownLatch = new CountDownLatch(1);
+
+
+
+  private static byte[] CF = Bytes.toBytes("cf");
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.getConfiguration().setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
+        DelayCloseCP.class.getName());
+    UTIL.startMiniCluster(3);
+    UTIL.getAdmin().balancerSwitch(false, true);
+    UTIL.createTable(TABLE_NAME, CF);
+    UTIL.waitTableAvailable(TABLE_NAME);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws Exception {
+    JVMClusterUtil.RegionServerThread rsThread = null;
+    for (JVMClusterUtil.RegionServerThread t : UTIL.getMiniHBaseCluster()
+        .getRegionServerThreads()) {
+      if (!t.getRegionServer().getRegions(TABLE_NAME).isEmpty()) {
+        rsThread = t;
+        break;
+      }
+    }
+    //find the rs and hri of the table
+    HRegionServer rs = rsThread.getRegionServer();
+    RegionInfo hri = rs.getRegions(TABLE_NAME).get(0).getRegionInfo();
+    MoveRegionProcedure moveRegionProcedure = new MoveRegionProcedure(
+      UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor()
+        .getEnvironment(),
+        new RegionPlan(hri, rs.getServerName(), rs.getServerName()), true);
+    long procID = UTIL.getMiniHBaseCluster().getMaster()
+      .getMasterProcedureExecutor().submitProcedure(moveRegionProcedure);
+    countDownLatch.await();
+    UTIL.getMiniHBaseCluster().stopMaster(0);
+    UTIL.getMiniHBaseCluster().startMaster();
+    //wait until master initialized
+    UTIL.waitFor(30000,
+      () -> UTIL.getMiniHBaseCluster().getMaster() != null && UTIL
+        .getMiniHBaseCluster().getMaster().isInitialized());
+    Assert.assertTrue("Should be 3 RS after master restart",
+        UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size() == 3);
+
+  }
+
+  public static class DelayCloseCP implements RegionCoprocessor,
+      RegionObserver {
+    @Override
+    public void preClose(ObserverContext<RegionCoprocessorEnvironment> c,
+        boolean abortRequested) throws IOException {
+      try {
+        if (!c.getEnvironment().getRegion().getRegionInfo().getTable().isSystemTable()) {
+          LOG.error("begin to sleep");
+          countDownLatch.countDown();
+          //Sleep here so we can stuck the RPC call
+          Thread.sleep(10000);
+          LOG.error("finish sleep");
+        }
+      } catch (Throwable t) {
+
+      }
+    }
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+      return Optional.of(this);
+    }
+  }
+
+}


[13/24] hbase git commit: HBASE-20846 Restore procedure locks when master restarts

Posted by md...@apache.org.
HBASE-20846 Restore procedure locks when master restarts


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

Branch: refs/heads/HBASE-20749
Commit: f3f17fa111f37233ddc42ddb9c38594e35d8d501
Parents: e44f506
Author: zhangduo <zh...@apache.org>
Authored: Sun Jul 22 15:10:06 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Jul 25 14:37:26 2018 +0800

----------------------------------------------------------------------
 .../procedure2/AbstractProcedureScheduler.java  |   2 +-
 .../hbase/procedure2/DelayedProcedure.java      |   5 +-
 .../hadoop/hbase/procedure2/Procedure.java      | 384 +++++++++-------
 .../hbase/procedure2/ProcedureExecutor.java     | 439 ++++++++++---------
 .../hadoop/hbase/procedure2/ProcedureUtil.java  |   7 +
 .../hbase/procedure2/RootProcedureState.java    |  44 +-
 .../hbase/procedure2/TimeoutExecutorThread.java |  28 +-
 .../procedure2/TestProcedureReplayOrder.java    |   8 +-
 .../procedure2/TestProcedureSuspended.java      |   6 -
 .../src/main/protobuf/Procedure.proto           |   3 +
 .../hbase/master/ClusterSchemaServiceImpl.java  |   4 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   7 +-
 .../master/assignment/GCRegionProcedure.java    |   5 -
 .../assignment/MergeTableRegionsProcedure.java  |  12 +-
 .../assignment/RegionTransitionProcedure.java   |  33 +-
 .../hbase/master/locking/LockProcedure.java     |   9 -
 .../AbstractStateMachineNamespaceProcedure.java |   6 +-
 .../AbstractStateMachineRegionProcedure.java    |   9 -
 .../AbstractStateMachineTableProcedure.java     |   8 +-
 .../procedure/CreateNamespaceProcedure.java     |  35 +-
 .../master/procedure/CreateTableProcedure.java  |  12 +-
 .../master/procedure/InitMetaProcedure.java     |   7 +-
 .../procedure/MasterProcedureScheduler.java     |  37 +-
 .../master/procedure/MasterProcedureUtil.java   |   2 +-
 .../hbase/master/procedure/PeerQueue.java       |  14 -
 .../master/procedure/ProcedureSyncWait.java     |   4 +-
 .../hadoop/hbase/master/procedure/Queue.java    |  13 +-
 .../replication/AbstractPeerProcedure.java      |  14 +-
 .../hbase-webapps/master/procedures.jsp         |   2 +-
 .../hbase/client/TestGetProcedureResult.java    |   2 +-
 .../assignment/TestAssignmentManager.java       |   3 +-
 .../procedure/TestMasterProcedureEvents.java    |   2 +-
 .../master/procedure/TestProcedureAdmin.java    |   2 +-
 .../hbase/procedure/TestFailedProcCleanup.java  |   5 +-
 .../security/access/TestAccessController.java   |   2 +-
 35 files changed, 624 insertions(+), 551 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java
index c036163..5645f89 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java
@@ -163,8 +163,8 @@ public abstract class AbstractProcedureScheduler implements ProcedureScheduler {
           return null;
         }
       }
-
       final Procedure pollResult = dequeue();
+
       pollCalls++;
       nullPollCalls += (pollResult == null) ? 1 : 0;
       return pollResult;

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java
index a9f3e7d..3fc9750 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java
@@ -24,8 +24,9 @@ import org.apache.yetus.audience.InterfaceAudience;
  * Vessel that carries a Procedure and a timeout.
  */
 @InterfaceAudience.Private
-class DelayedProcedure extends DelayedUtil.DelayedContainerWithTimestamp<Procedure<?>> {
-  public DelayedProcedure(Procedure<?> procedure) {
+class DelayedProcedure<TEnvironment>
+    extends DelayedUtil.DelayedContainerWithTimestamp<Procedure<TEnvironment>> {
+  public DelayedProcedure(Procedure<TEnvironment> procedure) {
     super(procedure, procedure.getTimeoutTimestamp());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
index 545bedf..58757bb 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -22,76 +22,94 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.metrics.Counter;
 import org.apache.hadoop.hbase.metrics.Histogram;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
 import org.apache.hadoop.hbase.procedure2.util.StringUtils;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.NonceKey;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
+
 /**
- * Base Procedure class responsible for Procedure Metadata;
- * e.g. state, submittedTime, lastUpdate, stack-indexes, etc.
- *
- * <p>Procedures are run by a {@link ProcedureExecutor} instance. They are submitted and then
- * the ProcedureExecutor keeps calling {@link #execute(Object)} until the Procedure is done.
- * Execute may be called multiple times in the case of failure or a restart, so code must be
- * idempotent. The return from an execute call is either: null to indicate we are done;
- * ourself if there is more to do; or, a set of sub-procedures that need to
- * be run to completion before the framework resumes our execution.
- *
- * <p>The ProcedureExecutor keeps its
- * notion of Procedure State in the Procedure itself; e.g. it stamps the Procedure as INITIALIZING,
- * RUNNABLE, SUCCESS, etc. Here are some of the States defined in the ProcedureState enum from
- * protos:
- *<ul>
- * <li>{@link #isFailed()} A procedure has executed at least once and has failed. The procedure
- * may or may not have rolled back yet. Any procedure in FAILED state will be eventually moved
- * to ROLLEDBACK state.</li>
- *
+ * Base Procedure class responsible for Procedure Metadata; e.g. state, submittedTime, lastUpdate,
+ * stack-indexes, etc.
+ * <p/>
+ * Procedures are run by a {@link ProcedureExecutor} instance. They are submitted and then the
+ * ProcedureExecutor keeps calling {@link #execute(Object)} until the Procedure is done. Execute may
+ * be called multiple times in the case of failure or a restart, so code must be idempotent. The
+ * return from an execute call is either: null to indicate we are done; ourself if there is more to
+ * do; or, a set of sub-procedures that need to be run to completion before the framework resumes
+ * our execution.
+ * <p/>
+ * The ProcedureExecutor keeps its notion of Procedure State in the Procedure itself; e.g. it stamps
+ * the Procedure as INITIALIZING, RUNNABLE, SUCCESS, etc. Here are some of the States defined in the
+ * ProcedureState enum from protos:
+ * <ul>
+ * <li>{@link #isFailed()} A procedure has executed at least once and has failed. The procedure may
+ * or may not have rolled back yet. Any procedure in FAILED state will be eventually moved to
+ * ROLLEDBACK state.</li>
  * <li>{@link #isSuccess()} A procedure is completed successfully without exception.</li>
- *
  * <li>{@link #isFinished()} As a procedure in FAILED state will be tried forever for rollback, only
  * condition when scheduler/ executor will drop procedure from further processing is when procedure
  * state is ROLLEDBACK or isSuccess() returns true. This is a terminal state of the procedure.</li>
- *
  * <li>{@link #isWaiting()} - Procedure is in one of the two waiting states
  * ({@link ProcedureState#WAITING}, {@link ProcedureState#WAITING_TIMEOUT}).</li>
- *</ul>
- * NOTE: These states are of the ProcedureExecutor. Procedure implementations in turn can keep
- * their own state. This can lead to confusion. Try to keep the two distinct.
- *
- * <p>rollback() is called when the procedure or one of the sub-procedures
- * has failed. The rollback step is supposed to cleanup the resources created
- * during the execute() step. In case of failure and restart, rollback() may be
- * called multiple times, so again the code must be idempotent.
- *
- * <p>Procedure can be made respect a locking regime. It has acquire/release methods as
- * well as an {@link #hasLock(Object)}. The lock implementation is up to the implementor.
- * If an entity needs to be locked for the life of a procedure -- not just the calls to
- * execute -- then implementations should say so with the {@link #holdLock(Object)}
- * method.
- *
- * <p>Procedures can be suspended or put in wait state with a callback that gets executed on
+ * </ul>
+ * NOTE: These states are of the ProcedureExecutor. Procedure implementations in turn can keep their
+ * own state. This can lead to confusion. Try to keep the two distinct.
+ * <p/>
+ * rollback() is called when the procedure or one of the sub-procedures has failed. The rollback
+ * step is supposed to cleanup the resources created during the execute() step. In case of failure
+ * and restart, rollback() may be called multiple times, so again the code must be idempotent.
+ * <p/>
+ * Procedure can be made respect a locking regime. It has acquire/release methods as well as an
+ * {@link #hasLock()}. The lock implementation is up to the implementor. If an entity needs to be
+ * locked for the life of a procedure -- not just the calls to execute -- then implementations
+ * should say so with the {@link #holdLock(Object)} method.
+ * <p/>
+ * And since we need to restore the lock when restarting to keep the logic correct(HBASE-20846), the
+ * implementation is a bit tricky so we add some comments hrre about it.
+ * <ul>
+ * <li>Make {@link #hasLock()} method final, and add a {@link #locked} field in Procedure to record
+ * whether we have the lock. We will set it to {@code true} in
+ * {@link #doAcquireLock(Object, ProcedureStore)} and to {@code false} in
+ * {@link #doReleaseLock(Object, ProcedureStore)}. The sub classes do not need to manage it any
+ * more.</li>
+ * <li>Also added a locked field in the proto message. When storing, the field will be set according
+ * to the return value of {@link #hasLock()}. And when loading, there is a new field in Procedure
+ * called {@link #lockedWhenLoading}. We will set it to {@code true} if the locked field in proto
+ * message is {@code true}.</li>
+ * <li>The reason why we can not set the {@link #locked} field directly to {@code true} by calling
+ * {@link #doAcquireLock(Object, ProcedureStore)} is that, during initialization, most procedures
+ * need to wait until master is initialized. So the solution here is that, we introduced a new
+ * method called {@link #waitInitialized(Object)} in Procedure, and move the wait master initialized
+ * related code from {@link #acquireLock(Object)} to this method. And we added a restoreLock method
+ * to Procedure, if {@link #lockedWhenLoading} is {@code true}, we will call the
+ * {@link #acquireLock(Object)} to get the lock, but do not set {@link #locked} to true. And later
+ * when we call {@link #doAcquireLock(Object, ProcedureStore)} and pass the
+ * {@link #waitInitialized(Object)} check, we will test {@link #lockedWhenLoading}, if it is
+ * {@code true}, when we just set the {@link #locked} field to true and return, without actually
+ * calling the {@link #acquireLock(Object)} method since we have already called it once.</li>
+ * </ul>
+ * <p/>
+ * Procedures can be suspended or put in wait state with a callback that gets executed on
  * Procedure-specified timeout. See {@link #setTimeout(int)}}, and
- * {@link #setTimeoutFailure(Object)}. See TestProcedureEvents and the
- * TestTimeoutEventProcedure class for an example usage.</p>
- *
- * <p>There are hooks for collecting metrics on submit of the procedure and on finish.
- * See {@link #updateMetricsOnSubmit(Object)} and
- * {@link #updateMetricsOnFinish(Object, long, boolean)}.
+ * {@link #setTimeoutFailure(Object)}. See TestProcedureEvents and the TestTimeoutEventProcedure
+ * class for an example usage.
+ * </p>
+ * <p/>
+ * There are hooks for collecting metrics on submit of the procedure and on finish. See
+ * {@link #updateMetricsOnSubmit(Object)} and {@link #updateMetricsOnFinish(Object, long, boolean)}.
  */
 @InterfaceAudience.Private
-@InterfaceStability.Evolving
 public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TEnvironment>> {
   private static final Logger LOG = LoggerFactory.getLogger(Procedure.class);
   public static final long NO_PROC_ID = -1;
@@ -122,6 +140,10 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
 
   private volatile byte[] result = null;
 
+  private volatile boolean locked = false;
+
+  private boolean lockedWhenLoading = false;
+
   /**
    * The main code of the procedure. It must be idempotent since execute()
    * may be called multiple times in case of machine failure in the middle
@@ -170,7 +192,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * be able to resume on failure.
    * @param serializer stores the serializable state
    */
-  protected abstract void serializeStateData(final ProcedureStateSerializer serializer)
+  protected abstract void serializeStateData(ProcedureStateSerializer serializer)
     throws IOException;
 
   /**
@@ -178,52 +200,65 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * state.
    * @param serializer contains the serialized state
    */
-  protected abstract void deserializeStateData(final ProcedureStateSerializer serializer)
+  protected abstract void deserializeStateData(ProcedureStateSerializer serializer)
     throws IOException;
 
   /**
-   * The user should override this method if they need a lock on an Entity.
-   * A lock can be anything, and it is up to the implementor. The Procedure
-   * Framework will call this method just before it invokes {@link #execute(Object)}.
-   * It calls {@link #releaseLock(Object)} after the call to execute.
-   *
-   * <p>If you need to hold the lock for the life of the Procedure -- i.e. you do not
-   * want any other Procedure interfering while this Procedure is running, see
-   * {@link #holdLock(Object)}.
-   *
-   * <p>Example: in our Master we can execute request in parallel for different tables.
-   * We can create t1 and create t2 and these creates can be executed at the same time.
-   * Anything else on t1/t2 is queued waiting that specific table create to happen.
-   *
-   * <p>There are 3 LockState:
-   * <ul><li>LOCK_ACQUIRED should be returned when the proc has the lock and the proc is
-   * ready to execute.</li>
-   * <li>LOCK_YIELD_WAIT should be returned when the proc has not the lock and the framework
-   * should take care of readding the procedure back to the runnable set for retry</li>
-   * <li>LOCK_EVENT_WAIT should be returned when the proc has not the lock and someone will
-   * take care of readding the procedure back to the runnable set when the lock is available.
-   * </li></ul>
+   * The {@link #doAcquireLock(Object, ProcedureStore)} will be split into two steps, first, it will
+   * call us to determine whether we need to wait for initialization, second, it will call
+   * {@link #acquireLock(Object)} to actually handle the lock for this procedure.
+   * <p/>
+   * This is because that when master restarts, we need to restore the lock state for all the
+   * procedures to not break the semantic if {@link #holdLock(Object)} is true. But the
+   * {@link ProcedureExecutor} will be started before the master finish initialization(as it is part
+   * of the initialization!), so we need to split the code into two steps, and when restore, we just
+   * restore the lock part and ignore the waitInitialized part. Otherwise there will be dead lock.
+   * @return true means we need to wait until the environment has been initialized, otherwise true.
+   */
+  protected boolean waitInitialized(TEnvironment env) {
+    return false;
+  }
+
+  /**
+   * The user should override this method if they need a lock on an Entity. A lock can be anything,
+   * and it is up to the implementor. The Procedure Framework will call this method just before it
+   * invokes {@link #execute(Object)}. It calls {@link #releaseLock(Object)} after the call to
+   * execute.
+   * <p/>
+   * If you need to hold the lock for the life of the Procedure -- i.e. you do not want any other
+   * Procedure interfering while this Procedure is running, see {@link #holdLock(Object)}.
+   * <p/>
+   * Example: in our Master we can execute request in parallel for different tables. We can create
+   * t1 and create t2 and these creates can be executed at the same time. Anything else on t1/t2 is
+   * queued waiting that specific table create to happen.
+   * <p/>
+   * There are 3 LockState:
+   * <ul>
+   * <li>LOCK_ACQUIRED should be returned when the proc has the lock and the proc is ready to
+   * execute.</li>
+   * <li>LOCK_YIELD_WAIT should be returned when the proc has not the lock and the framework should
+   * take care of readding the procedure back to the runnable set for retry</li>
+   * <li>LOCK_EVENT_WAIT should be returned when the proc has not the lock and someone will take
+   * care of readding the procedure back to the runnable set when the lock is available.</li>
+   * </ul>
    * @return the lock state as described above.
    */
-  protected LockState acquireLock(final TEnvironment env) {
+  protected LockState acquireLock(TEnvironment env) {
     return LockState.LOCK_ACQUIRED;
   }
 
   /**
    * The user should override this method, and release lock if necessary.
    */
-  protected void releaseLock(final TEnvironment env) {
+  protected void releaseLock(TEnvironment env) {
     // no-op
   }
 
   /**
    * Used to keep the procedure lock even when the procedure is yielding or suspended.
-   * Must implement {@link #hasLock(Object)} if you want to hold the lock for life
-   * of the Procedure.
-   * @see #hasLock(Object)
    * @return true if the procedure should hold on the lock until completionCleanup()
    */
-  protected boolean holdLock(final TEnvironment env) {
+  protected boolean holdLock(TEnvironment env) {
     return false;
   }
 
@@ -235,8 +270,8 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * @see #holdLock(Object)
    * @return true if the procedure has the lock, false otherwise.
    */
-  protected boolean hasLock(final TEnvironment env) {
-    return false;
+  protected final boolean hasLock() {
+    return locked;
   }
 
   /**
@@ -245,7 +280,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * operation before replay.
    * e.g. failing the procedure if the state on replay may be unknown.
    */
-  protected void beforeReplay(final TEnvironment env) {
+  protected void beforeReplay(TEnvironment env) {
     // no-op
   }
 
@@ -253,7 +288,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * Called when the procedure is ready to be added to the queue after
    * the loading/replay operation.
    */
-  protected void afterReplay(final TEnvironment env) {
+  protected void afterReplay(TEnvironment env) {
     // no-op
   }
 
@@ -263,7 +298,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * This operation will not be retried on failure. If a procedure took a lock,
    * it will have been released when this method runs.
    */
-  protected void completionCleanup(final TEnvironment env) {
+  protected void completionCleanup(TEnvironment env) {
     // no-op
   }
 
@@ -275,7 +310,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * @return Return true if the executor should yield on completion of an execution step.
    *         Defaults to return false.
    */
-  protected boolean isYieldAfterExecutionStep(final TEnvironment env) {
+  protected boolean isYieldAfterExecutionStep(TEnvironment env) {
     return false;
   }
 
@@ -288,7 +323,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * @return true if the executor should wait the client ack for the result.
    *         Defaults to return true.
    */
-  protected boolean shouldWaitClientAck(final TEnvironment env) {
+  protected boolean shouldWaitClientAck(TEnvironment env) {
     return true;
   }
 
@@ -298,7 +333,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * @param env The environment passed to the procedure executor
    * @return Container object for procedure related metric
    */
-  protected ProcedureMetrics getProcedureMetrics(final TEnvironment env) {
+  protected ProcedureMetrics getProcedureMetrics(TEnvironment env) {
     return null;
   }
 
@@ -308,7 +343,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * updates submitted counter if {@link #getProcedureMetrics(Object)} returns non-null
    * {@link ProcedureMetrics}.
    */
-  protected void updateMetricsOnSubmit(final TEnvironment env) {
+  protected void updateMetricsOnSubmit(TEnvironment env) {
     ProcedureMetrics metrics = getProcedureMetrics(env);
     if (metrics == null) {
       return;
@@ -322,21 +357,19 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
 
   /**
    * This function will be called just after procedure execution is finished. Override this method
-   * to update metrics at the end of the procedure. If {@link #getProcedureMetrics(Object)}
-   * returns non-null {@link ProcedureMetrics}, the default implementation adds runtime of a
-   * procedure to a time histogram for successfully completed procedures. Increments failed
-   * counter for failed procedures.
-   *
-   * TODO: As any of the sub-procedures on failure rolls back all procedures in the stack,
-   * including successfully finished siblings, this function may get called twice in certain
-   * cases for certain procedures. Explore further if this can be called once.
-   *
+   * to update metrics at the end of the procedure. If {@link #getProcedureMetrics(Object)} returns
+   * non-null {@link ProcedureMetrics}, the default implementation adds runtime of a procedure to a
+   * time histogram for successfully completed procedures. Increments failed counter for failed
+   * procedures.
+   * <p/>
+   * TODO: As any of the sub-procedures on failure rolls back all procedures in the stack, including
+   * successfully finished siblings, this function may get called twice in certain cases for certain
+   * procedures. Explore further if this can be called once.
    * @param env The environment passed to the procedure executor
    * @param runtime Runtime of the procedure in milliseconds
    * @param success true if procedure is completed successfully
    */
-  protected void updateMetricsOnFinish(final TEnvironment env, final long runtime,
-                                       boolean success) {
+  protected void updateMetricsOnFinish(TEnvironment env, long runtime, boolean success) {
     ProcedureMetrics metrics = getProcedureMetrics(env);
     if (metrics == null) {
       return;
@@ -362,8 +395,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   }
 
   /**
-   * Build the StringBuilder for the simple form of
-   * procedure string.
+   * Build the StringBuilder for the simple form of procedure string.
    * @return the StringBuilder
    */
   protected StringBuilder toStringSimpleSB() {
@@ -389,6 +421,8 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
     sb.append(", state="); // pState for Procedure State as opposed to any other kind.
     toStringState(sb);
 
+    sb.append(", hasLock=").append(locked);
+
     if (hasException()) {
       sb.append(", exception=" + getException());
     }
@@ -400,8 +434,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   }
 
   /**
-   * Extend the toString() information with more procedure
-   * details
+   * Extend the toString() information with more procedure details
    */
   public String toStringDetails() {
     final StringBuilder sb = toStringSimpleSB();
@@ -429,8 +462,8 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   }
 
   /**
-   * Called from {@link #toString()} when interpolating {@link Procedure} State.
-   * Allows decorating generic Procedure State with Procedure particulars.
+   * Called from {@link #toString()} when interpolating {@link Procedure} State. Allows decorating
+   * generic Procedure State with Procedure particulars.
    * @param builder Append current {@link ProcedureState}
    */
   protected void toStringState(StringBuilder builder) {
@@ -493,8 +526,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * Called by the ProcedureExecutor to assign the ID to the newly created procedure.
    */
   @VisibleForTesting
-  @InterfaceAudience.Private
-  protected void setProcId(final long procId) {
+  protected void setProcId(long procId) {
     this.procId = procId;
     this.submittedTime = EnvironmentEdgeManager.currentTime();
     setState(ProcedureState.RUNNABLE);
@@ -503,13 +535,11 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   /**
    * Called by the ProcedureExecutor to assign the parent to the newly created procedure.
    */
-  @InterfaceAudience.Private
-  protected void setParentProcId(final long parentProcId) {
+  protected void setParentProcId(long parentProcId) {
     this.parentProcId = parentProcId;
   }
 
-  @InterfaceAudience.Private
-  protected void setRootProcId(final long rootProcId) {
+  protected void setRootProcId(long rootProcId) {
     this.rootProcId = rootProcId;
   }
 
@@ -517,18 +547,16 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * Called by the ProcedureExecutor to set the value to the newly created procedure.
    */
   @VisibleForTesting
-  @InterfaceAudience.Private
-  protected void setNonceKey(final NonceKey nonceKey) {
+  protected void setNonceKey(NonceKey nonceKey) {
     this.nonceKey = nonceKey;
   }
 
   @VisibleForTesting
-  @InterfaceAudience.Private
-  public void setOwner(final String owner) {
+  public void setOwner(String owner) {
     this.owner = StringUtils.isEmpty(owner) ? null : owner;
   }
 
-  public void setOwner(final User owner) {
+  public void setOwner(User owner) {
     assert owner != null : "expected owner to be not null";
     setOwner(owner.getShortName());
   }
@@ -537,8 +565,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * Called on store load to initialize the Procedure internals after
    * the creation/deserialization.
    */
-  @InterfaceAudience.Private
-  protected void setSubmittedTime(final long submittedTime) {
+  protected void setSubmittedTime(long submittedTime) {
     this.submittedTime = submittedTime;
   }
 
@@ -548,7 +575,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   /**
    * @param timeout timeout interval in msec
    */
-  protected void setTimeout(final int timeout) {
+  protected void setTimeout(int timeout) {
     this.timeout = timeout;
   }
 
@@ -567,15 +594,13 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * Called on store load to initialize the Procedure internals after
    * the creation/deserialization.
    */
-  @InterfaceAudience.Private
-  protected void setLastUpdate(final long lastUpdate) {
+  protected void setLastUpdate(long lastUpdate) {
     this.lastUpdate = lastUpdate;
   }
 
   /**
    * Called by ProcedureExecutor after each time a procedure step is executed.
    */
-  @InterfaceAudience.Private
   protected void updateTimestamp() {
     this.lastUpdate = EnvironmentEdgeManager.currentTime();
   }
@@ -590,7 +615,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * the procedure is in the waiting queue.
    * @return the timestamp of the next timeout.
    */
-  @InterfaceAudience.Private
   protected long getTimeoutTimestamp() {
     return getLastUpdate() + getTimeout();
   }
@@ -616,10 +640,19 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * The procedure may leave a "result" on completion.
    * @param result the serialized result that will be passed to the client
    */
-  protected void setResult(final byte[] result) {
+  protected void setResult(byte[] result) {
     this.result = result;
   }
 
+  /**
+   * Will only be called when loading procedures from procedure store, where we need to record
+   * whether the procedure has already held a lock. Later we will call
+   * {@link #doAcquireLock(Object)} to actually acquire the lock.
+   */
+  final void lockedWhenLoading() {
+    this.lockedWhenLoading = true;
+  }
+
   // ==============================================================================================
   //  Runtime state, updated every operation by the ProcedureExecutor
   //
@@ -677,13 +710,11 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   }
 
   @VisibleForTesting
-  @InterfaceAudience.Private
   protected synchronized void setState(final ProcedureState state) {
     this.state = state;
     updateTimestamp();
   }
 
-  @InterfaceAudience.Private
   public synchronized ProcedureState getState() {
     return state;
   }
@@ -705,10 +736,10 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
 
   /**
    * Called by the ProcedureExecutor when the timeout set by setTimeout() is expired.
-   * @return true to let the framework handle the timeout as abort,
-   *         false in case the procedure handled the timeout itself.
+   * @return true to let the framework handle the timeout as abort, false in case the procedure
+   *         handled the timeout itself.
    */
-  protected synchronized boolean setTimeoutFailure(final TEnvironment env) {
+  protected synchronized boolean setTimeoutFailure(TEnvironment env) {
     if (state == ProcedureState.WAITING_TIMEOUT) {
       long timeDiff = EnvironmentEdgeManager.currentTime() - lastUpdate;
       setFailure("ProcedureExecutor", new TimeoutIOException(
@@ -729,8 +760,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   /**
    * Called by the ProcedureExecutor on procedure-load to restore the latch state
    */
-  @InterfaceAudience.Private
-  protected synchronized void setChildrenLatch(final int numChildren) {
+  protected synchronized void setChildrenLatch(int numChildren) {
     this.childrenLatch = numChildren;
     if (LOG.isTraceEnabled()) {
       LOG.trace("CHILD LATCH INCREMENT SET " +
@@ -741,7 +771,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   /**
    * Called by the ProcedureExecutor on procedure-load to restore the latch state
    */
-  @InterfaceAudience.Private
   protected synchronized void incChildrenLatch() {
     // TODO: can this be inferred from the stack? I think so...
     this.childrenLatch++;
@@ -753,7 +782,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   /**
    * Called by the ProcedureExecutor to notify that one of the sub-procedures has completed.
    */
-  @InterfaceAudience.Private
   private synchronized boolean childrenCountDown() {
     assert childrenLatch > 0: this;
     boolean b = --childrenLatch == 0;
@@ -770,17 +798,18 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    */
   synchronized boolean tryRunnable() {
     // Don't use isWaiting in the below; it returns true for WAITING and WAITING_TIMEOUT
-    boolean b = getState() == ProcedureState.WAITING && childrenCountDown();
-    if (b) setState(ProcedureState.RUNNABLE);
-    return b;
+    if (getState() == ProcedureState.WAITING && childrenCountDown()) {
+      setState(ProcedureState.RUNNABLE);
+      return true;
+    } else {
+      return false;
+    }
   }
 
-  @InterfaceAudience.Private
   protected synchronized boolean hasChildren() {
     return childrenLatch > 0;
   }
 
-  @InterfaceAudience.Private
   protected synchronized int getChildrenLatch() {
     return childrenLatch;
   }
@@ -789,7 +818,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * Called by the RootProcedureState on procedure execution.
    * Each procedure store its stack-index positions.
    */
-  @InterfaceAudience.Private
   protected synchronized void addStackIndex(final int index) {
     if (stackIndexes == null) {
       stackIndexes = new int[] { index };
@@ -800,7 +828,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
     }
   }
 
-  @InterfaceAudience.Private
   protected synchronized boolean removeStackIndex() {
     if (stackIndexes != null && stackIndexes.length > 1) {
       stackIndexes = Arrays.copyOf(stackIndexes, stackIndexes.length - 1);
@@ -815,7 +842,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * Called on store load to initialize the Procedure internals after
    * the creation/deserialization.
    */
-  @InterfaceAudience.Private
   protected synchronized void setStackIndexes(final List<Integer> stackIndexes) {
     this.stackIndexes = new int[stackIndexes.size()];
     for (int i = 0; i < this.stackIndexes.length; ++i) {
@@ -823,12 +849,10 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
     }
   }
 
-  @InterfaceAudience.Private
   protected synchronized boolean wasExecuted() {
     return stackIndexes != null;
   }
 
-  @InterfaceAudience.Private
   protected synchronized int[] getStackIndexes() {
     return stackIndexes;
   }
@@ -840,10 +864,9 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   /**
    * Internal method called by the ProcedureExecutor that starts the user-level code execute().
    * @throws ProcedureSuspendedException This is used when procedure wants to halt processing and
-   * skip out without changing states or releasing any locks held.
+   *           skip out without changing states or releasing any locks held.
    */
-  @InterfaceAudience.Private
-  protected Procedure<TEnvironment>[] doExecute(final TEnvironment env)
+  protected Procedure<TEnvironment>[] doExecute(TEnvironment env)
       throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException {
     try {
       updateTimestamp();
@@ -856,8 +879,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   /**
    * Internal method called by the ProcedureExecutor that starts the user-level code rollback().
    */
-  @InterfaceAudience.Private
-  protected void doRollback(final TEnvironment env)
+  protected void doRollback(TEnvironment env)
       throws IOException, InterruptedException {
     try {
       updateTimestamp();
@@ -867,19 +889,60 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
     }
   }
 
+  final void restoreLock(TEnvironment env) {
+    if (!lockedWhenLoading) {
+      LOG.debug("{} didn't hold the lock before restarting, skip acquiring lock.", this);
+      return;
+    }
+
+    LOG.debug("{} held the lock before restarting, call acquireLock to restore it.", this);
+    LockState state = acquireLock(env);
+    assert state == LockState.LOCK_ACQUIRED;
+  }
+
   /**
    * Internal method called by the ProcedureExecutor that starts the user-level code acquireLock().
    */
-  @InterfaceAudience.Private
-  protected LockState doAcquireLock(final TEnvironment env) {
-    return acquireLock(env);
+  final LockState doAcquireLock(TEnvironment env, ProcedureStore store) {
+    if (waitInitialized(env)) {
+      return LockState.LOCK_EVENT_WAIT;
+    }
+    if (lockedWhenLoading) {
+      // reset it so we will not consider it anymore
+      lockedWhenLoading = false;
+      locked = true;
+      // Here we return without persist the locked state, as lockedWhenLoading is true means
+      // that the locked field of the procedure stored in procedure store is true, so we do not need
+      // to store it again.
+      return LockState.LOCK_ACQUIRED;
+    }
+    LockState state = acquireLock(env);
+    if (state == LockState.LOCK_ACQUIRED) {
+      locked = true;
+      // persist that we have held the lock. This must be done before we actually execute the
+      // procedure, otherwise when restarting, we may consider the procedure does not have a lock,
+      // but it may have already done some changes as we have already executed it, and if another
+      // procedure gets the lock, then the semantic will be broken if the holdLock is true, as we do
+      // not expect that another procedure can be executed in the middle.
+      store.update(this);
+    }
+    return state;
   }
 
   /**
    * Internal method called by the ProcedureExecutor that starts the user-level code releaseLock().
    */
-  @InterfaceAudience.Private
-  protected void doReleaseLock(final TEnvironment env) {
+  final void doReleaseLock(TEnvironment env, ProcedureStore store) {
+    locked = false;
+    // persist that we have released the lock. This must be done before we actually release the
+    // lock. Another procedure may take this lock immediately after we release the lock, and if we
+    // crash before persist the information that we have already released the lock, then when
+    // restarting there will be two procedures which both have the lock and cause problems.
+    if (getState() != ProcedureState.ROLLEDBACK) {
+      // If the state is ROLLEDBACK, it means that we have already deleted the procedure from
+      // procedure store, so do not need to log the release operation any more.
+      store.update(this);
+    }
     releaseLock(env);
   }
 
@@ -896,7 +959,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * Get an hashcode for the specified Procedure ID
    * @return the hashcode for the specified procId
    */
-  public static long getProcIdHashCode(final long procId) {
+  public static long getProcIdHashCode(long procId) {
     long h = procId;
     h ^= h >> 16;
     h *= 0x85ebca6b;
@@ -906,15 +969,16 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
     return h;
   }
 
-  /*
+  /**
    * Helper to lookup the root Procedure ID given a specified procedure.
    */
-  @InterfaceAudience.Private
-  protected static Long getRootProcedureId(final Map<Long, Procedure> procedures,
-      Procedure<?> proc) {
+  protected static <T> Long getRootProcedureId(Map<Long, Procedure<T>> procedures,
+      Procedure<T> proc) {
     while (proc.hasParent()) {
       proc = procedures.get(proc.getParentProcId());
-      if (proc == null) return null;
+      if (proc == null) {
+        return null;
+      }
     }
     return proc.getProcId();
   }
@@ -924,7 +988,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * @param b the second procedure to be compared.
    * @return true if the two procedures have the same parent
    */
-  public static boolean haveSameParent(final Procedure<?> a, final Procedure<?> b) {
+  public static boolean haveSameParent(Procedure<?> a, Procedure<?> b) {
     return a.hasParent() && b.hasParent() && (a.getParentProcId() == b.getParentProcId());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index db7c118..f1bec72 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -19,8 +19,10 @@
 package org.apache.hadoop.hbase.procedure2;
 
 import java.io.IOException;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Deque;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -48,7 +50,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.NonceKey;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -71,7 +72,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedu
  * and get the result via getResult(procId)
  */
 @InterfaceAudience.Private
-@InterfaceStability.Evolving
 public class ProcedureExecutor<TEnvironment> {
   private static final Logger LOG = LoggerFactory.getLogger(ProcedureExecutor.class);
 
@@ -108,16 +108,16 @@ public class ProcedureExecutor<TEnvironment> {
     void procedureFinished(long procId);
   }
 
-  private static class CompletedProcedureRetainer {
-    private final Procedure<?> procedure;
+  private static final class CompletedProcedureRetainer<TEnvironment> {
+    private final Procedure<TEnvironment> procedure;
     private long clientAckTime;
 
-    public CompletedProcedureRetainer(Procedure<?> procedure) {
+    public CompletedProcedureRetainer(Procedure<TEnvironment> procedure) {
       this.procedure = procedure;
       clientAckTime = -1;
     }
 
-    public Procedure<?> getProcedure() {
+    public Procedure<TEnvironment> getProcedure() {
       return procedure;
     }
 
@@ -172,13 +172,13 @@ public class ProcedureExecutor<TEnvironment> {
     private static final String BATCH_SIZE_CONF_KEY = "hbase.procedure.cleaner.evict.batch.size";
     private static final int DEFAULT_BATCH_SIZE = 32;
 
-    private final Map<Long, CompletedProcedureRetainer> completed;
+    private final Map<Long, CompletedProcedureRetainer<TEnvironment>> completed;
     private final Map<NonceKey, Long> nonceKeysToProcIdsMap;
     private final ProcedureStore store;
     private Configuration conf;
 
-    public CompletedProcedureCleaner(final Configuration conf, final ProcedureStore store,
-        final Map<Long, CompletedProcedureRetainer> completedMap,
+    public CompletedProcedureCleaner(Configuration conf, final ProcedureStore store,
+        final Map<Long, CompletedProcedureRetainer<TEnvironment>> completedMap,
         final Map<NonceKey, Long> nonceKeysToProcIdsMap) {
       // set the timeout interval that triggers the periodic-procedure
       super(conf.getInt(CLEANER_INTERVAL_CONF_KEY, DEFAULT_CLEANER_INTERVAL));
@@ -205,10 +205,11 @@ public class ProcedureExecutor<TEnvironment> {
       int batchCount = 0;
 
       final long now = EnvironmentEdgeManager.currentTime();
-      final Iterator<Map.Entry<Long, CompletedProcedureRetainer>> it = completed.entrySet().iterator();
+      final Iterator<Map.Entry<Long, CompletedProcedureRetainer<TEnvironment>>> it =
+        completed.entrySet().iterator();
       while (it.hasNext() && store.isRunning()) {
-        final Map.Entry<Long, CompletedProcedureRetainer> entry = it.next();
-        final CompletedProcedureRetainer retainer = entry.getValue();
+        final Map.Entry<Long, CompletedProcedureRetainer<TEnvironment>> entry = it.next();
+        final CompletedProcedureRetainer<TEnvironment> retainer = entry.getValue();
         final Procedure<?> proc = retainer.getProcedure();
 
         // TODO: Select TTL based on Procedure type
@@ -240,28 +241,32 @@ public class ProcedureExecutor<TEnvironment> {
    * Once a Root-Procedure completes (success or failure), the result will be added to this map.
    * The user of ProcedureExecutor should call getResult(procId) to get the result.
    */
-  private final ConcurrentHashMap<Long, CompletedProcedureRetainer> completed = new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<Long, CompletedProcedureRetainer<TEnvironment>> completed =
+    new ConcurrentHashMap<>();
 
   /**
    * Map the the procId returned by submitProcedure(), the Root-ProcID, to the RootProcedureState.
    * The RootProcedureState contains the execution stack of the Root-Procedure,
    * It is added to the map by submitProcedure() and removed on procedure completion.
    */
-  private final ConcurrentHashMap<Long, RootProcedureState> rollbackStack = new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<Long, RootProcedureState<TEnvironment>> rollbackStack =
+    new ConcurrentHashMap<>();
 
   /**
    * Helper map to lookup the live procedures by ID.
    * This map contains every procedure. root-procedures and subprocedures.
    */
-  private final ConcurrentHashMap<Long, Procedure> procedures = new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<Long, Procedure<TEnvironment>> procedures =
+    new ConcurrentHashMap<>();
 
   /**
-   * Helper map to lookup whether the procedure already issued from the same client.
-   * This map contains every root procedure.
+   * Helper map to lookup whether the procedure already issued from the same client. This map
+   * contains every root procedure.
    */
   private final ConcurrentHashMap<NonceKey, Long> nonceKeysToProcIdsMap = new ConcurrentHashMap<>();
 
-  private final CopyOnWriteArrayList<ProcedureExecutorListener> listeners = new CopyOnWriteArrayList<>();
+  private final CopyOnWriteArrayList<ProcedureExecutorListener> listeners =
+    new CopyOnWriteArrayList<>();
 
   private Configuration conf;
 
@@ -287,7 +292,7 @@ public class ProcedureExecutor<TEnvironment> {
    * Overridden when we do the ProcedureTestingUtility.testRecoveryAndDoubleExecution trickery
    * (Should be ok).
    */
-  private TimeoutExecutorThread timeoutExecutor;
+  private TimeoutExecutorThread<TEnvironment> timeoutExecutor;
 
   private int corePoolSize;
   private int maxPoolSize;
@@ -357,27 +362,68 @@ public class ProcedureExecutor<TEnvironment> {
     });
   }
 
-  private void loadProcedures(final ProcedureIterator procIter,
-      final boolean abortOnCorruption) throws IOException {
-    final boolean debugEnabled = LOG.isDebugEnabled();
+  private void restoreLock(Procedure<TEnvironment> proc, Set<Long> restored) {
+    proc.restoreLock(getEnvironment());
+    restored.add(proc.getProcId());
+  }
+
+  private void restoreLocks(Deque<Procedure<TEnvironment>> stack, Set<Long> restored) {
+    while (!stack.isEmpty()) {
+      restoreLock(stack.pop(), restored);
+    }
+  }
+
+  // Restore the locks for all the procedures.
+  // Notice that we need to restore the locks starting from the root proc, otherwise there will be
+  // problem that a sub procedure may hold the exclusive lock first and then we are stuck when
+  // calling the acquireLock method for the parent procedure.
+  // The algorithm is straight-forward:
+  // 1. Use a set to record the procedures which locks have already been restored.
+  // 2. Use a stack to store the hierarchy of the procedures
+  // 3. For all the procedure, we will first try to find its parent and push it into the stack,
+  // unless
+  // a. We have no parent, i.e, we are the root procedure
+  // b. The lock has already been restored(by checking the set introduced in #1)
+  // then we start to pop the stack and call acquireLock for each procedure.
+  // Notice that this should be done for all procedures, not only the ones in runnableList.
+  private void restoreLocks() {
+    Set<Long> restored = new HashSet<>();
+    Deque<Procedure<TEnvironment>> stack = new ArrayDeque<>();
+    procedures.values().forEach(proc -> {
+      for (;;) {
+        if (restored.contains(proc.getProcId())) {
+          restoreLocks(stack, restored);
+          return;
+        }
+        if (!proc.hasParent()) {
+          restoreLock(proc, restored);
+          restoreLocks(stack, restored);
+          return;
+        }
+        stack.push(proc);
+        proc = procedures.get(proc.getParentProcId());
+      }
+    });
+  }
 
+  private void loadProcedures(ProcedureIterator procIter, boolean abortOnCorruption)
+      throws IOException {
     // 1. Build the rollback stack
     int runnablesCount = 0;
+    int failedCount = 0;
     while (procIter.hasNext()) {
       boolean finished = procIter.isNextFinished();
-      Procedure proc = procIter.next();
+      Procedure<TEnvironment> proc = procIter.next();
       NonceKey nonceKey = proc.getNonceKey();
       long procId = proc.getProcId();
 
       if (finished) {
-        completed.put(proc.getProcId(), new CompletedProcedureRetainer(proc));
-        if (debugEnabled) {
-          LOG.debug("Completed " + proc);
-        }
+        completed.put(proc.getProcId(), new CompletedProcedureRetainer<>(proc));
+        LOG.debug("Completed {}", proc);
       } else {
         if (!proc.hasParent()) {
           assert !proc.isFinished() : "unexpected finished procedure";
-          rollbackStack.put(proc.getProcId(), new RootProcedureState());
+          rollbackStack.put(proc.getProcId(), new RootProcedureState<>());
         }
 
         // add the procedure to the map
@@ -386,6 +432,8 @@ public class ProcedureExecutor<TEnvironment> {
 
         if (proc.getState() == ProcedureState.RUNNABLE) {
           runnablesCount++;
+        } else if (proc.getState() == ProcedureState.FAILED) {
+          failedCount++;
         }
       }
 
@@ -396,8 +444,19 @@ public class ProcedureExecutor<TEnvironment> {
     }
 
     // 2. Initialize the stacks
-    final ArrayList<Procedure> runnableList = new ArrayList(runnablesCount);
-    HashSet<Procedure> waitingSet = null;
+    // In the old implementation, for procedures in FAILED state, we will push it into the
+    // ProcedureScheduler directly to execute the rollback. But this does not work after we
+    // introduce the restore lock stage.
+    // For now, when we acquire a xlock, we will remove the queue from runQueue in scheduler, and
+    // then when a procedure which has lock access, for example, a sub procedure of the procedure
+    // which has the xlock, is pushed into the scheduler, we will add the queue back to let the
+    // workers poll from it. The assumption here is that, the procedure which has the xlock should
+    // have been polled out already, so when loading we can not add the procedure to scheduler first
+    // and then call acquireLock, since the procedure is still in the queue, and since we will
+    // remove the queue from runQueue, then no one can poll it out, then there is a dead lock
+    List<Procedure<TEnvironment>> runnableList = new ArrayList<>(runnablesCount);
+    List<Procedure<TEnvironment>> failedList = new ArrayList<>(failedCount);
+    Set<Procedure<TEnvironment>> waitingSet = null;
     procIter.reset();
     while (procIter.hasNext()) {
       if (procIter.isNextFinished()) {
@@ -405,12 +464,10 @@ public class ProcedureExecutor<TEnvironment> {
         continue;
       }
 
-      Procedure proc = procIter.next();
+      Procedure<TEnvironment> proc = procIter.next();
       assert !(proc.isFinished() && !proc.hasParent()) : "unexpected completed proc=" + proc;
 
-      if (debugEnabled) {
-        LOG.debug(String.format("Loading %s", proc));
-      }
+      LOG.debug("Loading {}", proc);
 
       Long rootProcId = getRootProcedureId(proc);
       if (rootProcId == null) {
@@ -420,14 +477,14 @@ public class ProcedureExecutor<TEnvironment> {
       }
 
       if (proc.hasParent()) {
-        Procedure parent = procedures.get(proc.getParentProcId());
+        Procedure<TEnvironment> parent = procedures.get(proc.getParentProcId());
         // corrupted procedures are handled later at step 3
         if (parent != null && !proc.isFinished()) {
           parent.incChildrenLatch();
         }
       }
 
-      RootProcedureState procStack = rollbackStack.get(rootProcId);
+      RootProcedureState<TEnvironment> procStack = rollbackStack.get(rootProcId);
       procStack.loadStack(proc);
 
       proc.setRootProcId(rootProcId);
@@ -447,8 +504,7 @@ public class ProcedureExecutor<TEnvironment> {
           waitingSet.add(proc);
           break;
         case FAILED:
-          // add the proc to the scheduler to perform the rollback
-          scheduler.addBack(proc);
+          failedList.add(proc);
           break;
         case ROLLEDBACK:
         case INITIALIZING:
@@ -462,13 +518,14 @@ public class ProcedureExecutor<TEnvironment> {
 
     // 3. Validate the stacks
     int corruptedCount = 0;
-    Iterator<Map.Entry<Long, RootProcedureState>> itStack = rollbackStack.entrySet().iterator();
+    Iterator<Map.Entry<Long, RootProcedureState<TEnvironment>>> itStack =
+      rollbackStack.entrySet().iterator();
     while (itStack.hasNext()) {
-      Map.Entry<Long, RootProcedureState> entry = itStack.next();
-      RootProcedureState procStack = entry.getValue();
+      Map.Entry<Long, RootProcedureState<TEnvironment>> entry = itStack.next();
+      RootProcedureState<TEnvironment> procStack = entry.getValue();
       if (procStack.isValid()) continue;
 
-      for (Procedure proc: procStack.getSubproceduresStack()) {
+      for (Procedure<TEnvironment> proc : procStack.getSubproceduresStack()) {
         LOG.error("Corrupted " + proc);
         procedures.remove(proc.getProcId());
         runnableList.remove(proc);
@@ -484,30 +541,22 @@ public class ProcedureExecutor<TEnvironment> {
 
     // 4. Push the procedures to the timeout executor
     if (waitingSet != null && !waitingSet.isEmpty()) {
-      for (Procedure proc: waitingSet) {
+      for (Procedure<TEnvironment> proc: waitingSet) {
         proc.afterReplay(getEnvironment());
         timeoutExecutor.add(proc);
       }
     }
-
-    // 5. Push the procedure to the scheduler
-    if (!runnableList.isEmpty()) {
-      // TODO: See ProcedureWALFormatReader#hasFastStartSupport
-      // some procedure may be started way before this stuff.
-      for (int i = runnableList.size() - 1; i >= 0; --i) {
-        Procedure proc = runnableList.get(i);
-        proc.afterReplay(getEnvironment());
-        if (!proc.hasParent()) {
-          sendProcedureLoadedNotification(proc.getProcId());
-        }
-        if (proc.wasExecuted()) {
-          scheduler.addFront(proc);
-        } else {
-          // if it was not in execution, it can wait.
-          scheduler.addBack(proc);
-        }
+    // 5. restore locks
+    restoreLocks();
+    // 6. Push the procedure to the scheduler
+    failedList.forEach(scheduler::addBack);
+    runnableList.forEach(p -> {
+      p.afterReplay(getEnvironment());
+      if (!p.hasParent()) {
+        sendProcedureLoadedNotification(p.getProcId());
       }
-    }
+      scheduler.addBack(p);
+    });
   }
 
   /**
@@ -529,7 +578,7 @@ public class ProcedureExecutor<TEnvironment> {
         corePoolSize, maxPoolSize);
 
     this.threadGroup = new ThreadGroup("PEWorkerGroup");
-    this.timeoutExecutor = new TimeoutExecutorThread(this, threadGroup);
+    this.timeoutExecutor = new TimeoutExecutorThread<>(this, threadGroup);
 
     // Create the workers
     workerId.set(0);
@@ -581,7 +630,7 @@ public class ProcedureExecutor<TEnvironment> {
     timeoutExecutor.add(new WorkerMonitor());
 
     // Add completed cleaner chore
-    addChore(new CompletedProcedureCleaner(conf, store, completed, nonceKeysToProcIdsMap));
+    addChore(new CompletedProcedureCleaner<>(conf, store, completed, nonceKeysToProcIdsMap));
   }
 
   public void stop() {
@@ -686,7 +735,7 @@ public class ProcedureExecutor<TEnvironment> {
    * Add a chore procedure to the executor
    * @param chore the chore to add
    */
-  public void addChore(final ProcedureInMemoryChore chore) {
+  public void addChore(ProcedureInMemoryChore<TEnvironment> chore) {
     chore.setState(ProcedureState.WAITING_TIMEOUT);
     timeoutExecutor.add(chore);
   }
@@ -696,7 +745,7 @@ public class ProcedureExecutor<TEnvironment> {
    * @param chore the chore to remove
    * @return whether the chore is removed, or it will be removed later
    */
-  public boolean removeChore(final ProcedureInMemoryChore chore) {
+  public boolean removeChore(ProcedureInMemoryChore<TEnvironment> chore) {
     chore.setState(ProcedureState.SUCCESS);
     return timeoutExecutor.remove(chore);
   }
@@ -830,17 +879,21 @@ public class ProcedureExecutor<TEnvironment> {
    * @param procOwner name of the owner of the procedure, used to inform the user
    * @param exception the failure to report to the user
    */
-  public void setFailureResultForNonce(final NonceKey nonceKey, final String procName,
-      final User procOwner, final IOException exception) {
-    if (nonceKey == null) return;
+  public void setFailureResultForNonce(NonceKey nonceKey, String procName, User procOwner,
+      IOException exception) {
+    if (nonceKey == null) {
+      return;
+    }
 
-    final Long procId = nonceKeysToProcIdsMap.get(nonceKey);
-    if (procId == null || completed.containsKey(procId)) return;
+    Long procId = nonceKeysToProcIdsMap.get(nonceKey);
+    if (procId == null || completed.containsKey(procId)) {
+      return;
+    }
 
-    Procedure<?> proc = new FailedProcedure(procId.longValue(),
-        procName, procOwner, nonceKey, exception);
+    Procedure<TEnvironment> proc =
+      new FailedProcedure<>(procId.longValue(), procName, procOwner, nonceKey, exception);
 
-    completed.putIfAbsent(procId, new CompletedProcedureRetainer(proc));
+    completed.putIfAbsent(procId, new CompletedProcedureRetainer<>(proc));
   }
 
   // ==========================================================================
@@ -851,7 +904,7 @@ public class ProcedureExecutor<TEnvironment> {
    * @param proc the new procedure to execute.
    * @return the procedure id, that can be used to monitor the operation
    */
-  public long submitProcedure(final Procedure proc) {
+  public long submitProcedure(Procedure<TEnvironment> proc) {
     return submitProcedure(proc, null);
   }
 
@@ -863,7 +916,7 @@ public class ProcedureExecutor<TEnvironment> {
    */
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH",
       justification = "FindBugs is blind to the check-for-null")
-  public long submitProcedure(final Procedure proc, final NonceKey nonceKey) {
+  public long submitProcedure(Procedure<TEnvironment> proc, NonceKey nonceKey) {
     Preconditions.checkArgument(lastProcId.get() >= 0);
 
     prepareProcedure(proc);
@@ -883,9 +936,7 @@ public class ProcedureExecutor<TEnvironment> {
 
     // Commit the transaction
     store.insert(proc, null);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Stored " + proc);
-    }
+    LOG.debug("Stored {}", proc);
 
     // Add the procedure to the executor
     return pushProcedure(proc);
@@ -896,7 +947,7 @@ public class ProcedureExecutor<TEnvironment> {
    * @param procs the new procedures to execute.
    */
   // TODO: Do we need to take nonces here?
-  public void submitProcedures(final Procedure[] procs) {
+  public void submitProcedures(Procedure<TEnvironment>[] procs) {
     Preconditions.checkArgument(lastProcId.get() >= 0);
     if (procs == null || procs.length <= 0) {
       return;
@@ -919,7 +970,7 @@ public class ProcedureExecutor<TEnvironment> {
     }
   }
 
-  private Procedure prepareProcedure(final Procedure proc) {
+  private Procedure<TEnvironment> prepareProcedure(Procedure<TEnvironment> proc) {
     Preconditions.checkArgument(proc.getState() == ProcedureState.INITIALIZING);
     Preconditions.checkArgument(!proc.hasParent(), "unexpected parent", proc);
     if (this.checkOwnerSet) {
@@ -928,14 +979,14 @@ public class ProcedureExecutor<TEnvironment> {
     return proc;
   }
 
-  private long pushProcedure(final Procedure proc) {
+  private long pushProcedure(Procedure<TEnvironment> proc) {
     final long currentProcId = proc.getProcId();
 
     // Update metrics on start of a procedure
     proc.updateMetricsOnSubmit(getEnvironment());
 
     // Create the rollback stack for the procedure
-    RootProcedureState stack = new RootProcedureState();
+    RootProcedureState<TEnvironment> stack = new RootProcedureState<>();
     rollbackStack.put(currentProcId, stack);
 
     // Submit the new subprocedures
@@ -952,7 +1003,7 @@ public class ProcedureExecutor<TEnvironment> {
    * @param procId the procedure to abort
    * @return true if the procedure exists and has received the abort, otherwise false.
    */
-  public boolean abort(final long procId) {
+  public boolean abort(long procId) {
     return abort(procId, true);
   }
 
@@ -963,8 +1014,8 @@ public class ProcedureExecutor<TEnvironment> {
    * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
    * @return true if the procedure exists and has received the abort, otherwise false.
    */
-  public boolean abort(final long procId, final boolean mayInterruptIfRunning) {
-    final Procedure proc = procedures.get(procId);
+  public boolean abort(long procId, boolean mayInterruptIfRunning) {
+    Procedure<TEnvironment> proc = procedures.get(procId);
     if (proc != null) {
       if (!mayInterruptIfRunning && proc.wasExecuted()) {
         return false;
@@ -977,20 +1028,20 @@ public class ProcedureExecutor<TEnvironment> {
   // ==========================================================================
   //  Executor query helpers
   // ==========================================================================
-  public Procedure getProcedure(final long procId) {
+  public Procedure<TEnvironment> getProcedure(final long procId) {
     return procedures.get(procId);
   }
 
-  public <T extends Procedure> T getProcedure(final Class<T> clazz, final long procId) {
-    final Procedure proc = getProcedure(procId);
+  public <T extends Procedure<TEnvironment>> T getProcedure(Class<T> clazz, long procId) {
+    Procedure<TEnvironment> proc = getProcedure(procId);
     if (clazz.isInstance(proc)) {
-      return (T)proc;
+      return clazz.cast(proc);
     }
     return null;
   }
 
-  public Procedure getResult(final long procId) {
-    CompletedProcedureRetainer retainer = completed.get(procId);
+  public Procedure<TEnvironment> getResult(long procId) {
+    CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId);
     if (retainer == null) {
       return null;
     } else {
@@ -1014,8 +1065,8 @@ public class ProcedureExecutor<TEnvironment> {
    * @param procId the ID of the procedure to check
    * @return true if the procedure execution is started, otherwise false.
    */
-  public boolean isStarted(final long procId) {
-    final Procedure proc = procedures.get(procId);
+  public boolean isStarted(long procId) {
+    Procedure<?> proc = procedures.get(procId);
     if (proc == null) {
       return completed.get(procId) != null;
     }
@@ -1026,13 +1077,11 @@ public class ProcedureExecutor<TEnvironment> {
    * Mark the specified completed procedure, as ready to remove.
    * @param procId the ID of the procedure to remove
    */
-  public void removeResult(final long procId) {
-    CompletedProcedureRetainer retainer = completed.get(procId);
+  public void removeResult(long procId) {
+    CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId);
     if (retainer == null) {
       assert !procedures.containsKey(procId) : "pid=" + procId + " is still running";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("pid=" + procId + " already removed by the cleaner.");
-      }
+      LOG.debug("pid={} already removed by the cleaner.", procId);
       return;
     }
 
@@ -1040,8 +1089,8 @@ public class ProcedureExecutor<TEnvironment> {
     retainer.setClientAckTime(EnvironmentEdgeManager.currentTime());
   }
 
-  public Procedure getResultOrProcedure(final long procId) {
-    CompletedProcedureRetainer retainer = completed.get(procId);
+  public Procedure<TEnvironment> getResultOrProcedure(long procId) {
+    CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId);
     if (retainer == null) {
       return procedures.get(procId);
     } else {
@@ -1056,15 +1105,16 @@ public class ProcedureExecutor<TEnvironment> {
    * @return true if the user is the owner of the procedure,
    *   false otherwise or the owner is unknown.
    */
-  public boolean isProcedureOwner(final long procId, final User user) {
-    if (user == null) return false;
-
-    final Procedure runningProc = procedures.get(procId);
+  public boolean isProcedureOwner(long procId, User user) {
+    if (user == null) {
+      return false;
+    }
+    final Procedure<TEnvironment> runningProc = procedures.get(procId);
     if (runningProc != null) {
       return runningProc.getOwner().equals(user.getShortName());
     }
 
-    final CompletedProcedureRetainer retainer = completed.get(procId);
+    final CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId);
     if (retainer != null) {
       return retainer.getProcedure().getOwner().equals(user.getShortName());
     }
@@ -1078,19 +1128,17 @@ public class ProcedureExecutor<TEnvironment> {
    * Get procedures.
    * @return the procedures in a list
    */
-  public List<Procedure<?>> getProcedures() {
-    final List<Procedure<?>> procedureLists = new ArrayList<>(procedures.size() + completed.size());
-    for (Procedure<?> procedure : procedures.values()) {
-      procedureLists.add(procedure);
-    }
+  public List<Procedure<TEnvironment>> getProcedures() {
+    List<Procedure<TEnvironment>> procedureList =
+      new ArrayList<>(procedures.size() + completed.size());
+    procedureList.addAll(procedures.values());
     // Note: The procedure could show up twice in the list with different state, as
     // it could complete after we walk through procedures list and insert into
     // procedureList - it is ok, as we will use the information in the Procedure
     // to figure it out; to prevent this would increase the complexity of the logic.
-    for (CompletedProcedureRetainer retainer: completed.values()) {
-      procedureLists.add(retainer.getProcedure());
-    }
-    return procedureLists;
+    completed.values().stream().map(CompletedProcedureRetainer::getProcedure)
+      .forEach(procedureList::add);
+    return procedureList;
   }
 
   // ==========================================================================
@@ -1169,14 +1217,14 @@ public class ProcedureExecutor<TEnvironment> {
     return procedures.keySet();
   }
 
-  Long getRootProcedureId(Procedure proc) {
+  Long getRootProcedureId(Procedure<TEnvironment> proc) {
     return Procedure.getRootProcedureId(procedures, proc);
   }
 
   // ==========================================================================
   //  Executions
   // ==========================================================================
-  private void executeProcedure(final Procedure proc) {
+  private void executeProcedure(Procedure<TEnvironment> proc) {
     final Long rootProcId = getRootProcedureId(proc);
     if (rootProcId == null) {
       // The 'proc' was ready to run but the root procedure was rolledback
@@ -1185,7 +1233,7 @@ public class ProcedureExecutor<TEnvironment> {
       return;
     }
 
-    final RootProcedureState procStack = rollbackStack.get(rootProcId);
+    RootProcedureState<TEnvironment> procStack = rollbackStack.get(rootProcId);
     if (procStack == null) {
       LOG.warn("RootProcedureState is null for " + proc.getProcId());
       return;
@@ -1197,7 +1245,7 @@ public class ProcedureExecutor<TEnvironment> {
           // we have the 'rollback-lock' we can start rollingback
           switch (executeRollback(rootProcId, procStack)) {
             case LOCK_ACQUIRED:
-                break;
+              break;
             case LOCK_YIELD_WAIT:
               procStack.unsetRollback();
               scheduler.yield(proc);
@@ -1239,7 +1287,6 @@ public class ProcedureExecutor<TEnvironment> {
       switch (lockState) {
         case LOCK_ACQUIRED:
           execProcedure(procStack, proc);
-          releaseLock(proc, false);
           break;
         case LOCK_YIELD_WAIT:
           LOG.info(lockState + " " + proc);
@@ -1254,12 +1301,6 @@ public class ProcedureExecutor<TEnvironment> {
       }
       procStack.release(proc);
 
-      // allows to kill the executor before something is stored to the wal.
-      // useful to test the procedure recovery.
-      if (testing != null && !isRunning()) {
-        break;
-      }
-
       if (proc.isSuccess()) {
         // update metrics on finishing the procedure
         proc.updateMetricsOnFinish(getEnvironment(), proc.elapsedTime(), true);
@@ -1275,33 +1316,31 @@ public class ProcedureExecutor<TEnvironment> {
     } while (procStack.isFailed());
   }
 
-  private LockState acquireLock(final Procedure proc) {
-    final TEnvironment env = getEnvironment();
-    // hasLock() is used in conjunction with holdLock().
-    // This allows us to not rewrite or carry around the hasLock() flag
-    // for every procedure. the hasLock() have meaning only if holdLock() is true.
-    if (proc.holdLock(env) && proc.hasLock(env)) {
+  private LockState acquireLock(Procedure<TEnvironment> proc) {
+    TEnvironment env = getEnvironment();
+    // if holdLock is true, then maybe we already have the lock, so just return LOCK_ACQUIRED if
+    // hasLock is true.
+    if (proc.hasLock()) {
       return LockState.LOCK_ACQUIRED;
     }
-    return proc.doAcquireLock(env);
+    return proc.doAcquireLock(env, store);
   }
 
-  private void releaseLock(final Procedure proc, final boolean force) {
-    final TEnvironment env = getEnvironment();
+  private void releaseLock(Procedure<TEnvironment> proc, boolean force) {
+    TEnvironment env = getEnvironment();
     // For how the framework works, we know that we will always have the lock
     // when we call releaseLock(), so we can avoid calling proc.hasLock()
-    if (force || !proc.holdLock(env)) {
-      proc.doReleaseLock(env);
+    if (force || !proc.holdLock(env) || proc.isFinished()) {
+      proc.doReleaseLock(env, store);
     }
   }
 
   /**
-   * Execute the rollback of the full procedure stack.
-   * Once the procedure is rolledback, the root-procedure will be visible as
-   * finished to user, and the result will be the fatal exception.
+   * Execute the rollback of the full procedure stack. Once the procedure is rolledback, the
+   * root-procedure will be visible as finished to user, and the result will be the fatal exception.
    */
-  private LockState executeRollback(final long rootProcId, final RootProcedureState procStack) {
-    final Procedure rootProc = procedures.get(rootProcId);
+  private LockState executeRollback(long rootProcId, RootProcedureState<TEnvironment> procStack) {
+    Procedure<TEnvironment> rootProc = procedures.get(rootProcId);
     RemoteProcedureException exception = rootProc.getException();
     // TODO: This needs doc. The root proc doesn't have an exception. Maybe we are
     // rolling back because the subprocedure does. Clarify.
@@ -1311,13 +1350,13 @@ public class ProcedureExecutor<TEnvironment> {
       store.update(rootProc);
     }
 
-    final List<Procedure> subprocStack = procStack.getSubproceduresStack();
+    List<Procedure<TEnvironment>> subprocStack = procStack.getSubproceduresStack();
     assert subprocStack != null : "Called rollback with no steps executed rootProc=" + rootProc;
 
     int stackTail = subprocStack.size();
     boolean reuseLock = false;
     while (stackTail --> 0) {
-      final Procedure proc = subprocStack.get(stackTail);
+      Procedure<TEnvironment> proc = subprocStack.get(stackTail);
 
       LockState lockState;
       if (!reuseLock && (lockState = acquireLock(proc)) != LockState.LOCK_ACQUIRED) {
@@ -1334,7 +1373,7 @@ public class ProcedureExecutor<TEnvironment> {
       // (e.g. StateMachineProcedure reuse the same instance)
       // we can avoid to lock/unlock each step
       reuseLock = stackTail > 0 && (subprocStack.get(stackTail - 1) == proc) && !abortRollback;
-      if (!reuseLock) {
+      if (!reuseLock && proc.hasLock()) {
         releaseLock(proc, false);
       }
 
@@ -1368,13 +1407,11 @@ public class ProcedureExecutor<TEnvironment> {
    * It updates the store with the new state (stack index)
    * or will remove completly the procedure in case it is a child.
    */
-  private LockState executeRollback(final Procedure proc) {
+  private LockState executeRollback(Procedure<TEnvironment> proc) {
     try {
       proc.doRollback(getEnvironment());
     } catch (IOException e) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Roll back attempt failed for " + proc, e);
-      }
+      LOG.debug("Roll back attempt failed for {}", proc, e);
       return LockState.LOCK_YIELD_WAIT;
     } catch (InterruptedException e) {
       handleInterruptedException(proc, e);
@@ -1387,9 +1424,10 @@ public class ProcedureExecutor<TEnvironment> {
     // allows to kill the executor before something is stored to the wal.
     // useful to test the procedure recovery.
     if (testing != null && testing.shouldKillBeforeStoreUpdate()) {
-      LOG.debug("TESTING: Kill before store update");
+      String msg = "TESTING: Kill before store update";
+      LOG.debug(msg);
       stop();
-      return LockState.LOCK_YIELD_WAIT;
+      throw new RuntimeException(msg);
     }
 
     if (proc.removeStackIndex()) {
@@ -1416,6 +1454,11 @@ public class ProcedureExecutor<TEnvironment> {
     return LockState.LOCK_ACQUIRED;
   }
 
+  private void yieldProcedure(Procedure<TEnvironment> proc) {
+    releaseLock(proc, false);
+    scheduler.yield(proc);
+  }
+
   /**
    * Executes <code>procedure</code>
    * <ul>
@@ -1445,10 +1488,10 @@ public class ProcedureExecutor<TEnvironment> {
    *  </li>
    *  </ul>
    */
-  private void execProcedure(final RootProcedureState procStack,
-      final Procedure<TEnvironment> procedure) {
+  private void execProcedure(RootProcedureState<TEnvironment> procStack,
+      Procedure<TEnvironment> procedure) {
     Preconditions.checkArgument(procedure.getState() == ProcedureState.RUNNABLE,
-        procedure.toString());
+      procedure.toString());
 
     // Procedures can suspend themselves. They skip out by throwing a ProcedureSuspendedException.
     // The exception is caught below and then we hurry to the exit without disturbing state. The
@@ -1475,22 +1518,16 @@ public class ProcedureExecutor<TEnvironment> {
           subprocs = null;
         }
       } catch (ProcedureSuspendedException e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Suspend " + procedure);
-        }
+        LOG.trace("Suspend {}", procedure);
         suspended = true;
       } catch (ProcedureYieldException e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Yield " + procedure + ": " + e.getMessage(), e);
-        }
-        scheduler.yield(procedure);
+        LOG.trace("Yield {}", procedure, e);
+        yieldProcedure(procedure);
         return;
       } catch (InterruptedException e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Yield interrupt " + procedure + ": " + e.getMessage(), e);
-        }
+        LOG.trace("Yield interrupt {}", procedure, e);
         handleInterruptedException(procedure, e);
-        scheduler.yield(procedure);
+        yieldProcedure(procedure);
         return;
       } catch (Throwable e) {
         // Catch NullPointerExceptions or similar errors...
@@ -1506,9 +1543,7 @@ public class ProcedureExecutor<TEnvironment> {
             // i.e. we go around this loop again rather than go back out on the scheduler queue.
             subprocs = null;
             reExecute = true;
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Short-circuit to next step on pid=" + procedure.getProcId());
-            }
+            LOG.trace("Short-circuit to next step on pid={}", procedure.getProcId());
           } else {
             // Yield the current procedure, and make the subprocedure runnable
             // subprocs may come back 'null'.
@@ -1519,9 +1554,7 @@ public class ProcedureExecutor<TEnvironment> {
                 collect(Collectors.toList()).toString()));
           }
         } else if (procedure.getState() == ProcedureState.WAITING_TIMEOUT) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Added to timeoutExecutor " + procedure);
-          }
+          LOG.trace("Added to timeoutExecutor {}", procedure);
           timeoutExecutor.add(procedure);
         } else if (!suspended) {
           // No subtask, so we are done
@@ -1535,9 +1568,10 @@ public class ProcedureExecutor<TEnvironment> {
       // allows to kill the executor before something is stored to the wal.
       // useful to test the procedure recovery.
       if (testing != null && testing.shouldKillBeforeStoreUpdate(suspended)) {
-        LOG.debug("TESTING: Kill before store update: " + procedure);
+        String msg = "TESTING: Kill before store update: " + procedure;
+        LOG.debug(msg);
         stop();
-        return;
+        throw new RuntimeException(msg);
       }
 
       // TODO: The code here doesn't check if store is running before persisting to the store as
@@ -1551,11 +1585,13 @@ public class ProcedureExecutor<TEnvironment> {
       updateStoreOnExec(procStack, procedure, subprocs);
 
       // if the store is not running we are aborting
-      if (!store.isRunning()) return;
+      if (!store.isRunning()) {
+        return;
+      }
       // if the procedure is kind enough to pass the slot to someone else, yield
       if (procedure.isRunnable() && !suspended &&
           procedure.isYieldAfterExecutionStep(getEnvironment())) {
-        scheduler.yield(procedure);
+        yieldProcedure(procedure);
         return;
       }
 
@@ -1566,6 +1602,11 @@ public class ProcedureExecutor<TEnvironment> {
       submitChildrenProcedures(subprocs);
     }
 
+    // we need to log the release lock operation before waking up the parent procedure, as there
+    // could be race that the parent procedure may call updateStoreOnExec ahead of us and remove all
+    // the sub procedures from store and cause problems...
+    releaseLock(procedure, false);
+
     // if the procedure is complete and has a parent, count down the children latch.
     // If 'suspended', do nothing to change state -- let other threads handle unsuspend event.
     if (!suspended && procedure.isFinished() && procedure.hasParent()) {
@@ -1573,12 +1614,12 @@ public class ProcedureExecutor<TEnvironment> {
     }
   }
 
-  private Procedure[] initializeChildren(final RootProcedureState procStack,
-      final Procedure procedure, final Procedure[] subprocs) {
+  private Procedure<TEnvironment>[] initializeChildren(RootProcedureState<TEnvironment> procStack,
+      Procedure<TEnvironment> procedure, Procedure<TEnvironment>[] subprocs) {
     assert subprocs != null : "expected subprocedures";
     final long rootProcId = getRootProcedureId(procedure);
     for (int i = 0; i < subprocs.length; ++i) {
-      final Procedure subproc = subprocs[i];
+      Procedure<TEnvironment> subproc = subprocs[i];
       if (subproc == null) {
         String msg = "subproc[" + i + "] is null, aborting the procedure";
         procedure.setFailure(new RemoteProcedureException(msg,
@@ -1609,9 +1650,9 @@ public class ProcedureExecutor<TEnvironment> {
     return subprocs;
   }
 
-  private void submitChildrenProcedures(final Procedure[] subprocs) {
+  private void submitChildrenProcedures(Procedure<TEnvironment>[] subprocs) {
     for (int i = 0; i < subprocs.length; ++i) {
-      final Procedure subproc = subprocs[i];
+      Procedure<TEnvironment> subproc = subprocs[i];
       subproc.updateMetricsOnSubmit(getEnvironment());
       assert !procedures.containsKey(subproc.getProcId());
       procedures.put(subproc.getProcId(), subproc);
@@ -1619,8 +1660,9 @@ public class ProcedureExecutor<TEnvironment> {
     }
   }
 
-  private void countDownChildren(final RootProcedureState procStack, final Procedure procedure) {
-    final Procedure parent = procedures.get(procedure.getParentProcId());
+  private void countDownChildren(RootProcedureState<TEnvironment> procStack,
+      Procedure<TEnvironment> procedure) {
+    Procedure<TEnvironment> parent = procedures.get(procedure.getParentProcId());
     if (parent == null) {
       assert procStack.isRollingback();
       return;
@@ -1637,17 +1679,15 @@ public class ProcedureExecutor<TEnvironment> {
     }
   }
 
-  private void updateStoreOnExec(final RootProcedureState procStack,
-      final Procedure procedure, final Procedure[] subprocs) {
+  private void updateStoreOnExec(RootProcedureState<TEnvironment> procStack,
+      Procedure<TEnvironment> procedure, Procedure<TEnvironment>[] subprocs) {
     if (subprocs != null && !procedure.isFailed()) {
       if (LOG.isTraceEnabled()) {
         LOG.trace("Stored " + procedure + ", children " + Arrays.toString(subprocs));
       }
       store.insert(procedure, subprocs);
     } else {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Store update " + procedure);
-      }
+      LOG.trace("Store update {}", procedure);
       if (procedure.isFinished() && !procedure.hasParent()) {
         // remove child procedures
         final long[] childProcIds = procStack.getSubprocedureIds();
@@ -1665,11 +1705,8 @@ public class ProcedureExecutor<TEnvironment> {
     }
   }
 
-  private void handleInterruptedException(final Procedure proc, final InterruptedException e) {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Interrupt during " + proc + ". suspend and retry it later.", e);
-    }
-
+  private void handleInterruptedException(Procedure<TEnvironment> proc, InterruptedException e) {
+    LOG.trace("Interrupt during {}. suspend and retry it later.", proc, e);
     // NOTE: We don't call Thread.currentThread().interrupt()
     // because otherwise all the subsequent calls e.g. Thread.sleep() will throw
     // the InterruptedException. If the master is going down, we will be notified
@@ -1677,9 +1714,13 @@ public class ProcedureExecutor<TEnvironment> {
     // (The interrupted procedure will be retried on the next run)
   }
 
-  private void execCompletionCleanup(final Procedure proc) {
+  private void execCompletionCleanup(Procedure<TEnvironment> proc) {
     final TEnvironment env = getEnvironment();
-    if (proc.holdLock(env) && proc.hasLock(env)) {
+    if (proc.hasLock()) {
+      LOG.warn("Usually this should not happen, we will release the lock before if the procedure" +
+        " is finished, even if the holdLock is true, arrive here means we have some holes where" +
+        " we do not release the lock. And the releaseLock below may fail since the procedure may" +
+        " have already been deleted from the procedure store.");
       releaseLock(proc, true);
     }
     try {
@@ -1690,11 +1731,11 @@ public class ProcedureExecutor<TEnvironment> {
     }
   }
 
-  private void procedureFinished(final Procedure proc) {
+  private void procedureFinished(Procedure<TEnvironment> proc) {
     // call the procedure completion cleanup handler
     execCompletionCleanup(proc);
 
-    CompletedProcedureRetainer retainer = new CompletedProcedureRetainer(proc);
+    CompletedProcedureRetainer<TEnvironment> retainer = new CompletedProcedureRetainer<>(proc);
 
     // update the executor internal state maps
     if (!proc.shouldWaitClientAck(getEnvironment())) {
@@ -1710,14 +1751,14 @@ public class ProcedureExecutor<TEnvironment> {
       scheduler.completionCleanup(proc);
     } catch (Throwable e) {
       // Catch NullPointerExceptions or similar errors...
-      LOG.error("CODE-BUG: uncatched runtime exception for completion cleanup: " + proc, e);
+      LOG.error("CODE-BUG: uncatched runtime exception for completion cleanup: {}", proc, e);
     }
 
     // Notify the listeners
     sendProcedureFinishedNotification(proc.getProcId());
   }
 
-  RootProcedureState getProcStack(long rootProcId) {
+  RootProcedureState<TEnvironment> getProcStack(long rootProcId) {
     return rollbackStack.get(rootProcId);
   }
 
@@ -1726,7 +1767,7 @@ public class ProcedureExecutor<TEnvironment> {
   // ==========================================================================
   private class WorkerThread extends StoppableThread {
     private final AtomicLong executionStartTime = new AtomicLong(Long.MAX_VALUE);
-    private volatile Procedure<?> activeProcedure;
+    private volatile Procedure<TEnvironment> activeProcedure;
 
     public WorkerThread(ThreadGroup group) {
       this(group, "PEWorker-");
@@ -1747,7 +1788,7 @@ public class ProcedureExecutor<TEnvironment> {
       long lastUpdate = EnvironmentEdgeManager.currentTime();
       try {
         while (isRunning() && keepAlive(lastUpdate)) {
-          Procedure<?> proc = scheduler.poll(keepAliveTime, TimeUnit.MILLISECONDS);
+          Procedure<TEnvironment> proc = scheduler.poll(keepAliveTime, TimeUnit.MILLISECONDS);
           if (proc == null) {
             continue;
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
index c42dfc4..1215008 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
@@ -202,6 +202,9 @@ public final class ProcedureUtil {
       builder.setNonce(proc.getNonceKey().getNonce());
     }
 
+    if (proc.hasLock()) {
+      builder.setLocked(true);
+    }
     return builder.build();
   }
 
@@ -255,6 +258,10 @@ public final class ProcedureUtil {
       proc.setNonceKey(new NonceKey(proto.getNonceGroup(), proto.getNonce()));
     }
 
+    if (proto.getLocked()) {
+      proc.lockedWhenLoading();
+    }
+
     ProcedureStateSerializer serializer = null;
 
     if (proto.getStateMessageCount() > 0) {


[24/24] hbase git commit: HBASE-20749 Update to checkstyle 8.12

Posted by md...@apache.org.
HBASE-20749 Update to checkstyle 8.12


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

Branch: refs/heads/HBASE-20749
Commit: c8a7ced12248ad248f233a56c291f86bbdac4d2b
Parents: cf481d3
Author: Mike Drob <md...@apache.org>
Authored: Fri Jul 6 09:43:00 2018 -0500
Committer: Mike Drob <md...@apache.org>
Committed: Fri Jul 27 15:01:48 2018 -0500

----------------------------------------------------------------------
 .../src/main/resources/hbase/checkstyle-suppressions.xml     | 8 +++-----
 pom.xml                                                      | 2 +-
 2 files changed, 4 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c8a7ced1/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
----------------------------------------------------------------------
diff --git a/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml b/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
index 1679496..1274426 100644
--- a/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
+++ b/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
@@ -1,7 +1,7 @@
 <?xml version="1.0"?>
 <!DOCTYPE suppressions PUBLIC
-    "-//Puppy Crawl//DTD Suppressions 1.0//EN"
-    "http://www.puppycrawl.com/dtds/suppressions_1_0.dtd">
+    "-//Checkstyle//DTD SuppressionFilter Configuration 1.2//EN"
+    "https://checkstyle.org/dtds/suppressions_1_2.dtd">
 <!--
   TODO Update to use the message suppression filter once we can update
   to checkstyle 8.6+
@@ -46,8 +46,6 @@
   <suppress checks="EmptyBlockCheck" files="TBoundedThreadPoolServer.java"/>
   <suppress checks="EqualsHashCode" files="StartcodeAgnosticServerName.java"/>
   <suppress checks="MethodLength" files="Branch1CoprocessorMethods.java"/>
-<!--
-  TODO use message filter once we can upgrade to checkstyle 8.6+
   <suppress checks="IllegalImport" message="org\.apache\.htrace\.core"/>
--->
+  <suppress checks="ImportOrder" message="Extra separation in import group before"/>
 </suppressions>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8a7ced1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 73bed74..0fe0345 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1513,7 +1513,7 @@
     <asciidoctorj.pdf.version>1.5.0-alpha.15</asciidoctorj.pdf.version>
     <build.helper.maven.version>3.0.0</build.helper.maven.version>
     <buildnumber.maven.version>1.4</buildnumber.maven.version>
-    <checkstyle.version>8.2</checkstyle.version>
+    <checkstyle.version>8.11</checkstyle.version>
     <exec.maven.version>1.6.0</exec.maven.version>
     <error-prone.version>2.2.0</error-prone.version>
     <findbugs-annotations>1.3.9-1</findbugs-annotations>


[06/24] hbase git commit: HBASE-20917 MetaTableMetrics#stop references uninitialized requestsMap for non-meta region

Posted by md...@apache.org.
HBASE-20917 MetaTableMetrics#stop references uninitialized requestsMap for non-meta region


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

Branch: refs/heads/HBASE-20749
Commit: 37de961f2041f55e90ff9db5fb0aa83308c92515
Parents: 4613f3e
Author: tedyu <yu...@gmail.com>
Authored: Sun Jul 22 20:50:51 2018 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Sun Jul 22 20:50:51 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/coprocessor/MetaTableMetrics.java   | 15 ++++++++-------
 1 file changed, 8 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/37de961f/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java
index 9bf35c0..64a6288 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java
@@ -309,9 +309,8 @@ public class MetaTableMetrics implements RegionCoprocessor {
   public void start(CoprocessorEnvironment env) throws IOException {
     if (env instanceof RegionCoprocessorEnvironment
         && ((RegionCoprocessorEnvironment) env).getRegionInfo().getTable() != null
-        && ((RegionCoprocessorEnvironment) env).getRegionInfo().getTable().getName() != null
-        && new String(((RegionCoprocessorEnvironment) env).getRegionInfo().getTable().getName(),
-          StandardCharsets.UTF_8).equals(TableName.META_TABLE_NAME.toString())) {
+        && ((RegionCoprocessorEnvironment) env).getRegionInfo().getTable()
+          .equals(TableName.META_TABLE_NAME)) {
       regionCoprocessorEnv = (RegionCoprocessorEnvironment) env;
       observer = new ExampleRegionObserverMeta();
       requestsMap = new ConcurrentHashMap<>();
@@ -324,11 +323,13 @@ public class MetaTableMetrics implements RegionCoprocessor {
   }
 
   @Override
-  public void stop(CoprocessorEnvironment e) throws IOException {
+  public void stop(CoprocessorEnvironment env) throws IOException {
     // since meta region can move around, clear stale metrics when stop.
-    for (String meterName : requestsMap.keySet()) {
-      MetricRegistry registry = regionCoprocessorEnv.getMetricRegistryForRegionServer();
-      registry.remove(meterName);
+    if (requestsMap != null) {
+      for (String meterName : requestsMap.keySet()) {
+        MetricRegistry registry = regionCoprocessorEnv.getMetricRegistryForRegionServer();
+        registry.remove(meterName);
+      }
     }
   }
 


[04/24] hbase git commit: HBASE-20915 Remove the commit column on our download page

Posted by md...@apache.org.
HBASE-20915 Remove the commit column on our download page


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

Branch: refs/heads/HBASE-20749
Commit: b4759ce6e72f50ccd9d410bd5917dc5a515414f1
Parents: 682aa69
Author: zhangduo <zh...@apache.org>
Authored: Sun Jul 22 15:35:37 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Jul 22 17:08:27 2018 +0800

----------------------------------------------------------------------
 src/site/xdoc/downloads.xml | 10 ----------
 1 file changed, 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b4759ce6/src/site/xdoc/downloads.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/downloads.xml b/src/site/xdoc/downloads.xml
index 8b35ac6..3f3f293 100644
--- a/src/site/xdoc/downloads.xml
+++ b/src/site/xdoc/downloads.xml
@@ -40,7 +40,6 @@ under the License.
       <th>Compatiblity Report</th>
       <th>Changes</th>
       <th>Release Notes</th>
-      <th>Commit</th>
       <th>Download</th>
     </tr>
     <tr>
@@ -60,9 +59,6 @@ under the License.
         <a href="https://apache.org/dist/hbase/2.1.0/RELEASENOTES.md">Release Notes</a>
       </td>
       <td style="test-align: left">
-        <a href="https://git-wip-us.apache.org/repos/asf?p=hbase.git;a=commit;h=e1673bb0bbfea21d6e5dba73e013b09b8b49b89b">e1673bb0bbfea21d6e5dba73e013b09b8b49b89b</a>
-      </td>
-      <td style="test-align: left">
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.0/hbase-2.1.0-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/2.1.0/hbase-2.1.0-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.0/hbase-2.1.0-src.tar.gz.asc">asc</a>) <br />
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.0/hbase-2.1.0-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.1.0/hbase-2.1.0-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.0/hbase-2.1.0-bin.tar.gz.asc">asc</a>) <br />
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.0/hbase-2.1.0-client-bin.tar.gz">client-bin</a> (<a href="https://apache.org/dist/hbase/2.1.0/hbase-2.1.0-client-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.0/hbase-2.1.0-client-bin.tar.gz.asc">asc</a>)
@@ -85,9 +81,6 @@ under the License.
         <a href="https://apache.org/dist/hbase/2.0.1/RELEASENOTES.md">Release Notes</a>
       </td>
       <td style="test-align: left">
-        <a href="https://git-wip-us.apache.org/repos/asf?p=hbase.git;a=commit;h=987f7b6d37c2fcacc942cc66e5c5122aba8fdfbe">987f7b6d37c2fcacc942cc66e5c5122aba8fdfbe</a>
-      </td>
-      <td style="test-align: left">
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.0.1/hbase-2.0.1-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/2.0.1/hbase-2.0.1-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.1/hbase-2.0.1-src.tar.gz.asc">asc</a>) <br />
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.0.1/hbase-2.0.1-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.0.1/hbase-2.0.1-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.1/hbase-2.0.1-bin.tar.gz.asc">asc</a>)
       </td>
@@ -109,9 +102,6 @@ under the License.
         <a href="https://apache.org/dist/hbase/2.0.0/RELEASENOTES.md">Release Notes</a>
       </td>
       <td style="test-align: left">
-        <a href="https://git-wip-us.apache.org/repos/asf?p=hbase.git;a=commit;h=7483b111e4da77adbfc8062b3b22cbe7c2cb91c1">7483b111e4da77adbfc8062b3b22cbe7c2cb91c1</a>
-      </td>
-      <td style="test-align: left">
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.0.0/hbase-2.0.0-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/2.0.0/hbase-2.0.0-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.0/hbase-2.0.0-src.tar.gz.asc">asc</a>) <br />
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.0.0/hbase-2.0.0-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.0.0/hbase-2.0.0-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.0/hbase-2.0.0-bin.tar.gz.asc">asc</a>)
       </td>


[08/24] hbase git commit: HBASE-20878 Data loss if merging regions while ServerCrashProcedure executing

Posted by md...@apache.org.
HBASE-20878 Data loss if merging regions while ServerCrashProcedure executing


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

Branch: refs/heads/HBASE-20749
Commit: b631727bdfaa141d591d6194abba47a19c1dcbde
Parents: 4804483
Author: Allan Yang <al...@apache.org>
Authored: Tue Jul 24 10:00:28 2018 +0800
Committer: Allan Yang <al...@apache.org>
Committed: Tue Jul 24 10:00:28 2018 +0800

----------------------------------------------------------------------
 .../src/main/protobuf/MasterProcedure.proto     |   1 +
 .../assignment/MergeTableRegionsProcedure.java  |  35 ++++-
 .../assignment/SplitTableRegionProcedure.java   |   2 +-
 .../TestMergeTableRegionsWhileRSCrash.java      | 130 +++++++++++++++++++
 4 files changed, 166 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b631727b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index d651011..5227e64 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -269,6 +269,7 @@ enum MergeTableRegionsState {
   MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 9;
   MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 10;
   MERGE_TABLE_REGIONS_POST_OPERATION = 11;
+  MERGE_TABLE_REGIONS_CHECK_CLOSED_REGIONS = 12;
 }
 
 message MergeTableRegionsStateData {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b631727b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index 4d454d7..f45cd5d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -219,7 +219,7 @@ public class MergeTableRegionsProcedure
 
   @Override
   protected Flow executeFromState(final MasterProcedureEnv env,
-      final MergeTableRegionsState state) {
+      MergeTableRegionsState state) {
     LOG.trace("{} execute state={}", this, state);
     try {
       switch (state) {
@@ -236,6 +236,10 @@ public class MergeTableRegionsProcedure
           break;
         case MERGE_TABLE_REGIONS_CLOSE_REGIONS:
           addChildProcedure(createUnassignProcedures(env, getRegionReplication(env)));
+          setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CHECK_CLOSED_REGIONS);
+          break;
+        case MERGE_TABLE_REGIONS_CHECK_CLOSED_REGIONS:
+          checkClosedRegions(env);
           setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CREATE_MERGED_REGION);
           break;
         case MERGE_TABLE_REGIONS_CREATE_MERGED_REGION:
@@ -314,6 +318,8 @@ public class MergeTableRegionsProcedure
         case MERGE_TABLE_REGIONS_WRITE_MAX_SEQUENCE_ID_FILE:
           cleanupMergedRegion(env);
           break;
+        case MERGE_TABLE_REGIONS_CHECK_CLOSED_REGIONS:
+          break;
         case MERGE_TABLE_REGIONS_CLOSE_REGIONS:
           rollbackCloseRegionsForMerge(env);
           break;
@@ -462,6 +468,33 @@ public class MergeTableRegionsProcedure
   }
 
   /**
+   * check the closed regions
+   * @param env the master env
+   * @throws IOException IOException
+   */
+  private void checkClosedRegions(final MasterProcedureEnv env) throws IOException {
+    checkClosedRegion(env, regionsToMerge[0]);
+    checkClosedRegion(env, regionsToMerge[1]);
+  }
+
+  /**
+   * Check whether there is recovered.edits in the closed region
+   * If any, that means this region is not closed property, we need
+   * to abort region merge to prevent data loss
+   * @param env master env
+   * @param regionInfo regioninfo
+   * @throws IOException IOException
+   */
+  private void checkClosedRegion(final MasterProcedureEnv env,
+      RegionInfo regionInfo) throws IOException {
+    if (WALSplitter.hasRecoveredEdits(env.getMasterServices().getFileSystem(),
+        env.getMasterConfiguration(), regionInfo)) {
+      throw new IOException("Recovered.edits are found in Region: " + regionInfo
+          + ", abort merge to prevent data loss");
+    }
+  }
+
+  /**
    * Prepare merge and do some check
    * @param env MasterProcedureEnv
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hbase/blob/b631727b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index f0ea25b..f36b4c5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -141,7 +141,7 @@ public class SplitTableRegionProcedure
   /**
    * Check whether there is recovered.edits in the closed region
    * If any, that means this region is not closed property, we need
-   * to abort region merge to prevent data loss
+   * to abort region split to prevent data loss
    * @param env master env
    * @throws IOException IOException
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/b631727b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
new file mode 100644
index 0000000..b979bb8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
+import org.apache.hadoop.hbase.master.assignment.UnassignProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestMergeTableRegionsWhileRSCrash {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMergeTableRegionsWhileRSCrash.class);
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestMergeTableRegionsWhileRSCrash.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static TableName TABLE_NAME = TableName.valueOf("test");
+  private static Admin admin;
+  private static byte[] CF = Bytes.toBytes("cf");
+  private static byte[] SPLITKEY = Bytes.toBytes("row5");
+  private static CountDownLatch mergeCommitArrive = new CountDownLatch(1);
+  private static Table TABLE;
+
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    UTIL.startMiniCluster(1);
+    admin = UTIL.getHBaseAdmin();
+    byte[][] splitKeys = new byte[1][];
+    splitKeys[0] = SPLITKEY;
+    TABLE = UTIL.createTable(TABLE_NAME, CF, splitKeys);
+    UTIL.waitTableAvailable(TABLE_NAME);
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    //write some rows to the table
+    for (int i = 0; i < 10; i++) {
+      byte[] row = Bytes.toBytes("row" + i);
+      Put put = new Put(row);
+      put.addColumn(CF, CF, CF);
+      TABLE.put(put);
+    }
+    MasterProcedureEnv env = UTIL.getMiniHBaseCluster().getMaster()
+        .getMasterProcedureExecutor().getEnvironment();
+    final ProcedureExecutor<MasterProcedureEnv> executor = UTIL.getMiniHBaseCluster()
+        .getMaster().getMasterProcedureExecutor();
+    List<RegionInfo> regionInfos = admin.getRegions(TABLE_NAME);
+    MergeTableRegionsProcedure mergeTableRegionsProcedure = new MergeTableRegionsProcedure(
+        env, regionInfos.get(0), regionInfos.get(1));
+    executor.submitProcedure(mergeTableRegionsProcedure);
+    UTIL.waitFor(30000, () -> executor.getProcedures().stream()
+        .filter(p -> p instanceof UnassignProcedure)
+        .map(p -> (UnassignProcedure) p)
+        .anyMatch(p -> TABLE_NAME.equals(p.getTableName())));
+    UTIL.getMiniHBaseCluster().killRegionServer(
+        UTIL.getMiniHBaseCluster().getRegionServer(0).getServerName());
+    UTIL.getMiniHBaseCluster().startRegionServer();
+    UTIL.waitUntilNoRegionsInTransition();
+    Scan scan = new Scan();
+    ResultScanner results = TABLE.getScanner(scan);
+    int count = 0;
+    Result result = null;
+    while ((result = results.next()) != null) {
+      count++;
+    }
+    Assert.assertEquals("There should be 10 rows!", 10, count);
+
+
+
+
+  }
+
+
+
+}


[03/24] hbase git commit: HBASE-19893 restore_snapshot is broken in master branch when region splits

Posted by md...@apache.org.
HBASE-19893 restore_snapshot is broken in master branch when region splits

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-20749
Commit: 682aa696777415012af3cc696190fbbd6fc199c0
Parents: 2eee7a3
Author: Toshihiro Suzuki <br...@gmail.com>
Authored: Tue Feb 13 01:55:07 2018 +0900
Committer: tedyu <yu...@gmail.com>
Committed: Sat Jul 21 07:37:10 2018 -0700

----------------------------------------------------------------------
 .../hbase/favored/FavoredNodesManager.java      | 32 ++++----
 .../hbase/master/assignment/RegionStates.java   |  5 +-
 .../procedure/RestoreSnapshotProcedure.java     | 85 +++++++++++++++++---
 .../hbase/snapshot/RestoreSnapshotHelper.java   |  3 +-
 .../hadoop/hbase/HBaseTestingUtility.java       | 36 ++++++---
 .../client/TestRestoreSnapshotFromClient.java   | 29 ++++++-
 6 files changed, 151 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/682aa696/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java
index f5c1d91..67d4071 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java
@@ -269,21 +269,25 @@ public class FavoredNodesManager {
     return result;
   }
 
-  public synchronized void deleteFavoredNodesForRegions(Collection<RegionInfo> regionInfoList) {
-    for (RegionInfo hri : regionInfoList) {
-      List<ServerName> favNodes = getFavoredNodes(hri);
-      if (favNodes != null) {
-        if (primaryRSToRegionMap.containsKey(favNodes.get(PRIMARY.ordinal()))) {
-          primaryRSToRegionMap.get(favNodes.get(PRIMARY.ordinal())).remove(hri);
-        }
-        if (secondaryRSToRegionMap.containsKey(favNodes.get(SECONDARY.ordinal()))) {
-          secondaryRSToRegionMap.get(favNodes.get(SECONDARY.ordinal())).remove(hri);
-        }
-        if (teritiaryRSToRegionMap.containsKey(favNodes.get(TERTIARY.ordinal()))) {
-          teritiaryRSToRegionMap.get(favNodes.get(TERTIARY.ordinal())).remove(hri);
-        }
-        globalFavoredNodesAssignmentPlan.removeFavoredNodes(hri);
+  public synchronized void deleteFavoredNodesForRegion(RegionInfo regionInfo) {
+    List<ServerName> favNodes = getFavoredNodes(regionInfo);
+    if (favNodes != null) {
+      if (primaryRSToRegionMap.containsKey(favNodes.get(PRIMARY.ordinal()))) {
+        primaryRSToRegionMap.get(favNodes.get(PRIMARY.ordinal())).remove(regionInfo);
+      }
+      if (secondaryRSToRegionMap.containsKey(favNodes.get(SECONDARY.ordinal()))) {
+        secondaryRSToRegionMap.get(favNodes.get(SECONDARY.ordinal())).remove(regionInfo);
       }
+      if (teritiaryRSToRegionMap.containsKey(favNodes.get(TERTIARY.ordinal()))) {
+        teritiaryRSToRegionMap.get(favNodes.get(TERTIARY.ordinal())).remove(regionInfo);
+      }
+      globalFavoredNodesAssignmentPlan.removeFavoredNodes(regionInfo);
+    }
+  }
+
+  public synchronized void deleteFavoredNodesForRegions(Collection<RegionInfo> regionInfoList) {
+    for (RegionInfo regionInfo : regionInfoList) {
+      deleteFavoredNodesForRegion(regionInfo);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/682aa696/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
index e4b49af..b1b2371 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -516,6 +516,10 @@ public class RegionStates {
     }
   }
 
+  public void deleteRegions(final List<RegionInfo> regionInfos) {
+    regionInfos.forEach(this::deleteRegion);
+  }
+
   ArrayList<RegionStateNode> getTableRegionStateNodes(final TableName tableName) {
     final ArrayList<RegionStateNode> regions = new ArrayList<RegionStateNode>();
     for (RegionStateNode node: regionsMap.tailMap(tableName.getName()).values()) {
@@ -769,7 +773,6 @@ public class RegionStates {
     setServerState(serverName, ServerState.OFFLINE);
   }
 
-  @VisibleForTesting
   public void updateRegionState(final RegionInfo regionInfo, final State state) {
     final RegionStateNode regionNode = getOrCreateRegionStateNode(regionInfo);
     synchronized (regionNode) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/682aa696/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
index 09f6259..f542449 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
@@ -34,11 +34,15 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
+import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MetricsSnapshot;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
@@ -391,7 +395,7 @@ public class RestoreSnapshotProcedure
         env.getMasterServices().getConfiguration(),
         fs,
         manifest,
-              modifiedTableDescriptor,
+        modifiedTableDescriptor,
         rootDir,
         monitorException,
         getMonitorStatus());
@@ -419,12 +423,13 @@ public class RestoreSnapshotProcedure
   private void updateMETA(final MasterProcedureEnv env) throws IOException {
     try {
       Connection conn = env.getMasterServices().getConnection();
+      int regionReplication = modifiedTableDescriptor.getRegionReplication();
 
       // 1. Prepare to restore
       getMonitorStatus().setStatus("Preparing to restore each region");
 
-      // 2. Applies changes to hbase:meta
-      // (2.1). Removes the current set of regions from META
+      // 2. Applies changes to hbase:meta and in-memory states
+      // (2.1). Removes the current set of regions from META and in-memory states
       //
       // By removing also the regions to restore (the ones present both in the snapshot
       // and in the current state) we ensure that no extra fields are present in META
@@ -433,26 +438,25 @@ public class RestoreSnapshotProcedure
       // that are not correct after the restore.
       if (regionsToRemove != null) {
         MetaTableAccessor.deleteRegions(conn, regionsToRemove);
+        deleteRegionsFromInMemoryStates(regionsToRemove, env, regionReplication);
       }
 
-      // (2.2). Add the new set of regions to META
+      // (2.2). Add the new set of regions to META and in-memory states
       //
       // At this point the old regions are no longer present in META.
       // and the set of regions present in the snapshot will be written to META.
       // All the information in hbase:meta are coming from the .regioninfo of each region present
       // in the snapshot folder.
       if (regionsToAdd != null) {
-        MetaTableAccessor.addRegionsToMeta(
-          conn,
-          regionsToAdd,
-          modifiedTableDescriptor.getRegionReplication());
+        MetaTableAccessor.addRegionsToMeta(conn, regionsToAdd, regionReplication);
+        addRegionsToInMemoryStates(regionsToAdd, env, regionReplication);
       }
 
       if (regionsToRestore != null) {
-        MetaTableAccessor.overwriteRegions(
-          conn,
-          regionsToRestore,
-          modifiedTableDescriptor.getRegionReplication());
+        MetaTableAccessor.overwriteRegions(conn, regionsToRestore, regionReplication);
+
+        deleteRegionsFromInMemoryStates(regionsToRestore, env, regionReplication);
+        addRegionsToInMemoryStates(regionsToRestore, env, regionReplication);
       }
 
       RestoreSnapshotHelper.RestoreMetaChanges metaChanges =
@@ -479,6 +483,63 @@ public class RestoreSnapshotProcedure
       monitorStatus.getCompletionTimestamp() - monitorStatus.getStartTime());
   }
 
+  /**
+   * Delete regions from in-memory states
+   * @param regionInfos regions to delete
+   * @param env MasterProcedureEnv
+   * @param regionReplication the number of region replications
+   */
+  private void deleteRegionsFromInMemoryStates(List<RegionInfo> regionInfos,
+      MasterProcedureEnv env, int regionReplication) {
+    FavoredNodesManager fnm = env.getMasterServices().getFavoredNodesManager();
+
+    env.getAssignmentManager().getRegionStates().deleteRegions(regionInfos);
+    env.getMasterServices().getServerManager().removeRegions(regionInfos);
+    if (fnm != null) {
+      fnm.deleteFavoredNodesForRegions(regionInfos);
+    }
+
+    // For region replicas
+    if (regionReplication > 1) {
+      for (RegionInfo regionInfo : regionInfos) {
+        for (int i = 1; i < regionReplication; i++) {
+          RegionInfo regionInfoForReplica =
+              RegionReplicaUtil.getRegionInfoForReplica(regionInfo, i);
+          env.getAssignmentManager().getRegionStates().deleteRegion(regionInfoForReplica);
+          env.getMasterServices().getServerManager().removeRegion(regionInfoForReplica);
+          if (fnm != null) {
+            fnm.deleteFavoredNodesForRegion(regionInfoForReplica);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Add regions to in-memory states
+   * @param regionInfos regions to add
+   * @param env MasterProcedureEnv
+   * @param regionReplication the number of region replications
+   */
+  private void addRegionsToInMemoryStates(List<RegionInfo> regionInfos, MasterProcedureEnv env,
+      int regionReplication) {
+    AssignmentManager am = env.getAssignmentManager();
+    for (RegionInfo regionInfo : regionInfos) {
+      if (regionInfo.isSplit()) {
+        am.getRegionStates().updateRegionState(regionInfo, RegionState.State.SPLIT);
+      } else {
+        am.getRegionStates().updateRegionState(regionInfo, RegionState.State.CLOSED);
+
+        // For region replicas
+        for (int i = 1; i < regionReplication; i++) {
+          RegionInfo regionInfoForReplica =
+              RegionReplicaUtil.getRegionInfoForReplica(regionInfo, i);
+          am.getRegionStates().updateRegionState(regionInfoForReplica, RegionState.State.CLOSED);
+        }
+      }
+    }
+  }
+
   private void restoreSnapshotAcl(final MasterProcedureEnv env) throws IOException {
     if (restoreAcl && snapshot.hasUsersAndPermissions() && snapshot.getUsersAndPermissions() != null
         && SnapshotDescriptionUtils

http://git-wip-us.apache.org/repos/asf/hbase/blob/682aa696/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
index 179dfe5..7edf734 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
@@ -229,7 +229,8 @@ public class RestoreSnapshotHelper {
         if (regionNames.contains(regionName)) {
           LOG.info("region to restore: " + regionName);
           regionNames.remove(regionName);
-          metaChanges.addRegionToRestore(regionInfo);
+          metaChanges.addRegionToRestore(ProtobufUtil.toRegionInfo(regionManifests.get(regionName)
+              .getRegionInfo()));
         } else {
           LOG.info("region to remove: " + regionName);
           metaChanges.addRegionToRemove(regionInfo);

http://git-wip-us.apache.org/repos/asf/hbase/blob/682aa696/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 03d5d76..b938d28 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
@@ -2411,6 +2411,19 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     return rows;
   }
 
+  /**
+   * Returns all regions of the specified table
+   *
+   * @param tableName the table name
+   * @return all regions of the specified table
+   * @throws IOException when getting the regions fails.
+   */
+  private List<RegionInfo> getRegions(TableName tableName) throws IOException {
+    try (Admin admin = getConnection().getAdmin()) {
+      return admin.getRegions(tableName);
+    }
+  }
+
   /*
    * Find any other region server which is different from the one identified by parameter
    * @param rs
@@ -2429,9 +2442,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   /**
    * Tool to get the reference to the region server object that holds the
    * region of the specified user table.
-   * It first searches for the meta rows that contain the region of the
-   * specified table, then gets the index of that RS, and finally retrieves
-   * the RS's reference.
    * @param tableName user table to lookup in hbase:meta
    * @return region server that holds it, null if the row doesn't exist
    * @throws IOException
@@ -2439,21 +2449,27 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    */
   public HRegionServer getRSForFirstRegionInTable(TableName tableName)
       throws IOException, InterruptedException {
-    List<byte[]> metaRows = getMetaTableRows(tableName);
-    if (metaRows == null || metaRows.isEmpty()) {
+    List<RegionInfo> regions = getRegions(tableName);
+    if (regions == null || regions.isEmpty()) {
       return null;
     }
-    LOG.debug("Found " + metaRows.size() + " rows for table " +
-      tableName);
-    byte [] firstrow = metaRows.get(0);
-    LOG.debug("FirstRow=" + Bytes.toString(firstrow));
+    LOG.debug("Found " + regions.size() + " regions for table " +
+        tableName);
+
+    byte[] firstRegionName = regions.stream()
+        .filter(r -> !r.isOffline())
+        .map(RegionInfo::getRegionName)
+        .findFirst()
+        .orElseThrow(() -> new IOException("online regions not found in table " + tableName));
+
+    LOG.debug("firstRegionName=" + Bytes.toString(firstRegionName));
     long pause = getConfiguration().getLong(HConstants.HBASE_CLIENT_PAUSE,
       HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
     int numRetries = getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
       HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
     RetryCounter retrier = new RetryCounter(numRetries+1, (int)pause, TimeUnit.MICROSECONDS);
     while(retrier.shouldRetry()) {
-      int index = getMiniHBaseCluster().getServerWith(firstrow);
+      int index = getMiniHBaseCluster().getServerWith(firstRegionName);
       if (index != -1) {
         return getMiniHBaseCluster().getRegionServerThreads().get(index).getRegionServer();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/682aa696/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
index eb8b20e..07044ee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -307,6 +306,12 @@ public class TestRestoreSnapshotFromClient {
     // Take a snapshot
     admin.snapshot(snapshotName1, tableName);
 
+    // Load more data
+    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
+
+    // Split the second region
+    splitRegion(regionInfos.get(1));
+
     // Restore the snapshot
     admin.disableTable(tableName);
     admin.restoreSnapshot(snapshotName1);
@@ -315,6 +320,28 @@ public class TestRestoreSnapshotFromClient {
     verifyRowCount(TEST_UTIL, tableName, snapshot1Rows);
   }
 
+  @Test
+  public void testRestoreSnapshotAfterTruncate() throws Exception {
+    TableName tableName = TableName.valueOf("testRestoreSnapshotAfterTruncate");
+    SnapshotTestingUtils.createTable(TEST_UTIL, tableName, getNumReplicas(), FAMILY);
+    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
+    int numOfRows = 0;
+
+    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
+      numOfRows = countRows(table);
+    }
+    // take snapshot
+    admin.snapshot("snap", tableName);
+    admin.disableTable(tableName);
+    admin.truncateTable(tableName, false);
+    admin.disableTable(tableName);
+    admin.restoreSnapshot("snap");
+
+    admin.enableTable(tableName);
+    verifyRowCount(TEST_UTIL, tableName, numOfRows);
+    SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
+  }
+
   // ==========================================================================
   //  Helpers
   // ==========================================================================


[20/24] hbase git commit: Revert "HBASE-20949 Add logs for debugging"

Posted by md...@apache.org.
Revert "HBASE-20949 Add logs for debugging"

This reverts commit 8b8de1f8a77b5b9f6d4b8cfb7eeb3d545a69d0f2.


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

Branch: refs/heads/HBASE-20749
Commit: d43e28dc8269d19596aaf801de8e63c8bbd8b68f
Parents: 973b4dd
Author: zhangduo <zh...@apache.org>
Authored: Fri Jul 27 08:40:46 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Jul 27 08:40:46 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/procedure2/LockAndQueue.java  | 13 +++----------
 .../master/procedure/MasterProcedureScheduler.java    | 14 +++++++-------
 2 files changed, 10 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d43e28dc/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
index f86c7c8..ae8daa2 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.hbase.procedure2;
 import java.util.function.Predicate;
 import java.util.stream.Stream;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Locking for mutual exclusion between procedures. Used only by procedure framework internally.
@@ -50,7 +48,6 @@ import org.slf4j.LoggerFactory;
  */
 @InterfaceAudience.Private
 public class LockAndQueue implements LockStatus {
-  private static final Logger LOG = LoggerFactory.getLogger(LockAndQueue.class);
   private final ProcedureDeque queue = new ProcedureDeque();
   private Procedure<?> exclusiveLockOwnerProcedure = null;
   private int sharedLock = 0;
@@ -114,13 +111,11 @@ public class LockAndQueue implements LockStatus {
    */
   public boolean trySharedLock(Procedure<?> proc) {
     if (hasExclusiveLock() && !hasLockAccess(proc)) {
-      LOG.debug("{} acquire shared lock {} failed", proc, this, new Exception());
       return false;
     }
     // If no one holds the xlock, then we are free to hold the sharedLock
     // If the parent proc or we have already held the xlock, then we return true here as
     // xlock is more powerful then shared lock.
-    LOG.debug("{} acquire shared lock {} succeeded", proc, this, new Exception());
     sharedLock++;
     return true;
   }
@@ -128,8 +123,7 @@ public class LockAndQueue implements LockStatus {
   /**
    * @return whether we should wake the procedures waiting on the lock here.
    */
-  public boolean releaseSharedLock(Procedure<?> proc) {
-    LOG.debug("{} release shared lock {}", proc, this, new Exception());
+  public boolean releaseSharedLock() {
     // hasExclusiveLock could be true, it usually means we acquire shared lock while we or our
     // parent have held the xlock. And since there is still an exclusive lock, we do not need to
     // wake any procedures.
@@ -192,8 +186,7 @@ public class LockAndQueue implements LockStatus {
 
   @Override
   public String toString() {
-    return String.format("%08x", hashCode()) + ": exclusiveLockOwner=" +
-      (hasExclusiveLock() ? getExclusiveLockProcIdOwner() : "NONE") + ", sharedLockCount=" +
-      getSharedLockCount() + ", waitingProcCount=" + queue.size();
+    return "exclusiveLockOwner=" + (hasExclusiveLock() ? getExclusiveLockProcIdOwner() : "NONE") +
+      ", sharedLockCount=" + getSharedLockCount() + ", waitingProcCount=" + queue.size();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d43e28dc/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index 4180a96..2a29ee1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -526,7 +526,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         return true;
       }
       if (!tableLock.tryExclusiveLock(procedure)) {
-        namespaceLock.releaseSharedLock(procedure);
+        namespaceLock.releaseSharedLock();
         waitProcedure(tableLock, procedure);
         logLockedResource(LockedResourceType.TABLE, table.getNameAsString());
         return true;
@@ -552,7 +552,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       if (tableLock.releaseExclusiveLock(procedure)) {
         waitingCount += wakeWaitingProcedures(tableLock);
       }
-      if (namespaceLock.releaseSharedLock(procedure)) {
+      if (namespaceLock.releaseSharedLock()) {
         waitingCount += wakeWaitingProcedures(namespaceLock);
       }
       addToRunQueue(tableRunQueue, getTableQueue(table));
@@ -584,7 +584,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       }
 
       if (!tableLock.trySharedLock(procedure)) {
-        namespaceLock.releaseSharedLock(procedure);
+        namespaceLock.releaseSharedLock();
         waitProcedure(tableLock, procedure);
         return null;
       }
@@ -606,11 +606,11 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       final LockAndQueue namespaceLock = locking.getNamespaceLock(table.getNamespaceAsString());
       final LockAndQueue tableLock = locking.getTableLock(table);
       int waitingCount = 0;
-      if (tableLock.releaseSharedLock(procedure)) {
+      if (tableLock.releaseSharedLock()) {
         addToRunQueue(tableRunQueue, getTableQueue(table));
         waitingCount += wakeWaitingProcedures(tableLock);
       }
-      if (namespaceLock.releaseSharedLock(procedure)) {
+      if (namespaceLock.releaseSharedLock()) {
         waitingCount += wakeWaitingProcedures(namespaceLock);
       }
       wakePollIfNeeded(waitingCount);
@@ -784,7 +784,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
 
       final LockAndQueue namespaceLock = locking.getNamespaceLock(namespace);
       if (!namespaceLock.tryExclusiveLock(procedure)) {
-        systemNamespaceTableLock.releaseSharedLock(procedure);
+        systemNamespaceTableLock.releaseSharedLock();
         waitProcedure(namespaceLock, procedure);
         logLockedResource(LockedResourceType.NAMESPACE, namespace);
         return true;
@@ -811,7 +811,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       if (namespaceLock.releaseExclusiveLock(procedure)) {
         waitingCount += wakeWaitingProcedures(namespaceLock);
       }
-      if (systemNamespaceTableLock.releaseSharedLock(procedure)) {
+      if (systemNamespaceTableLock.releaseSharedLock()) {
         addToRunQueue(tableRunQueue, getTableQueue(TableName.NAMESPACE_TABLE_NAME));
         waitingCount += wakeWaitingProcedures(systemNamespaceTableLock);
       }


[17/24] hbase git commit: HBASE-20932 Effective MemStoreSize::hashCode()

Posted by md...@apache.org.
HBASE-20932 Effective MemStoreSize::hashCode()

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-20749
Commit: a392c017ed05c541403307e7fc75fbad38acff4a
Parents: 1913164
Author: Mingliang Liu <li...@apache.org>
Authored: Wed Jul 25 14:32:08 2018 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Jul 25 14:40:45 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/regionserver/MemStoreSize.java     | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a392c017/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java
index 97a416e..e9b4cea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java
@@ -100,9 +100,9 @@ public class MemStoreSize {
 
   @Override
   public int hashCode() {
-    long h = 31 * this.dataSize;
-    h = h + 31 * this.heapSize;
-    h = h + 31 * this.offHeapSize;
+    long h = this.dataSize;
+    h = h * 31 + this.heapSize;
+    h = h * 31 + this.offHeapSize;
     return (int) h;
   }
 


[22/24] hbase git commit: HBASE-20939 There will be race when we call suspendIfNotReady and then throw ProcedureSuspendedException

Posted by md...@apache.org.
HBASE-20939 There will be race when we call suspendIfNotReady and then throw ProcedureSuspendedException


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

Branch: refs/heads/HBASE-20749
Commit: 7178a98258dbb28496c2c4f3fbbf8e552ead8bdb
Parents: 80b40a3
Author: zhangduo <zh...@apache.org>
Authored: Fri Jul 27 17:26:40 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Jul 27 17:27:12 2018 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/IdLock.java    | 137 ++++++++++++++++++
 .../hbase/procedure2/ProcedureExecutor.java     |  20 +--
 .../procedure2/ProcedureSuspendedException.java |   6 +-
 .../org/apache/hadoop/hbase/util/IdLock.java    | 138 -------------------
 4 files changed, 152 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7178a982/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IdLock.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IdLock.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IdLock.java
new file mode 100644
index 0000000..269bf83
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IdLock.java
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.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:
+ *
+ * <pre>
+ * IdLock.Entry lockEntry = idLock.getLockEntry(id);
+ * try {
+ *   // User code.
+ * } finally {
+ *   idLock.releaseLockEntry(lockEntry);
+ * }</pre>
+ */
+@InterfaceAudience.Private
+public class IdLock {
+
+  /** An entry returned to the client as a lock object */
+  public static final class Entry {
+    private final long id;
+    private int numWaiters;
+    private boolean locked = true;
+
+    private Entry(long id) {
+      this.id = id;
+    }
+
+    @Override
+    public String toString() {
+      return "id=" + id + ", numWaiter=" + numWaiters + ", isLocked="
+          + locked;
+    }
+  }
+
+  private ConcurrentMap<Long, Entry> map = new ConcurrentHashMap<>();
+
+  /**
+   * Blocks until the lock corresponding to the given id is acquired.
+   *
+   * @param id an arbitrary number to lock on
+   * @return an "entry" to pass to {@link #releaseLockEntry(Entry)} to release
+   *         the lock
+   * @throws IOException if interrupted
+   */
+  public Entry getLockEntry(long id) throws IOException {
+    Entry entry = new Entry(id);
+    Entry existing;
+    while ((existing = map.putIfAbsent(entry.id, entry)) != null) {
+      synchronized (existing) {
+        if (existing.locked) {
+          ++existing.numWaiters;  // Add ourselves to waiters.
+          while (existing.locked) {
+            try {
+              existing.wait();
+            } catch (InterruptedException e) {
+              --existing.numWaiters;  // Remove ourselves from waiters.
+              throw new InterruptedIOException(
+                  "Interrupted waiting to acquire sparse lock");
+            }
+          }
+
+          --existing.numWaiters;  // Remove ourselves from waiters.
+          existing.locked = true;
+          return existing;
+        }
+        // If the entry is not locked, it might already be deleted from the
+        // map, so we cannot return it. We need to get our entry into the map
+        // or get someone else's locked entry.
+      }
+    }
+    return entry;
+  }
+
+  /**
+   * Must be called in a finally block to decrease the internal counter and
+   * remove the monitor object for the given id if the caller is the last
+   * client.
+   *
+   * @param entry the return value of {@link #getLockEntry(long)}
+   */
+  public void releaseLockEntry(Entry entry) {
+    synchronized (entry) {
+      entry.locked = false;
+      if (entry.numWaiters > 0) {
+        entry.notify();
+      } else {
+        map.remove(entry.id);
+      }
+    }
+  }
+
+  /** For testing */
+  void assertMapEmpty() {
+    assert map.isEmpty();
+  }
+
+  @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/7178a982/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index f1bec72..e2215c6 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator
 import org.apache.hadoop.hbase.procedure2.util.StringUtils;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.IdLock;
 import org.apache.hadoop.hbase.util.NonceKey;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -313,6 +314,14 @@ public class ProcedureExecutor<TEnvironment> {
 
   private final boolean checkOwnerSet;
 
+  // To prevent concurrent execution of the same procedure.
+  // For some rare cases, especially if the procedure uses ProcedureEvent, it is possible that the
+  // procedure is woken up before we finish the suspend which causes the same procedures to be
+  // executed in parallel. This does lead to some problems, see HBASE-20939&HBASE-20949, and is also
+  // a bit confusing to the developers. So here we introduce this lock to prevent the concurrent
+  // execution of the same procedure.
+  private final IdLock procExecutionLock = new IdLock();
+
   public ProcedureExecutor(final Configuration conf, final TEnvironment environment,
       final ProcedureStore store) {
     this(conf, environment, store, new SimpleProcedureScheduler());
@@ -1496,14 +1505,7 @@ public class ProcedureExecutor<TEnvironment> {
     // Procedures can suspend themselves. They skip out by throwing a ProcedureSuspendedException.
     // The exception is caught below and then we hurry to the exit without disturbing state. The
     // idea is that the processing of this procedure will be unsuspended later by an external event
-    // such the report of a region open. TODO: Currently, its possible for two worker threads
-    // to be working on the same procedure concurrently (locking in procedures is NOT about
-    // concurrency but about tying an entity to a procedure; i.e. a region to a particular
-    // procedure instance). This can make for issues if both threads are changing state.
-    // See env.getProcedureScheduler().wakeEvent(regionNode.getProcedureEvent());
-    // in RegionTransitionProcedure#reportTransition for example of Procedure putting
-    // itself back on the scheduler making it possible for two threads running against
-    // the one Procedure. Might be ok if they are both doing different, idempotent sections.
+    // such the report of a region open.
     boolean suspended = false;
 
     // Whether to 're-' -execute; run through the loop again.
@@ -1798,12 +1800,14 @@ public class ProcedureExecutor<TEnvironment> {
           LOG.trace("Execute pid={} runningCount={}, activeCount={}", proc.getProcId(),
             runningCount, activeCount);
           executionStartTime.set(EnvironmentEdgeManager.currentTime());
+          IdLock.Entry lockEntry = procExecutionLock.getLockEntry(proc.getProcId());
           try {
             executeProcedure(proc);
           } catch (AssertionError e) {
             LOG.info("ASSERT pid=" + proc.getProcId(), e);
             throw e;
           } finally {
+            procExecutionLock.releaseLockEntry(lockEntry);
             activeCount = activeExecutorCount.decrementAndGet();
             runningCount = store.setRunningProcedureCount(activeCount);
             LOG.trace("Halt pid={} runningCount={}, activeCount={}", proc.getProcId(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/7178a982/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureSuspendedException.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureSuspendedException.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureSuspendedException.java
index 5090fb1..9f52121 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureSuspendedException.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureSuspendedException.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.procedure2;
 
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 
 @InterfaceAudience.Private
-@InterfaceStability.Stable
 public class ProcedureSuspendedException extends ProcedureException {
+
+  private static final long serialVersionUID = -8328419627678496269L;
+
   /** default constructor */
   public ProcedureSuspendedException() {
     super();

http://git-wip-us.apache.org/repos/asf/hbase/blob/7178a982/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
deleted file mode 100644
index eba9acd..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdLock.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- *
- * 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.util;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.yetus.audience.InterfaceAudience;
-
-import org.apache.hbase.thirdparty.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:
- *
- * <pre>
- * IdLock.Entry lockEntry = idLock.getLockEntry(id);
- * try {
- *   // User code.
- * } finally {
- *   idLock.releaseLockEntry(lockEntry);
- * }</pre>
- */
-@InterfaceAudience.Private
-public class IdLock {
-
-  /** An entry returned to the client as a lock object */
-  public static class Entry {
-    private final long id;
-    private int numWaiters;
-    private boolean locked = true;
-
-    private Entry(long id) {
-      this.id = id;
-    }
-
-    @Override
-    public String toString() {
-      return "id=" + id + ", numWaiter=" + numWaiters + ", isLocked="
-          + locked;
-    }
-  }
-
-  private ConcurrentMap<Long, Entry> map = new ConcurrentHashMap<>();
-
-  /**
-   * Blocks until the lock corresponding to the given id is acquired.
-   *
-   * @param id an arbitrary number to lock on
-   * @return an "entry" to pass to {@link #releaseLockEntry(Entry)} to release
-   *         the lock
-   * @throws IOException if interrupted
-   */
-  public Entry getLockEntry(long id) throws IOException {
-    Entry entry = new Entry(id);
-    Entry existing;
-    while ((existing = map.putIfAbsent(entry.id, entry)) != null) {
-      synchronized (existing) {
-        if (existing.locked) {
-          ++existing.numWaiters;  // Add ourselves to waiters.
-          while (existing.locked) {
-            try {
-              existing.wait();
-            } catch (InterruptedException e) {
-              --existing.numWaiters;  // Remove ourselves from waiters.
-              throw new InterruptedIOException(
-                  "Interrupted waiting to acquire sparse lock");
-            }
-          }
-
-          --existing.numWaiters;  // Remove ourselves from waiters.
-          existing.locked = true;
-          return existing;
-        }
-        // If the entry is not locked, it might already be deleted from the
-        // map, so we cannot return it. We need to get our entry into the map
-        // or get someone else's locked entry.
-      }
-    }
-    return entry;
-  }
-
-  /**
-   * Must be called in a finally block to decrease the internal counter and
-   * remove the monitor object for the given id if the caller is the last
-   * client.
-   *
-   * @param entry the return value of {@link #getLockEntry(long)}
-   */
-  public void releaseLockEntry(Entry entry) {
-    synchronized (entry) {
-      entry.locked = false;
-      if (entry.numWaiters > 0) {
-        entry.notify();
-      } else {
-        map.remove(entry.id);
-      }
-    }
-  }
-
-  /** For testing */
-  void assertMapEmpty() {
-    assert map.isEmpty();
-  }
-
-  @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);
-    }
-  }
-}


[02/24] hbase git commit: HBASE-20910 Fix dev-support/submit-patch.py by opening file with 'b' mode

Posted by md...@apache.org.
HBASE-20910 Fix dev-support/submit-patch.py by opening file with 'b' mode

Signed-off-by: tedyu <yu...@gmail.com>
Signed-off-by: Mike Drob <md...@apache.org>


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

Branch: refs/heads/HBASE-20749
Commit: 2eee7a3f8bc9749c10e9d311a55696e66fd0b09d
Parents: eb906e2
Author: Ankit Singhal <an...@gmail.com>
Authored: Fri Jul 20 15:23:38 2018 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Fri Jul 20 15:31:11 2018 -0700

----------------------------------------------------------------------
 dev-support/submit-patch.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2eee7a3f/dev-support/submit-patch.py
----------------------------------------------------------------------
diff --git a/dev-support/submit-patch.py b/dev-support/submit-patch.py
index ad39495..f5d772f 100755
--- a/dev-support/submit-patch.py
+++ b/dev-support/submit-patch.py
@@ -249,7 +249,7 @@ logger.info(" Patch name: %s", patch_filename)
 patch_filepath = os.path.join(patch_dir, patch_filename)
 
 diff = git.format_patch(base_branch, stdout = True)
-with open(patch_filepath, "w") as f:
+with open(patch_filepath, "wb") as f:
     f.write(diff.encode('utf8'))
 
 if args.jira_id is not None:


[12/24] hbase git commit: HBASE-20846 Restore procedure locks when master restarts

Posted by md...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java
index 46185ea..2fc0030 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java
@@ -22,11 +22,9 @@ import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 
 /**
@@ -42,8 +40,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedu
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-class RootProcedureState {
-  private static final Logger LOG = LoggerFactory.getLogger(RootProcedureState.class);
+class RootProcedureState<TEnvironment> {
 
   private enum State {
     RUNNING,         // The Procedure is running or ready to run
@@ -51,8 +48,8 @@ class RootProcedureState {
     ROLLINGBACK,     // The Procedure failed and the execution was rolledback
   }
 
-  private Set<Procedure> subprocs = null;
-  private ArrayList<Procedure> subprocStack = null;
+  private Set<Procedure<TEnvironment>> subprocs = null;
+  private ArrayList<Procedure<TEnvironment>> subprocStack = null;
   private State state = State.RUNNING;
   private int running = 0;
 
@@ -91,22 +88,19 @@ class RootProcedureState {
   }
 
   protected synchronized long[] getSubprocedureIds() {
-    if (subprocs == null) return null;
-    int index = 0;
-    final long[] subIds = new long[subprocs.size()];
-    for (Procedure proc: subprocs) {
-      subIds[index++] = proc.getProcId();
+    if (subprocs == null) {
+      return null;
     }
-    return subIds;
+    return subprocs.stream().mapToLong(Procedure::getProcId).toArray();
   }
 
-  protected synchronized List<Procedure> getSubproceduresStack() {
+  protected synchronized List<Procedure<TEnvironment>> getSubproceduresStack() {
     return subprocStack;
   }
 
   protected synchronized RemoteProcedureException getException() {
     if (subprocStack != null) {
-      for (Procedure proc: subprocStack) {
+      for (Procedure<TEnvironment> proc: subprocStack) {
         if (proc.hasException()) {
           return proc.getException();
         }
@@ -118,8 +112,10 @@ class RootProcedureState {
   /**
    * Called by the ProcedureExecutor to mark the procedure step as running.
    */
-  protected synchronized boolean acquire(final Procedure proc) {
-    if (state != State.RUNNING) return false;
+  protected synchronized boolean acquire(Procedure<TEnvironment> proc) {
+    if (state != State.RUNNING) {
+      return false;
+    }
 
     running++;
     return true;
@@ -128,7 +124,7 @@ class RootProcedureState {
   /**
    * Called by the ProcedureExecutor to mark the procedure step as finished.
    */
-  protected synchronized void release(final Procedure proc) {
+  protected synchronized void release(Procedure<TEnvironment> proc) {
     running--;
   }
 
@@ -142,7 +138,7 @@ class RootProcedureState {
    * Called by the ProcedureExecutor after the procedure step is completed,
    * to add the step to the rollback list (or procedure stack)
    */
-  protected synchronized void addRollbackStep(final Procedure proc) {
+  protected synchronized void addRollbackStep(Procedure<TEnvironment> proc) {
     if (proc.isFailed()) {
       state = State.FAILED;
     }
@@ -153,8 +149,10 @@ class RootProcedureState {
     subprocStack.add(proc);
   }
 
-  protected synchronized void addSubProcedure(final Procedure proc) {
-    if (!proc.hasParent()) return;
+  protected synchronized void addSubProcedure(Procedure<TEnvironment> proc) {
+    if (!proc.hasParent()) {
+      return;
+    }
     if (subprocs == null) {
       subprocs = new HashSet<>();
     }
@@ -168,7 +166,7 @@ class RootProcedureState {
    * to the store only the Procedure we executed, and nothing else.
    * on load we recreate the full stack by aggregating each procedure stack-positions.
    */
-  protected synchronized void loadStack(final Procedure proc) {
+  protected synchronized void loadStack(Procedure<TEnvironment> proc) {
     addSubProcedure(proc);
     int[] stackIndexes = proc.getStackIndexes();
     if (stackIndexes != null) {
@@ -196,7 +194,7 @@ class RootProcedureState {
    */
   protected synchronized boolean isValid() {
     if (subprocStack != null) {
-      for (Procedure proc: subprocStack) {
+      for (Procedure<TEnvironment> proc : subprocStack) {
         if (proc == null) {
           return false;
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java
index e5e3230..9e050a2 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java
@@ -31,15 +31,15 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedu
  * @see InlineChore
  */
 @InterfaceAudience.Private
-class TimeoutExecutorThread extends StoppableThread {
+class TimeoutExecutorThread<TEnvironment> extends StoppableThread {
 
   private static final Logger LOG = LoggerFactory.getLogger(TimeoutExecutorThread.class);
 
-  private final ProcedureExecutor<?> executor;
+  private final ProcedureExecutor<TEnvironment> executor;
 
   private final DelayQueue<DelayedWithTimeout> queue = new DelayQueue<>();
 
-  public TimeoutExecutorThread(ProcedureExecutor<?> executor, ThreadGroup group) {
+  public TimeoutExecutorThread(ProcedureExecutor<TEnvironment> executor, ThreadGroup group) {
     super(group, "ProcExecTimeout");
     setDaemon(true);
     this.executor = executor;
@@ -65,7 +65,7 @@ class TimeoutExecutorThread extends StoppableThread {
       if (task instanceof InlineChore) {
         execInlineChore((InlineChore) task);
       } else if (task instanceof DelayedProcedure) {
-        execDelayedProcedure((DelayedProcedure) task);
+        execDelayedProcedure((DelayedProcedure<TEnvironment>) task);
       } else {
         LOG.error("CODE-BUG unknown timeout task type {}", task);
       }
@@ -77,15 +77,15 @@ class TimeoutExecutorThread extends StoppableThread {
     queue.add(chore);
   }
 
-  public void add(Procedure<?> procedure) {
+  public void add(Procedure<TEnvironment> procedure) {
     assert procedure.getState() == ProcedureState.WAITING_TIMEOUT;
     LOG.info("ADDED {}; timeout={}, timestamp={}", procedure, procedure.getTimeout(),
       procedure.getTimeoutTimestamp());
-    queue.add(new DelayedProcedure(procedure));
+    queue.add(new DelayedProcedure<>(procedure));
   }
 
-  public boolean remove(Procedure<?> procedure) {
-    return queue.remove(new DelayedProcedure(procedure));
+  public boolean remove(Procedure<TEnvironment> procedure) {
+    return queue.remove(new DelayedProcedure<>(procedure));
   }
 
   private void execInlineChore(InlineChore chore) {
@@ -93,13 +93,13 @@ class TimeoutExecutorThread extends StoppableThread {
     add(chore);
   }
 
-  private void execDelayedProcedure(DelayedProcedure delayed) {
+  private void execDelayedProcedure(DelayedProcedure<TEnvironment> delayed) {
     // TODO: treat this as a normal procedure, add it to the scheduler and
     // let one of the workers handle it.
     // Today we consider ProcedureInMemoryChore as InlineChores
-    Procedure<?> procedure = delayed.getObject();
+    Procedure<TEnvironment> procedure = delayed.getObject();
     if (procedure instanceof ProcedureInMemoryChore) {
-      executeInMemoryChore((ProcedureInMemoryChore) procedure);
+      executeInMemoryChore((ProcedureInMemoryChore<TEnvironment>) procedure);
       // if the procedure is in a waiting state again, put it back in the queue
       procedure.updateTimestamp();
       if (procedure.isWaiting()) {
@@ -111,7 +111,7 @@ class TimeoutExecutorThread extends StoppableThread {
     }
   }
 
-  private void executeInMemoryChore(ProcedureInMemoryChore chore) {
+  private void executeInMemoryChore(ProcedureInMemoryChore<TEnvironment> chore) {
     if (!chore.isWaiting()) {
       return;
     }
@@ -126,12 +126,12 @@ class TimeoutExecutorThread extends StoppableThread {
     }
   }
 
-  private void executeTimedoutProcedure(Procedure proc) {
+  private void executeTimedoutProcedure(Procedure<TEnvironment> proc) {
     // The procedure received a timeout. if the procedure itself does not handle it,
     // call abort() and add the procedure back in the queue for rollback.
     if (proc.setTimeoutFailure(executor.getEnvironment())) {
       long rootProcId = executor.getRootProcedureId(proc);
-      RootProcedureState procStack = executor.getProcStack(rootProcId);
+      RootProcedureState<TEnvironment> procStack = executor.getProcStack(rootProcId);
       procStack.abort();
       executor.getStore().update(proc);
       executor.getScheduler().addFront(proc);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java
index 319ddb2..2bbd53d 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
@@ -42,7 +43,12 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.protobuf.Int64Value;
 
-@Category({MasterTests.class, LargeTests.class})
+/**
+ * For now we do not guarantee this, we will restore the locks when restarting ProcedureExecutor so
+ * we should use lock to obtain the correct order. Ignored.
+ */
+@Ignore
+@Category({ MasterTests.class, LargeTests.class })
 public class TestProcedureReplayOrder {
 
   @ClassRule

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java
index a9e919c..c1c9187 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java
@@ -227,7 +227,6 @@ public class TestProcedureSuspended {
     protected void releaseLock(final TestProcEnv env) {
       LOG.info("RELEASE LOCK " + this + " " + hasLock);
       lock.set(false);
-      hasLock = false;
     }
 
     @Override
@@ -235,11 +234,6 @@ public class TestProcedureSuspended {
       return true;
     }
 
-    @Override
-    protected boolean hasLock(final TestProcEnv env) {
-      return hasLock;
-    }
-
     public ArrayList<Long> getTimestamps() {
       return timestamps;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Procedure.proto b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
index 2c5f1aa..b4a3107 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
@@ -63,6 +63,9 @@ message Procedure {
   // Nonce to prevent same procedure submit by multiple times
   optional uint64 nonce_group = 13 [default = 0];
   optional uint64 nonce = 14 [default = 0];
+
+  // whether the procedure has held the lock
+  optional bool locked = 16 [default = false];
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
index 7ad5b56..5af7614 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
@@ -81,8 +81,8 @@ class ClusterSchemaServiceImpl extends AbstractService implements ClusterSchemaS
     return this.tableNamespaceManager;
   }
 
-  private long submitProcedure(final Procedure<?> procedure, final NonceKey nonceKey)
-      throws ServiceNotRunningException {
+  private long submitProcedure(final Procedure<MasterProcedureEnv> procedure,
+      final NonceKey nonceKey) throws ServiceNotRunningException {
     checkIsRunning();
     ProcedureExecutor<MasterProcedureEnv> pe = this.masterServices.getMasterProcedureExecutor();
     return pe.submitProcedure(procedure, nonceKey);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/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 f1bec35..b7148d5 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
@@ -923,7 +923,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     InitMetaProcedure initMetaProc = null;
     if (assignmentManager.getRegionStates().getRegionState(RegionInfoBuilder.FIRST_META_REGIONINFO)
       .isOffline()) {
-      Optional<Procedure<?>> optProc = procedureExecutor.getProcedures().stream()
+      Optional<Procedure<MasterProcedureEnv>> optProc = procedureExecutor.getProcedures().stream()
         .filter(p -> p instanceof InitMetaProcedure).findAny();
       if (optProc.isPresent()) {
         initMetaProc = (InitMetaProcedure) optProc.get();
@@ -3202,7 +3202,8 @@ public class HMaster extends HRegionServer implements MasterServices {
       cpHost.preGetProcedures();
     }
 
-    final List<Procedure<?>> procList = this.procedureExecutor.getProcedures();
+    @SuppressWarnings({ "unchecked", "rawtypes" })
+    List<Procedure<?>> procList = (List) this.procedureExecutor.getProcedures();
 
     if (cpHost != null) {
       cpHost.postGetProcedures(procList);
@@ -3717,7 +3718,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     HashMap<String, List<Pair<ServerName, ReplicationLoadSource>>> replicationLoadSourceMap =
         new HashMap<>(peerList.size());
     peerList.stream()
-        .forEach(peer -> replicationLoadSourceMap.put(peer.getPeerId(), new ArrayList()));
+        .forEach(peer -> replicationLoadSourceMap.put(peer.getPeerId(), new ArrayList<>()));
     for (ServerName serverName : serverNames) {
       List<ReplicationLoadSource> replicationLoadSources =
           getServerManager().getLoad(serverName).getReplicationLoadSourceList();

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
index bbb27e1..0b6e45b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
@@ -148,9 +148,4 @@ public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCReg
         serializer.deserialize(MasterProcedureProtos.GCRegionStateData.class);
     setRegion(ProtobufUtil.toRegionInfo(msg.getRegionInfo()));
   }
-
-  @Override
-  protected org.apache.hadoop.hbase.procedure2.Procedure.LockState acquireLock(MasterProcedureEnv env) {
-    return super.acquireLock(env);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index f45cd5d..580b9a9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -82,7 +82,6 @@ public class MergeTableRegionsProcedure
     extends AbstractStateMachineTableProcedure<MergeTableRegionsState> {
   private static final Logger LOG = LoggerFactory.getLogger(MergeTableRegionsProcedure.class);
   private Boolean traceEnabled;
-  private volatile boolean lock = false;
   private ServerName regionLocation;
   private RegionInfo[] regionsToMerge;
   private RegionInfo mergedRegion;
@@ -420,24 +419,20 @@ public class MergeTableRegionsProcedure
 
   @Override
   protected LockState acquireLock(final MasterProcedureEnv env) {
-    if (env.waitInitialized(this)) return LockState.LOCK_EVENT_WAIT;
     if (env.getProcedureScheduler().waitRegions(this, getTableName(),
         mergedRegion, regionsToMerge[0], regionsToMerge[1])) {
       try {
         LOG.debug(LockState.LOCK_EVENT_WAIT + " " + env.getProcedureScheduler().dumpLocks());
       } catch (IOException e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
+        // Ignore, just for logging
       }
       return LockState.LOCK_EVENT_WAIT;
     }
-    this.lock = true;
     return LockState.LOCK_ACQUIRED;
   }
 
   @Override
   protected void releaseLock(final MasterProcedureEnv env) {
-    this.lock = false;
     env.getProcedureScheduler().wakeRegions(this, getTableName(),
       mergedRegion, regionsToMerge[0], regionsToMerge[1]);
   }
@@ -448,11 +443,6 @@ public class MergeTableRegionsProcedure
   }
 
   @Override
-  protected boolean hasLock(MasterProcedureEnv env) {
-    return this.lock;
-  }
-
-  @Override
   public TableName getTableName() {
     return mergedRegion.getTable();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index c3b2458..4054778 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -34,14 +34,16 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+
 /**
  * Base class for the Assign and Unassign Procedure.
  *
@@ -114,8 +116,6 @@ public abstract class RegionTransitionProcedure
    */
   private int attempt;
 
-  private volatile boolean lock = false;
-
   // Required by the Procedure framework to create the procedure on replay
   public RegionTransitionProcedure() {}
 
@@ -419,15 +419,17 @@ public abstract class RegionTransitionProcedure
   }
 
   @Override
-  protected LockState acquireLock(final MasterProcedureEnv env) {
+  protected boolean waitInitialized(MasterProcedureEnv env) {
     // Unless we are assigning meta, wait for meta to be available and loaded.
-    if (!isMeta()) {
-      AssignmentManager am = env.getAssignmentManager();
-      if (am.waitMetaLoaded(this) || am.waitMetaAssigned(this, regionInfo)) {
-        return LockState.LOCK_EVENT_WAIT;
-      }
+    if (isMeta()) {
+      return false;
     }
+    AssignmentManager am = env.getAssignmentManager();
+    return am.waitMetaLoaded(this) || am.waitMetaAssigned(this, regionInfo);
+  }
 
+  @Override
+  protected LockState acquireLock(final MasterProcedureEnv env) {
     // TODO: Revisit this and move it to the executor
     if (env.getProcedureScheduler().waitRegion(this, getRegionInfo())) {
       try {
@@ -438,14 +440,12 @@ public abstract class RegionTransitionProcedure
       }
       return LockState.LOCK_EVENT_WAIT;
     }
-    this.lock = true;
     return LockState.LOCK_ACQUIRED;
   }
 
   @Override
   protected void releaseLock(final MasterProcedureEnv env) {
     env.getProcedureScheduler().wakeRegion(this, getRegionInfo());
-    lock = false;
   }
 
   @Override
@@ -454,11 +454,6 @@ public abstract class RegionTransitionProcedure
   }
 
   @Override
-  protected boolean hasLock(final MasterProcedureEnv env) {
-    return lock;
-  }
-
-  @Override
   protected boolean shouldWaitClientAck(MasterProcedureEnv env) {
     // The operation is triggered internally on the server
     // the client does not know about this procedure.

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
index b4c55f4..3a87bbc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
@@ -76,8 +76,6 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
   private String description;
   // True when recovery of master lock from WALs
   private boolean recoveredMasterLock;
-  // this is for internal working
-  private boolean hasLock;
 
   private final ProcedureEvent<LockProcedure> event = new ProcedureEvent<>(this);
   // True if this proc acquired relevant locks. This value is for client checks.
@@ -306,7 +304,6 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
   protected LockState acquireLock(final MasterProcedureEnv env) {
     boolean ret = lock.acquireLock(env);
     locked.set(ret);
-    hasLock = ret;
     if (ret) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("LOCKED " + toString());
@@ -321,7 +318,6 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
   @Override
   protected void releaseLock(final MasterProcedureEnv env) {
     lock.releaseLock(env);
-    hasLock = false;
   }
 
   /**
@@ -423,11 +419,6 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
     return true;
   }
 
-  @Override
-  public boolean hasLock(final MasterProcedureEnv env) {
-    return hasLock;
-  }
-
   ///////////////////////
   // LOCK IMPLEMENTATIONS
   ///////////////////////

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
index 574706a..341d116 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
@@ -66,8 +66,12 @@ public abstract class AbstractStateMachineNamespaceProcedure<TState>
   }
 
   @Override
+  protected boolean waitInitialized(MasterProcedureEnv env) {
+    return env.waitInitialized(this);
+  }
+
+  @Override
   protected LockState acquireLock(final MasterProcedureEnv env) {
-    if (env.waitInitialized(this)) return LockState.LOCK_EVENT_WAIT;
     if (env.getProcedureScheduler().waitNamespaceExclusiveLock(this, getNamespaceName())) {
       return LockState.LOCK_EVENT_WAIT;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
index e711ca0..3b5e3b5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 public abstract class AbstractStateMachineRegionProcedure<TState>
     extends AbstractStateMachineTableProcedure<TState> {
   private RegionInfo hri;
-  private volatile boolean lock = false;
 
   public AbstractStateMachineRegionProcedure(final MasterProcedureEnv env,
       final RegionInfo hri) {
@@ -100,25 +99,17 @@ public abstract class AbstractStateMachineRegionProcedure<TState>
 
   @Override
   protected LockState acquireLock(final MasterProcedureEnv env) {
-    if (env.waitInitialized(this)) return LockState.LOCK_EVENT_WAIT;
     if (env.getProcedureScheduler().waitRegions(this, getTableName(), getRegion())) {
       return LockState.LOCK_EVENT_WAIT;
     }
-    this.lock = true;
     return LockState.LOCK_ACQUIRED;
   }
 
   @Override
   protected void releaseLock(final MasterProcedureEnv env) {
-    this.lock = false;
     env.getProcedureScheduler().wakeRegions(this, getTableName(), getRegion());
   }
 
-  @Override
-  protected boolean hasLock(final MasterProcedureEnv env) {
-    return this.lock;
-  }
-
   protected void setFailure(Throwable cause) {
     super.setFailure(getClass().getSimpleName(), cause);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
index 1af2445..50a0149 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
@@ -89,10 +89,12 @@ public abstract class AbstractStateMachineTableProcedure<TState>
   }
 
   @Override
+  protected boolean waitInitialized(MasterProcedureEnv env) {
+    return env.waitInitialized(this);
+  }
+
+  @Override
   protected LockState acquireLock(final MasterProcedureEnv env) {
-    if (env.waitInitialized(this)) {
-      return LockState.LOCK_EVENT_WAIT;
-    }
     if (env.getProcedureScheduler().waitTableExclusiveLock(this, getTableName())) {
       return LockState.LOCK_EVENT_WAIT;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
index c63f420..2f56e83 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
@@ -131,7 +131,7 @@ public class CreateNamespaceProcedure
 
   @Override
   protected CreateNamespaceState getState(final int stateId) {
-    return CreateNamespaceState.valueOf(stateId);
+    return CreateNamespaceState.forNumber(stateId);
   }
 
   @Override
@@ -171,15 +171,18 @@ public class CreateNamespaceProcedure
   }
 
   @Override
-  protected LockState acquireLock(final MasterProcedureEnv env) {
-    if (!env.getMasterServices().isInitialized()) {
-      // Namespace manager might not be ready if master is not fully initialized,
-      // return false to reject user namespace creation; return true for default
-      // and system namespace creation (this is part of master initialization).
-      if (!isBootstrapNamespace() && env.waitInitialized(this)) {
-        return LockState.LOCK_EVENT_WAIT;
-      }
+  protected boolean waitInitialized(MasterProcedureEnv env) {
+    // Namespace manager might not be ready if master is not fully initialized,
+    // return false to reject user namespace creation; return true for default
+    // and system namespace creation (this is part of master initialization).
+    if (isBootstrapNamespace()) {
+      return false;
     }
+    return env.waitInitialized(this);
+  }
+
+  @Override
+  protected LockState acquireLock(final MasterProcedureEnv env) {
     if (env.getProcedureScheduler().waitNamespaceExclusiveLock(this, getNamespaceName())) {
       return LockState.LOCK_EVENT_WAIT;
     }
@@ -263,20 +266,6 @@ public class CreateNamespaceProcedure
     }
   }
 
-  /**
-   * remove quota for the namespace if exists
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   **/
-  private void rollbackSetNamespaceQuota(final MasterProcedureEnv env) throws IOException {
-    try {
-      DeleteNamespaceProcedure.removeNamespaceQuota(env, nsDescriptor.getName());
-    } catch (Exception e) {
-      // Ignore exception
-      LOG.debug("Rollback of setNamespaceQuota throws exception: " + e);
-    }
-  }
-
   private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
     return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
index acee1af..faad3dd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
@@ -220,10 +220,16 @@ public class CreateTableProcedure
   }
 
   @Override
-  protected LockState acquireLock(final MasterProcedureEnv env) {
-    if (!getTableName().isSystemTable() && env.waitInitialized(this)) {
-      return LockState.LOCK_EVENT_WAIT;
+  protected boolean waitInitialized(MasterProcedureEnv env) {
+    if (getTableName().isSystemTable()) {
+      // Creating system table is part of the initialization, so do not wait here.
+      return false;
     }
+    return super.waitInitialized(env);
+  }
+
+  @Override
+  protected LockState acquireLock(final MasterProcedureEnv env) {
     if (env.getProcedureScheduler().waitTableExclusiveLock(this, getTableName())) {
       return LockState.LOCK_EVENT_WAIT;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
index 4736d65..d984632 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
@@ -63,8 +63,13 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
   }
 
   @Override
-  protected LockState acquireLock(MasterProcedureEnv env) {
+  protected boolean waitInitialized(MasterProcedureEnv env) {
     // we do not need to wait for master initialized, we are part of the initialization.
+    return false;
+  }
+
+  @Override
+  protected LockState acquireLock(MasterProcedureEnv env) {
     if (env.getProcedureScheduler().waitTableExclusiveLock(this, getTableName())) {
       return LockState.LOCK_EVENT_WAIT;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index ecf72e0..2a29ee1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -143,21 +143,13 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
 
   private <T extends Comparable<T>> void doAdd(final FairQueue<T> fairq,
       final Queue<T> queue, final Procedure<?> proc, final boolean addFront) {
-    if (!queue.getLockStatus().hasExclusiveLock() ||
-      queue.getLockStatus().isLockOwner(proc.getProcId())) {
-      // if the queue was not remove for an xlock execution
-      // or the proc is the lock owner, put the queue back into execution
+    if (!queue.getLockStatus().hasExclusiveLock()) {
+      // if the queue was not remove for an xlock execution,put the queue back into execution
       queue.add(proc, addFront);
       addToRunQueue(fairq, queue);
-    } else if (queue.getLockStatus().hasParentLock(proc)) {
-      // always add it to front as its parent has the xlock
-      // usually the addFront is true if we arrive here as we will call addFront for adding sub
-      // proc, but sometimes we may retry on the proc which means we will arrive here through yield,
-      // so it is possible the addFront here is false.
+    } else if (queue.getLockStatus().hasLockAccess(proc)) {
+      // always add it to front as the have the lock access.
       queue.add(proc, true);
-      // our (proc) parent has the xlock,
-      // so the queue is not in the fairq (run-queue)
-      // add it back to let the child run (inherit the lock)
       addToRunQueue(fairq, queue);
     } else {
       queue.add(proc, addFront);
@@ -386,9 +378,6 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     if (proc != null) {
       priority = MasterProcedureUtil.getServerPriority(proc);
     } else {
-      LOG.warn("Usually this should not happen as proc can only be null when calling from " +
-        "wait/wake lock, which means at least we should have one procedure in the queue which " +
-        "wants to acquire the lock or just released the lock.");
       priority = 1;
     }
     node = new ServerQueue(serverName, priority, locking.getServerLock(serverName));
@@ -848,9 +837,12 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     try {
       final LockAndQueue lock = locking.getServerLock(serverName);
       if (lock.tryExclusiveLock(procedure)) {
-        // We do not need to create a new queue so just pass null, as in tests we may pass
-        // procedures other than ServerProcedureInterface
-        removeFromRunQueue(serverRunQueue, getServerQueue(serverName, null));
+        // In tests we may pass procedures other than ServerProcedureInterface, just pass null if
+        // so.
+        removeFromRunQueue(serverRunQueue,
+          getServerQueue(serverName,
+            procedure instanceof ServerProcedureInterface ? (ServerProcedureInterface) procedure
+              : null));
         return false;
       }
       waitProcedure(lock, procedure);
@@ -873,9 +865,12 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       final LockAndQueue lock = locking.getServerLock(serverName);
       // Only SCP will acquire/release server lock so do not need to check the return value here.
       lock.releaseExclusiveLock(procedure);
-      // We do not need to create a new queue so just pass null, as in tests we may pass procedures
-      // other than ServerProcedureInterface
-      addToRunQueue(serverRunQueue, getServerQueue(serverName, null));
+      // In tests we may pass procedures other than ServerProcedureInterface, just pass null if
+      // so.
+      addToRunQueue(serverRunQueue,
+        getServerQueue(serverName,
+          procedure instanceof ServerProcedureInterface ? (ServerProcedureInterface) procedure
+            : null));
       int waitingCount = wakeWaitingProcedures(lock);
       wakePollIfNeeded(waitingCount);
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
index 587cc82..58263d3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
@@ -99,7 +99,7 @@ public final class MasterProcedureUtil {
     protected abstract void run() throws IOException;
     protected abstract String getDescription();
 
-    protected long submitProcedure(final Procedure<?> proc) {
+    protected long submitProcedure(final Procedure<MasterProcedureEnv> proc) {
       assert procId == null : "submitProcedure() was already called, running procId=" + procId;
       procId = getProcedureExecutor().submitProcedure(proc, nonceKey);
       return procId;

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java
index 86d8e43..0e80e2a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java
@@ -30,20 +30,6 @@ class PeerQueue extends Queue<String> {
   }
 
   @Override
-  public boolean isAvailable() {
-    if (isEmpty()) {
-      return false;
-    }
-    if (getLockStatus().hasExclusiveLock()) {
-      // if we have an exclusive lock already taken
-      // only child of the lock owner can be executed
-      Procedure<?> nextProc = peek();
-      return nextProc != null && getLockStatus().hasLockAccess(nextProc);
-    }
-    return true;
-  }
-
-  @Override
   public boolean requireExclusiveLock(Procedure<?> proc) {
     return requirePeerExclusiveLock((PeerProcedureInterface) proc);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
index df0875e..328ac00 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
@@ -106,7 +106,7 @@ public final class ProcedureSyncWait {
     }
 
   public static Future<byte[]> submitProcedure(final ProcedureExecutor<MasterProcedureEnv> procExec,
-      final Procedure<?> proc) {
+      final Procedure<MasterProcedureEnv> proc) {
     if (proc.isInitializing()) {
       procExec.submitProcedure(proc);
     }
@@ -114,7 +114,7 @@ public final class ProcedureSyncWait {
   }
 
   public static byte[] submitAndWaitProcedure(ProcedureExecutor<MasterProcedureEnv> procExec,
-      final Procedure<?> proc) throws IOException {
+      final Procedure<MasterProcedureEnv> proc) throws IOException {
     if (proc.isInitializing()) {
       procExec.submitProcedure(proc);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/Queue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/Queue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/Queue.java
index f7bea2a..43e66d0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/Queue.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/Queue.java
@@ -63,7 +63,18 @@ abstract class Queue<TKey extends Comparable<TKey>> extends AvlLinkedNode<Queue<
   // This should go away when we have the new AM and its events
   // and we move xlock to the lock-event-queue.
   public boolean isAvailable() {
-    return !lockStatus.hasExclusiveLock() && !isEmpty();
+    if (isEmpty()) {
+      return false;
+    }
+    if (getLockStatus().hasExclusiveLock()) {
+      // If we have an exclusive lock already taken, only child of the lock owner can be executed
+      // And now we will restore locks when master restarts, so it is possible that the procedure
+      // which is holding the lock is also in the queue, so we need to use hasLockAccess here
+      // instead of hasParentLock
+      Procedure<?> nextProc = peek();
+      return nextProc != null && getLockStatus().hasLockAccess(nextProc);
+    }
+    return true;
   }
 
   // ======================================================================

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
index 458e073..e133a65 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
@@ -36,8 +36,6 @@ public abstract class AbstractPeerProcedure<TState>
 
   protected String peerId;
 
-  private volatile boolean locked;
-
   // used to keep compatible with old client where we can only returns after updateStorage.
   protected ProcedurePrepareLatch latch;
 
@@ -59,17 +57,20 @@ public abstract class AbstractPeerProcedure<TState>
   }
 
   @Override
+  protected boolean waitInitialized(MasterProcedureEnv env) {
+    return env.waitInitialized(this);
+  }
+
+  @Override
   protected LockState acquireLock(MasterProcedureEnv env) {
     if (env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)) {
       return LockState.LOCK_EVENT_WAIT;
     }
-    locked = true;
     return LockState.LOCK_ACQUIRED;
   }
 
   @Override
   protected void releaseLock(MasterProcedureEnv env) {
-    locked = false;
     env.getProcedureScheduler().wakePeerExclusiveLock(this, peerId);
   }
 
@@ -79,11 +80,6 @@ public abstract class AbstractPeerProcedure<TState>
   }
 
   @Override
-  protected boolean hasLock(MasterProcedureEnv env) {
-    return locked;
-  }
-
-  @Override
   protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.serializeStateData(serializer);
     serializer.serialize(PeerProcedureStateData.newBuilder().setPeerId(peerId).build());

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp b/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
index 4e546cd..f617237 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
@@ -46,7 +46,7 @@
   long millisFromLastRoll = walStore.getMillisFromLastRoll();
   ArrayList<ProcedureWALFile> procedureWALFiles = walStore.getActiveLogs();
   Set<ProcedureWALFile> corruptedWALFiles = walStore.getCorruptedLogs();
-  List<Procedure<?>> procedures = procExecutor.getProcedures();
+  List<Procedure<MasterProcedureEnv>> procedures = procExecutor.getProcedures();
   Collections.sort(procedures, new Comparator<Procedure>() {
     @Override
     public int compare(Procedure lhs, Procedure rhs) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetProcedureResult.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetProcedureResult.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetProcedureResult.java
index 8b1584f..4186594 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetProcedureResult.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetProcedureResult.java
@@ -124,7 +124,7 @@ public class TestGetProcedureResult {
 
   @Test
   public void testRace() throws Exception {
-    ProcedureExecutor<?> executor =
+    ProcedureExecutor<MasterProcedureEnv> executor =
       UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
     DummyProcedure p = new DummyProcedure();
     long procId = executor.submitProcedure(p);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
index 08ecb81..443bbab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
 import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher;
 import org.apache.hadoop.hbase.procedure2.Procedure;
@@ -434,7 +435,7 @@ public class TestAssignmentManager {
     am.wakeMetaLoadedEvent();
   }
 
-  private Future<byte[]> submitProcedure(final Procedure<?> proc) {
+  private Future<byte[]> submitProcedure(final Procedure<MasterProcedureEnv> proc) {
     return ProcedureSyncWait.submitProcedure(master.getMasterProcedureExecutor(), proc);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java
index 9a0e2f6..a56e842 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java
@@ -111,7 +111,7 @@ public class TestMasterProcedureEvents {
   }
 
   private void testProcedureEventWaitWake(final HMaster master, final ProcedureEvent<?> event,
-      final Procedure<?> proc) throws Exception {
+      final Procedure<MasterProcedureEnv> proc) throws Exception {
     final ProcedureExecutor<MasterProcedureEnv> procExec = master.getMasterProcedureExecutor();
     final MasterProcedureScheduler procSched = procExec.getEnvironment().getProcedureScheduler();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
index c003379..02f0257 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
@@ -207,7 +207,7 @@ public class TestProcedureAdmin {
     // Wait for one step to complete
     ProcedureTestingUtility.waitProcedure(procExec, procId);
 
-    List<Procedure<?>> procedures = procExec.getProcedures();
+    List<Procedure<MasterProcedureEnv>> procedures = procExec.getProcedures();
     assertTrue(procedures.size() >= 1);
     boolean found = false;
     for (Procedure<?> proc: procedures) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestFailedProcCleanup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestFailedProcCleanup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestFailedProcCleanup.java
index 3e21951..1402bbd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestFailedProcCleanup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestFailedProcCleanup.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -88,7 +89,7 @@ public class TestFailedProcCleanup {
       LOG.debug("Ignoring exception: ", e);
       Thread.sleep(evictionDelay * 3);
     }
-    List<Procedure<?>> procedureInfos =
+    List<Procedure<MasterProcedureEnv>> procedureInfos =
         TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor().getProcedures();
     for (Procedure procedureInfo : procedureInfos) {
       if (procedureInfo.getProcName().equals("CreateTableProcedure")
@@ -109,7 +110,7 @@ public class TestFailedProcCleanup {
       LOG.debug("Ignoring exception: ", e);
       Thread.sleep(evictionDelay * 3);
     }
-    List<Procedure<?>> procedureInfos =
+    List<Procedure<MasterProcedureEnv>> procedureInfos =
         TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor().getProcedures();
     for (Procedure procedureInfo : procedureInfos) {
       if (procedureInfo.getProcName().equals("CreateTableProcedure")

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3f17fa1/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index a0b5d9d..163c2ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -587,7 +587,7 @@ public class TestAccessController extends SecureTestUtil {
     Procedure proc = new TestTableDDLProcedure(procExec.getEnvironment(), tableName);
     proc.setOwner(USER_OWNER);
     procExec.submitProcedure(proc);
-    final List<Procedure<?>> procList = procExec.getProcedures();
+    final List<Procedure<MasterProcedureEnv>> procList = procExec.getProcedures();
 
     AccessTestAction getProceduresAction = new AccessTestAction() {
       @Override


[21/24] hbase git commit: HBASE-20921 Possible NPE in ReopenTableRegionsProcedure

Posted by md...@apache.org.
HBASE-20921 Possible NPE in ReopenTableRegionsProcedure


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

Branch: refs/heads/HBASE-20749
Commit: 80b40a3b588d9d250c1f3fd0ce4ee50376fbe25e
Parents: d43e28d
Author: Allan Yang <al...@apache.org>
Authored: Fri Jul 27 09:36:22 2018 +0800
Committer: Allan Yang <al...@apache.org>
Committed: Fri Jul 27 09:36:50 2018 +0800

----------------------------------------------------------------------
 .../hbase/master/assignment/RegionStates.java   |   6 +
 .../procedure/ReopenTableRegionsProcedure.java  |   2 +-
 .../assignment/TestModifyTableWhileMerging.java | 109 +++++++++++++++++++
 3 files changed, 116 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/80b40a3b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
index b1b2371..9f01293 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -643,6 +643,12 @@ public class RegionStates {
    */
   public HRegionLocation checkReopened(HRegionLocation oldLoc) {
     RegionStateNode node = getRegionStateNode(oldLoc.getRegion());
+    // HBASE-20921
+    // if the oldLoc's state node does not exist, that means the region is
+    // merged or split, no need to check it
+    if (node == null) {
+      return null;
+    }
     synchronized (node) {
       if (oldLoc.getSeqNum() >= 0) {
         // in OPEN state before

http://git-wip-us.apache.org/repos/asf/hbase/blob/80b40a3b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java
index 7928c5b..8f3aa22 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java
@@ -124,7 +124,7 @@ public class ReopenTableRegionsProcedure
   @Override
   protected void rollbackState(MasterProcedureEnv env, ReopenTableRegionsState state)
       throws IOException, InterruptedException {
-    throw new UnsupportedOperationException();
+    throw new UnsupportedOperationException("unhandled state=" + state);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/80b40a3b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java
new file mode 100644
index 0000000..16ad373
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java
@@ -0,0 +1,109 @@
+/**
+ * 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.assignment;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestModifyTableWhileMerging {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestModifyTableWhileMerging.class);
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestModifyTableWhileMerging.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static TableName TABLE_NAME = TableName.valueOf("test");
+  private static Admin admin;
+  private static Table client;
+  private static byte[] CF = Bytes.toBytes("cf");
+  private static byte[] SPLITKEY = Bytes.toBytes("bbbbbbb");
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    //Set procedure executor thread to 1, making reproducing this issue of HBASE-20921 easier
+    UTIL.getConfiguration().setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+    UTIL.startMiniCluster(1);
+    admin = UTIL.getHBaseAdmin();
+    byte[][] splitKeys = new byte[1][];
+    splitKeys[0] = SPLITKEY;
+    client = UTIL.createTable(TABLE_NAME, CF, splitKeys);
+    UTIL.waitTableAvailable(TABLE_NAME);
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    TableDescriptor tableDescriptor = client.getDescriptor();
+    ProcedureExecutor<MasterProcedureEnv> executor = UTIL.getMiniHBaseCluster().getMaster()
+        .getMasterProcedureExecutor();
+    MasterProcedureEnv env = executor.getEnvironment();
+    List<RegionInfo> regionInfos = admin.getRegions(TABLE_NAME);
+    MergeTableRegionsProcedure mergeTableRegionsProcedure = new MergeTableRegionsProcedure(
+      UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor()
+        .getEnvironment(), regionInfos.get(0), regionInfos.get(1));
+    ModifyTableProcedure modifyTableProcedure = new ModifyTableProcedure(env, tableDescriptor);
+    long procModify = executor.submitProcedure(modifyTableProcedure);
+    UTIL.waitFor(30000, () -> executor.getProcedures().stream()
+      .filter(p -> p instanceof ModifyTableProcedure)
+      .map(p -> (ModifyTableProcedure) p)
+      .anyMatch(p -> TABLE_NAME.equals(p.getTableName())));
+    long proc = executor.submitProcedure(mergeTableRegionsProcedure);
+    UTIL.waitFor(3000000, () -> UTIL.getMiniHBaseCluster().getMaster()
+        .getMasterProcedureExecutor().isFinished(procModify));
+    Assert.assertEquals("Modify Table procedure should success!",
+        ProcedureProtos.ProcedureState.SUCCESS, modifyTableProcedure.getState());
+  }
+
+}


[15/24] hbase git commit: HBASE-20649 Validate HFiles do not have PREFIX_TREE DataBlockEncoding

Posted by md...@apache.org.
HBASE-20649 Validate HFiles do not have PREFIX_TREE DataBlockEncoding

Amending-Author: Balazs Meszaros <ba...@cloudera.com>

Signed-off-by: Sean Busbey <bu...@apache.org>


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

Branch: refs/heads/HBASE-20749
Commit: ba5d1c1f28301adc99019d9d6c4a04fac98ae511
Parents: 44f6ef1
Author: Peter Somogyi <ps...@apache.org>
Authored: Tue Jun 26 10:47:06 2018 +0200
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed Jul 25 09:00:00 2018 -0500

----------------------------------------------------------------------
 .../hbase/tool/DataBlockEncodingValidator.java  |   5 +-
 .../hbase/tool/HFileContentValidator.java       | 126 +++++++++++++++++++
 .../hadoop/hbase/tool/PreUpgradeValidator.java  |   7 ++
 src/main/asciidoc/_chapters/ops_mgt.adoc        | 102 ++++++++++++++-
 4 files changed, 234 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ba5d1c1f/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/DataBlockEncodingValidator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/DataBlockEncodingValidator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/DataBlockEncodingValidator.java
index e72521b..c909725 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/DataBlockEncodingValidator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/DataBlockEncodingValidator.java
@@ -76,9 +76,8 @@ public class DataBlockEncodingValidator extends AbstractHBaseTool {
 
     if (incompatibilities > 0) {
       LOG.warn("There are {} column families with incompatible Data Block Encodings. Do not "
-          + "upgrade until these encodings are converted to a supported one.", incompatibilities);
-      LOG.warn("Check http://hbase.apache.org/book.html#upgrade2.0.prefix-tree.removed "
-          + "for instructions.");
+          + "upgrade until these encodings are converted to a supported one. "
+          + "Check https://s.apache.org/prefixtree for instructions.", incompatibilities);
     } else {
       LOG.info("The used Data Block Encodings are compatible with HBase 2.0.");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ba5d1c1f/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/HFileContentValidator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/HFileContentValidator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/HFileContentValidator.java
new file mode 100644
index 0000000..d60844b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/HFileContentValidator.java
@@ -0,0 +1,126 @@
+/**
+ *
+ * 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.tool;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+public class HFileContentValidator extends AbstractHBaseTool {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HFileContentValidator.class);
+
+  /**
+   * Check HFile contents are readable by HBase 2.
+   *
+   * @param conf used configuration
+   * @return number of HFiles corrupted HBase
+   * @throws IOException if a remote or network exception occurs
+   */
+  private boolean validateHFileContent(Configuration conf) throws IOException {
+    FileSystem fileSystem = FSUtils.getCurrentFileSystem(conf);
+
+    ExecutorService threadPool = createThreadPool(conf);
+    HFileCorruptionChecker checker;
+
+    try {
+      checker = new HFileCorruptionChecker(conf, threadPool, false);
+
+      Path rootDir = FSUtils.getRootDir(conf);
+      LOG.info("Validating HFile contents under {}", rootDir);
+
+      Collection<Path> tableDirs = FSUtils.getTableDirs(fileSystem, rootDir);
+      checker.checkTables(tableDirs);
+
+      Path archiveRootDir = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY);
+      LOG.info("Validating HFile contents under {}", archiveRootDir);
+
+      List<Path> archiveTableDirs = FSUtils.getTableDirs(fileSystem, archiveRootDir);
+      checker.checkTables(archiveTableDirs);
+    } finally {
+      threadPool.shutdown();
+
+      try {
+        threadPool.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      }
+    }
+
+    int checkedFiles = checker.getHFilesChecked();
+    Collection<Path> corrupted = checker.getCorrupted();
+
+    if (corrupted.isEmpty()) {
+      LOG.info("Checked {} HFiles, none of them are corrupted.", checkedFiles);
+      LOG.info("There are no incompatible HFiles.");
+
+      return true;
+    } else {
+      LOG.info("Checked {} HFiles, {} are corrupted.", checkedFiles, corrupted.size());
+
+      for (Path path : corrupted) {
+        LOG.info("Corrupted file: {}", path);
+      }
+
+      LOG.info("Change data block encodings before upgrading. "
+          + "Check https://s.apache.org/prefixtree for instructions.");
+
+      return false;
+    }
+  }
+
+  private ExecutorService createThreadPool(Configuration conf) {
+    int availableProcessors = Runtime.getRuntime().availableProcessors();
+    int numThreads = conf.getInt("hfilevalidator.numthreads", availableProcessors);
+    return Executors.newFixedThreadPool(numThreads,
+        Threads.getNamedThreadFactory("hfile-validator"));
+  }
+
+  @Override
+  protected void addOptions() {
+  }
+
+  @Override
+  protected void processOptions(CommandLine cmd) {
+  }
+
+  @Override
+  protected int doWork() throws Exception {
+    return (validateHFileContent(getConf())) ? EXIT_SUCCESS : EXIT_FAILURE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ba5d1c1f/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/PreUpgradeValidator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/PreUpgradeValidator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/PreUpgradeValidator.java
index 7bf3074..818004c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/PreUpgradeValidator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/PreUpgradeValidator.java
@@ -38,6 +38,7 @@ import org.slf4j.LoggerFactory;
  * <ul>
  * <li>validate-cp: Validates Co-processors compatibility</li>
  * <li>validate-dbe: Check Data Block Encoding for column families</li>
+ * <li>validate-hfile: Check for corrupted HFiles</li>
  * </ul>
  * </p>
  */
@@ -49,6 +50,7 @@ public class PreUpgradeValidator implements Tool {
   public static final String TOOL_NAME = "pre-upgrade";
   public static final String VALIDATE_CP_NAME = "validate-cp";
   public static final String VALIDATE_DBE_NAME = "validate-dbe";
+  public static final String VALIDATE_HFILE = "validate-hfile";
 
   private Configuration configuration;
 
@@ -69,6 +71,8 @@ public class PreUpgradeValidator implements Tool {
         VALIDATE_CP_NAME);
     System.out.printf(" %-15s Validate DataBlockEncodings are compatible with HBase%n",
         VALIDATE_DBE_NAME);
+    System.out.printf(" %-15s Validate HFile contents are readable%n",
+        VALIDATE_HFILE);
     System.out.println("For further information, please use command -h");
   }
 
@@ -88,6 +92,9 @@ public class PreUpgradeValidator implements Tool {
       case VALIDATE_DBE_NAME:
         tool = new DataBlockEncodingValidator();
         break;
+      case VALIDATE_HFILE:
+        tool = new HFileContentValidator();
+        break;
       case "-h":
         printUsage();
         return AbstractHBaseTool.EXIT_FAILURE;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ba5d1c1f/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index 01e6de6..7e7001c 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -898,7 +898,8 @@ $ bin/hbase pre-upgrade validate-cp -table .*
 It validates every table level co-processors where the table name matches to `.*` regular expression.
 
 ==== DataBlockEncoding validation
-HBase 2.0 removed `PREFIX_TREE` Data Block Encoding from column families.
+HBase 2.0 removed `PREFIX_TREE` Data Block Encoding from column families. For further information
+please check <<upgrade2.0.prefix-tree.removed,_prefix-tree_ encoding removed>>.
 To verify that none of the column families are using incompatible Data Block Encodings in the cluster run the following command.
 
 [source, bash]
@@ -906,8 +907,103 @@ To verify that none of the column families are using incompatible Data Block Enc
 $ bin/hbase pre-upgrade validate-dbe
 ----
 
-This check validates all column families and print out any incompatibilities.
-To change `PREFIX_TREE` encoding to supported one check <<upgrade2.0.prefix-tree.removed,_prefix-tree_ encoding removed>>.
+This check validates all column families and print out any incompatibilities. For example:
+
+----
+2018-07-13 09:58:32,028 WARN  [main] tool.DataBlockEncodingValidator: Incompatible DataBlockEncoding for table: t, cf: f, encoding: PREFIX_TREE
+----
+
+Which means that Data Block Encoding of table `t`, column family `f` is incompatible. To fix, use `alter` command in HBase shell:
+
+----
+alter 't', { NAME => 'f', DATA_BLOCK_ENCODING => 'FAST_DIFF' }
+----
+
+Please also validate HFiles, which is described in the next section.
+
+==== HFile Content validation
+Even though Data Block Encoding is changed from `PREFIX_TREE` it is still possible to have HFiles that contain data encoded that way.
+To verify that HFiles are readable with HBase 2 please use _HFile content validator_.
+
+[source, bash]
+----
+$ bin/hbase pre-upgrade validate-hfile
+----
+
+The tool will log the corrupt HFiles and details about the root cause.
+If the problem is about PREFIX_TREE encoding it is necessary to change encodings before upgrading to HBase 2.
+
+The following log message shows an example of incorrect HFiles.
+
+----
+2018-06-05 16:20:46,976 WARN  [hfilevalidator-pool1-t3] hbck.HFileCorruptionChecker: Found corrupt HFile hdfs://example.com:8020/hbase/data/default/t/72ea7f7d625ee30f959897d1a3e2c350/prefix/7e6b3d73263c4851bf2b8590a9b3791e
+org.apache.hadoop.hbase.io.hfile.CorruptHFileException: Problem reading HFile Trailer from file hdfs://example.com:8020/hbase/data/default/t/72ea7f7d625ee30f959897d1a3e2c350/prefix/7e6b3d73263c4851bf2b8590a9b3791e
+    ...
+Caused by: java.io.IOException: Invalid data block encoding type in file info: PREFIX_TREE
+    ...
+Caused by: java.lang.IllegalArgumentException: No enum constant org.apache.hadoop.hbase.io.encoding.DataBlockEncoding.PREFIX_TREE
+    ...
+2018-06-05 16:20:47,322 INFO  [main] tool.HFileContentValidator: Corrupted file: hdfs://example.com:8020/hbase/data/default/t/72ea7f7d625ee30f959897d1a3e2c350/prefix/7e6b3d73263c4851bf2b8590a9b3791e
+2018-06-05 16:20:47,383 INFO  [main] tool.HFileContentValidator: Corrupted file: hdfs://example.com:8020/hbase/archive/data/default/t/56be41796340b757eb7fff1eb5e2a905/f/29c641ae91c34fc3bee881f45436b6d1
+----
+
+===== Fixing PREFIX_TREE errors
+
+It's possible to get `PREFIX_TREE` errors after changing Data Block Encoding to a supported one. It can happen
+because there are some HFiles which still encoded with `PREFIX_TREE` or there are still some snapshots.
+
+For fixing HFiles, please run a major compaction on the table (it was `default:t` according to the log message):
+
+----
+major_compact 't'
+----
+
+HFiles can be referenced from snapshots, too. It's the case when the HFile is located under `archive/data`.
+The first step is to determine which snapshot references that HFile (the name of the file was `29c641ae91c34fc3bee881f45436b6d1`
+according to the logs):
+
+[source, bash]
+----
+for snapshot in $(hbase snapshotinfo -list-snapshots 2> /dev/null | tail -n -1 | cut -f 1 -d \|);
+do
+  echo "checking snapshot named '${snapshot}'";
+  hbase snapshotinfo -snapshot "${snapshot}" -files 2> /dev/null | grep 29c641ae91c34fc3bee881f45436b6d1;
+done
+----
+
+The output of this shell script is:
+
+----
+checking snapshot named 't_snap'
+   1.0 K t/56be41796340b757eb7fff1eb5e2a905/f/29c641ae91c34fc3bee881f45436b6d1 (archive)
+----
+
+Which means `t_snap` snapshot references the incompatible HFile. If the snapshot is still needed,
+then it has to be recreated with HBase shell:
+
+----
+# creating a new namespace for the cleanup process
+create_namespace 'pre_upgrade_cleanup'
+
+# creating a new snapshot
+clone_snapshot 't_snap', 'pre_upgrade_cleanup:t'
+alter 'pre_upgrade_cleanup:t', { NAME => 'f', DATA_BLOCK_ENCODING => 'FAST_DIFF' }
+major_compact 'pre_upgrade_cleanup:t'
+
+# removing the invalid snapshot
+delete_snapshot 't_snap'
+
+# creating a new snapshot
+snapshot 'pre_upgrade_cleanup:t', 't_snap'
+
+# removing temporary table
+disable 'pre_upgrade_cleanup:t'
+drop 'pre_upgrade_cleanup:t'
+drop_namespace 'pre_upgrade_cleanup'
+----
+
+For further information, please refer to
+link:https://issues.apache.org/jira/browse/HBASE-20649?focusedCommentId=16535476#comment-16535476[HBASE-20649].
 
 [[ops.regionmgt]]
 == Region Management


[11/24] hbase git commit: HBASE-20873 Update doc for Endpoint-based Export

Posted by md...@apache.org.
HBASE-20873 Update doc for Endpoint-based Export

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/HBASE-20749
Commit: e44f506694e606cb3bd2a854c5e72afa802d26d0
Parents: 3a3855a
Author: Wei-Chiu Chuang <we...@cloudera.com>
Authored: Fri Jul 13 19:01:22 2018 -0700
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Wed Jul 25 10:38:12 2018 +0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/ops_mgt.adoc | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e44f5066/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index b6be867..01e6de6 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -503,9 +503,12 @@ $ bin/hbase org.apache.hadoop.hbase.mapreduce.Export <tablename> <outputdir> [<v
 ----
 
 *endpoint-based Export*
+
+NOTE: Make sure the Export coprocessor is enabled by adding `org.apache.hadoop.hbase.coprocessor.Export` to `hbase.coprocessor.region.classes`.
 ----
 $ bin/hbase org.apache.hadoop.hbase.coprocessor.Export <tablename> <outputdir> [<versions> [<starttime> [<endtime>]]]
 ----
+The outputdir is a HDFS directory that does not exist prior to the export. When done, the exported files will be owned by the user invoking the export command.
 
 *The Comparison of Endpoint-based Export And Mapreduce-based Export*
 |===


[09/24] hbase git commit: HBASE-20565 ColumnRangeFilter combined with ColumnPaginationFilter can produce incorrect result

Posted by md...@apache.org.
HBASE-20565 ColumnRangeFilter combined with ColumnPaginationFilter can produce incorrect result


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

Branch: refs/heads/HBASE-20749
Commit: a888af3dc9320a5dc3c2e41e46e31522c7a0e356
Parents: b631727
Author: huzheng <op...@gmail.com>
Authored: Tue Jul 17 20:37:39 2018 +0800
Committer: huzheng <op...@gmail.com>
Committed: Tue Jul 24 10:29:24 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/filter/ColumnRangeFilter.java  | 25 +++---
 .../hadoop/hbase/filter/FilterListWithAND.java  | 14 +++-
 .../hbase/filter/TestColumnRangeFilter.java     | 84 ++++++++++++++------
 .../hadoop/hbase/filter/TestFilterList.java     |  8 +-
 4 files changed, 88 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a888af3d/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
index cd103b8..f981ed7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
@@ -208,20 +208,23 @@ public class ColumnRangeFilter extends FilterBase {
   }
 
   /**
-   * @param other
-   * @return true if and only if the fields of the filter that are serialized
-   * are equal to the corresponding fields in other.  Used for testing.
+   * @param o filter to serialize.
+   * @return true if and only if the fields of the filter that are serialized are equal to the
+   *         corresponding fields in other. Used for testing.
    */
   @Override
   boolean areSerializedFieldsEqual(Filter o) {
-   if (o == this) return true;
-   if (!(o instanceof ColumnRangeFilter)) return false;
-
-   ColumnRangeFilter other = (ColumnRangeFilter)o;
-   return Bytes.equals(this.getMinColumn(),other.getMinColumn())
-     && this.getMinColumnInclusive() == other.getMinColumnInclusive()
-     && Bytes.equals(this.getMaxColumn(), other.getMaxColumn())
-     && this.getMaxColumnInclusive() == other.getMaxColumnInclusive();
+    if (o == this) {
+      return true;
+    }
+    if (!(o instanceof ColumnRangeFilter)) {
+      return false;
+    }
+    ColumnRangeFilter other = (ColumnRangeFilter) o;
+    return Bytes.equals(this.getMinColumn(), other.getMinColumn())
+        && this.getMinColumnInclusive() == other.getMinColumnInclusive()
+        && Bytes.equals(this.getMaxColumn(), other.getMaxColumn())
+        && this.getMaxColumnInclusive() == other.getMaxColumnInclusive();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a888af3d/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java
index 9f2ca21..ebff3a5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java
@@ -154,6 +154,11 @@ public class FilterListWithAND extends FilterListBase {
         "Received code is not valid. rc: " + rc + ", localRC: " + localRC);
   }
 
+  private boolean isIncludeRelatedReturnCode(ReturnCode rc) {
+    return isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.INCLUDE_AND_NEXT_COL,
+      ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
+  }
+
   @Override
   public ReturnCode filterCell(Cell c) throws IOException {
     if (isEmpty()) {
@@ -168,11 +173,16 @@ public class FilterListWithAND extends FilterListBase {
       }
       ReturnCode localRC;
       localRC = filter.filterCell(c);
-      rc = mergeReturnCode(rc, localRC);
-
       if (localRC == ReturnCode.SEEK_NEXT_USING_HINT) {
         seekHintFilters.add(filter);
       }
+      rc = mergeReturnCode(rc, localRC);
+      // Only when rc is INCLUDE* case, we should pass the cell to the following sub-filters.
+      // otherwise we may mess up the global state (such as offset, count..) in the following
+      // sub-filters. (HBASE-20565)
+      if (!isIncludeRelatedReturnCode(rc)) {
+        return rc;
+      }
     }
     if (!seekHintFilters.isEmpty()) {
       return ReturnCode.SEEK_NEXT_USING_HINT;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a888af3d/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java
index 98bc57b..c69466e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.filter;
 
 import static org.junit.Assert.*;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.FilterList.Operator;
 import org.apache.hadoop.hbase.testclassification.FilterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -226,38 +228,68 @@ public class TestColumnRangeFilter {
     Scan scan = new Scan();
     scan.setMaxVersions();
     for (StringRange s : rangeMap.keySet()) {
-      filter = new ColumnRangeFilter(s.getStart() == null ? null
-          : Bytes.toBytes(s.getStart()), s.isStartInclusive(),
-          s.getEnd() == null ? null : Bytes.toBytes(s.getEnd()),
+      filter = new ColumnRangeFilter(s.getStart() == null ? null : Bytes.toBytes(s.getStart()),
+          s.isStartInclusive(), s.getEnd() == null ? null : Bytes.toBytes(s.getEnd()),
           s.isEndInclusive());
-      scan.setFilter(filter);
-      ResultScanner scanner = ht.getScanner(scan);
-      List<Cell> results = new ArrayList<>();
-      LOG.info("scan column range: " + s.toString());
-      long timeBeforeScan = System.currentTimeMillis();
+      assertEquals(rangeMap.get(s).size(), cellsCount(ht, filter));
+    }
+    ht.close();
+  }
+
+  @Test
+  public void TestColumnRangeFilterWithColumnPaginationFilter() throws Exception {
+    String family = "Family";
+    String table = "TestColumnRangeFilterWithColumnPaginationFilter";
+    try (Table ht =
+        TEST_UTIL.createTable(TableName.valueOf(table), Bytes.toBytes(family), Integer.MAX_VALUE)) {
+      // one row.
+      String row = "row";
+      // One version
+      long timestamp = 100;
+      // 10 columns
+      int[] columns = { 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 };
+      String valueString = "ValueString";
+
+      Put p = new Put(Bytes.toBytes(row));
+      p.setDurability(Durability.SKIP_WAL);
+      for (int column : columns) {
+        KeyValue kv =
+            KeyValueTestUtil.create(row, family, Integer.toString(column), timestamp, valueString);
+        p.add(kv);
+      }
+      ht.put(p);
+
+      TEST_UTIL.flush();
+
+      // Column range from 1 to 9.
+      StringRange stringRange = new StringRange("1", true, "9", false);
+      ColumnRangeFilter filter1 = new ColumnRangeFilter(Bytes.toBytes(stringRange.getStart()),
+          stringRange.isStartInclusive(), Bytes.toBytes(stringRange.getEnd()),
+          stringRange.isEndInclusive());
+
+      ColumnPaginationFilter filter2 = new ColumnPaginationFilter(5, 0);
+      ColumnPaginationFilter filter3 = new ColumnPaginationFilter(5, 1);
+      ColumnPaginationFilter filter4 = new ColumnPaginationFilter(5, 2);
+      ColumnPaginationFilter filter5 = new ColumnPaginationFilter(5, 6);
+      ColumnPaginationFilter filter6 = new ColumnPaginationFilter(5, 9);
+      assertEquals(5, cellsCount(ht, new FilterList(Operator.MUST_PASS_ALL, filter1, filter2)));
+      assertEquals(5, cellsCount(ht, new FilterList(Operator.MUST_PASS_ALL, filter1, filter3)));
+      assertEquals(5, cellsCount(ht, new FilterList(Operator.MUST_PASS_ALL, filter1, filter4)));
+      assertEquals(2, cellsCount(ht, new FilterList(Operator.MUST_PASS_ALL, filter1, filter5)));
+      assertEquals(0, cellsCount(ht, new FilterList(Operator.MUST_PASS_ALL, filter1, filter6)));
+    }
+  }
 
+  private int cellsCount(Table table, Filter filter) throws IOException {
+    Scan scan = new Scan().setFilter(filter).readAllVersions();
+    try (ResultScanner scanner = table.getScanner(scan)) {
+      List<Cell> results = new ArrayList<>();
       Result result;
       while ((result = scanner.next()) != null) {
-        for (Cell kv : result.listCells()) {
-          results.add(kv);
-        }
-      }
-      long scanTime = System.currentTimeMillis() - timeBeforeScan;
-      scanner.close();
-      LOG.info("scan time = " + scanTime + "ms");
-      LOG.info("found " + results.size() + " results");
-      LOG.info("Expecting " + rangeMap.get(s).size() + " results");
-
-      /*
-      for (KeyValue kv : results) {
-        LOG.info("found row " + Bytes.toString(kv.getRow()) + ", column "
-            + Bytes.toString(kv.getQualifier()));
+        result.listCells().forEach(results::add);
       }
-      */
-
-      assertEquals(rangeMap.get(s).size(), results.size());
+      return results.size();
     }
-    ht.close();
   }
 
   List<String> generateRandomWords(int numberOfWords, int maxLengthOfWords) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a888af3d/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
index b2fe9d0..5f9515a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
@@ -525,7 +525,7 @@ public class TestFilterList {
     filterList = new FilterList(Operator.MUST_PASS_ALL,
         Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
     filterList.filterCell(null);
-    assertEquals(0, comparator.compare(filterList.getNextCellHint(null), maxKeyValue));
+    assertEquals(0, comparator.compare(filterList.getNextCellHint(null), minKeyValue));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL,
         Arrays.asList(new Filter [] { filterMaxHint, filterMinHint } ));
@@ -536,7 +536,7 @@ public class TestFilterList {
     filterList = new FilterList(Operator.MUST_PASS_ALL,
         Arrays.asList(new Filter[] { filterNoHint, filterMinHint, filterMaxHint }));
     filterList.filterCell(null);
-    assertEquals(0, comparator.compare(filterList.getNextCellHint(null), maxKeyValue));
+    assertEquals(0, comparator.compare(filterList.getNextCellHint(null), minKeyValue));
     filterList = new FilterList(Operator.MUST_PASS_ALL,
         Arrays.asList(new Filter[] { filterNoHint, filterMaxHint }));
     filterList.filterCell(null);
@@ -744,10 +744,10 @@ public class TestFilterList {
     assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter5, filter6);
-    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
+    assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter6);
-    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
+    assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter1);
     assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv1));


[07/24] hbase git commit: HBASE-20893 Data loss if splitting region while ServerCrashProcedure executing

Posted by md...@apache.org.
HBASE-20893 Data loss if splitting region while ServerCrashProcedure executing


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

Branch: refs/heads/HBASE-20749
Commit: 4804483f7e55edf91a8e9d7ad30ad8239a96eaf3
Parents: 37de961
Author: Allan Yang <al...@apache.org>
Authored: Mon Jul 23 14:48:43 2018 +0800
Committer: Allan Yang <al...@163.com>
Committed: Mon Jul 23 14:48:43 2018 +0800

----------------------------------------------------------------------
 .../src/main/protobuf/MasterProcedure.proto     |   1 +
 .../assignment/SplitTableRegionProcedure.java   |  23 ++++
 .../apache/hadoop/hbase/wal/WALSplitter.java    |  24 ++++
 .../master/TestSplitRegionWhileRSCrash.java     | 122 +++++++++++++++++++
 4 files changed, 170 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4804483f/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index a062e9a..d651011 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -248,6 +248,7 @@ enum SplitTableRegionState {
   SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_META = 8;
   SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS = 9;
   SPLIT_TABLE_REGION_POST_OPERATION = 10;
+  SPLIT_TABLE_REGIONS_CHECK_CLOSED_REGIONS = 11;
 }
 
 message SplitTableRegionStateData {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4804483f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index 2306037..f0ea25b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -139,6 +139,21 @@ public class SplitTableRegionProcedure
   }
 
   /**
+   * Check whether there is recovered.edits in the closed region
+   * If any, that means this region is not closed property, we need
+   * to abort region merge to prevent data loss
+   * @param env master env
+   * @throws IOException IOException
+   */
+  private void checkClosedRegion(final MasterProcedureEnv env) throws IOException {
+    if (WALSplitter.hasRecoveredEdits(env.getMasterServices().getFileSystem(),
+        env.getMasterConfiguration(), getRegion())) {
+      throw new IOException("Recovered.edits are found in Region: " + getRegion()
+          + ", abort split to prevent data loss");
+    }
+  }
+
+  /**
    * Check whether the region is splittable
    * @param env MasterProcedureEnv
    * @param regionToSplit parent Region to be split
@@ -238,6 +253,10 @@ public class SplitTableRegionProcedure
           break;
         case SPLIT_TABLE_REGION_CLOSE_PARENT_REGION:
           addChildProcedure(createUnassignProcedures(env, getRegionReplication(env)));
+          setNextState(SplitTableRegionState.SPLIT_TABLE_REGIONS_CHECK_CLOSED_REGIONS);
+          break;
+        case SPLIT_TABLE_REGIONS_CHECK_CLOSED_REGIONS:
+          checkClosedRegion(env);
           setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS);
           break;
         case SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS:
@@ -312,6 +331,10 @@ public class SplitTableRegionProcedure
         case SPLIT_TABLE_REGION_WRITE_MAX_SEQUENCE_ID_FILE:
           // Doing nothing, as re-open parent region would clean up daughter region directories.
           break;
+        case SPLIT_TABLE_REGIONS_CHECK_CLOSED_REGIONS:
+          // Doing nothing, in SPLIT_TABLE_REGION_CLOSE_PARENT_REGION,
+          // we will bring parent region online
+          break;
         case SPLIT_TABLE_REGION_CLOSE_PARENT_REGION:
           openParentRegion(env);
           break;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4804483f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index f020e7a..65d5fb7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
@@ -544,6 +545,29 @@ public class WALSplitter {
   }
 
   /**
+   * Check whether there is recovered.edits in the region dir
+   * @param fs FileSystem
+   * @param conf conf
+   * @param regionInfo the region to check
+   * @throws IOException IOException
+   * @return true if recovered.edits exist in the region dir
+   */
+  public static boolean hasRecoveredEdits(final FileSystem fs,
+      final Configuration conf, final RegionInfo regionInfo) throws IOException {
+    // No recovered.edits for non default replica regions
+    if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
+      return false;
+    }
+    Path rootDir = FSUtils.getRootDir(conf);
+    //Only default replica region can reach here, so we can use regioninfo
+    //directly without converting it to default replica's regioninfo.
+    Path regionDir = HRegion.getRegionDir(rootDir, regionInfo);
+    NavigableSet<Path> files = getSplitEditFilesSorted(fs, regionDir);
+    return files != null && !files.isEmpty();
+  }
+
+
+  /**
    * Returns sorted set of edit files made by splitter, excluding files
    * with '.temp' suffix.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/4804483f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java
new file mode 100644
index 0000000..a881575
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java
@@ -0,0 +1,122 @@
+/*
+ * 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;
+
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.assignment.SplitTableRegionProcedure;
+import org.apache.hadoop.hbase.master.assignment.UnassignProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestSplitRegionWhileRSCrash {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestSplitRegionWhileRSCrash.class);
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestSplitRegionWhileRSCrash.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static TableName TABLE_NAME = TableName.valueOf("test");
+  private static Admin admin;
+  private static byte[] CF = Bytes.toBytes("cf");
+  private static CountDownLatch mergeCommitArrive = new CountDownLatch(1);
+  private static Table TABLE;
+
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    UTIL.startMiniCluster(1);
+    admin = UTIL.getHBaseAdmin();
+    TABLE = UTIL.createTable(TABLE_NAME, CF);
+    UTIL.waitTableAvailable(TABLE_NAME);
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    MasterProcedureEnv env = UTIL.getMiniHBaseCluster().getMaster()
+        .getMasterProcedureExecutor().getEnvironment();
+    final ProcedureExecutor<MasterProcedureEnv> executor = UTIL.getMiniHBaseCluster()
+        .getMaster().getMasterProcedureExecutor();
+    List<RegionInfo> regionInfos = admin.getRegions(TABLE_NAME);
+    //Since a flush request will be sent while initializing SplitTableRegionProcedure
+    //Create SplitTableRegionProcedure first before put data
+    SplitTableRegionProcedure splitProcedure = new SplitTableRegionProcedure(
+        env, regionInfos.get(0), Bytes.toBytes("row5"));
+    //write some rows to the table
+    LOG.info("Begin to put data");
+    for (int i = 0; i < 10; i++) {
+      byte[] row = Bytes.toBytes("row" + i);
+      Put put = new Put(row);
+      put.addColumn(CF, CF, CF);
+      TABLE.put(put);
+    }
+    executor.submitProcedure(splitProcedure);
+    LOG.info("SplitProcedure submitted");
+    UTIL.waitFor(30000, () -> executor.getProcedures().stream()
+        .filter(p -> p instanceof UnassignProcedure)
+        .map(p -> (UnassignProcedure) p)
+        .anyMatch(p -> TABLE_NAME.equals(p.getTableName())));
+    UTIL.getMiniHBaseCluster().killRegionServer(
+        UTIL.getMiniHBaseCluster().getRegionServer(0).getServerName());
+    UTIL.getMiniHBaseCluster().startRegionServer();
+    UTIL.waitUntilNoRegionsInTransition();
+    Scan scan = new Scan();
+    ResultScanner results = TABLE.getScanner(scan);
+    int count = 0;
+    Result result = null;
+    while ((result = results.next()) != null) {
+      count++;
+    }
+    Assert.assertEquals("There should be 10 rows!", 10, count);
+  }
+}
\ No newline at end of file


[16/24] hbase git commit: HBASE-20928 Rewrite calculation of midpoint - addemdum

Posted by md...@apache.org.
HBASE-20928 Rewrite calculation of midpoint - addemdum

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-20749
Commit: 1913164970b17365e51662cf9cc9acacdda04fec
Parents: ba5d1c1
Author: Xu Cang <xc...@salesforce.com>
Authored: Tue Jul 24 11:37:21 2018 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Jul 25 10:00:58 2018 -0700

----------------------------------------------------------------------
 hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/19131649/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
index 12aaa67..e63c0db 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
@@ -2068,7 +2068,7 @@ public class Bytes implements Comparable<Bytes> {
     int high = arr.length - 1;
     KeyValue.KeyOnlyKeyValue r = new KeyValue.KeyOnlyKeyValue();
     while (low <= high) {
-      int mid = (low+high) >>> 1;
+      int mid = low + ((high - low) >> 1);
       // we have to compare in this order, because the comparator order
       // has special logic when the 'left side' is a special key.
       r.setKey(arr[mid], 0, arr[mid].length);


[10/24] hbase git commit: HBASE-20928 Rewrite calculation of midpoint in binarySearch functions to prevent overflow

Posted by md...@apache.org.
HBASE-20928 Rewrite calculation of midpoint in binarySearch functions to prevent overflow

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-20749
Commit: 3a3855aade174d89087c5959320b7662347c1275
Parents: a888af3
Author: Saurabh Singh <sa...@csm-hadoop-spark-box.example.com>
Authored: Tue Jul 24 11:23:22 2018 +0530
Committer: tedyu <yu...@gmail.com>
Committed: Tue Jul 24 08:16:30 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java  | 4 ++--
 .../src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java    | 2 +-
 .../src/main/java/org/apache/hadoop/hbase/util/Bytes.java      | 6 +++---
 .../java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java | 2 +-
 .../java/org/apache/hadoop/hbase/regionserver/CellFlatMap.java | 2 +-
 .../apache/hadoop/hbase/util/BoundedPriorityBlockingQueue.java | 2 +-
 6 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3a3855aa/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
index 1dd4e19..e1f00e2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
@@ -126,10 +126,10 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
   private int binarySearch(Cell seekCell, boolean seekBefore) {
     int low = 0;
     int high = rowNumber - 1;
-    int mid = (low + high) >>> 1;
+    int mid = low + ((high - low) >> 1);
     int comp = 0;
     while (low <= high) {
-      mid = (low + high) >>> 1;
+      mid = low + ((high - low) >> 1);
       ByteBuffer row = getRow(mid);
       comp = compareRows(row, seekCell);
       if (comp < 0) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3a3855aa/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
index 17b2a7c..68cf56e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
@@ -527,7 +527,7 @@ public abstract class ByteBuff {
     int high = toIndex - 1;
 
     while (low <= high) {
-      int mid = (low + high) >>> 1;
+      int mid = low + ((high - low) >> 1);
       int midVal = a.get(mid) & 0xff;
 
       if (midVal < unsignedKey) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3a3855aa/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
index 15facea..12aaa67 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
@@ -2029,7 +2029,7 @@ public class Bytes implements Comparable<Bytes> {
     int high = arr.length - 1;
 
     while (low <= high) {
-      int mid = (low + high) >>> 1;
+      int mid = low + ((high - low) >> 1);
       // we have to compare in this order, because the comparator order
       // has special logic when the 'left side' is a special key.
       int cmp = Bytes.BYTES_RAWCOMPARATOR
@@ -2104,7 +2104,7 @@ public class Bytes implements Comparable<Bytes> {
     int low = 0;
     int high = arr.length - 1;
     while (low <= high) {
-      int mid = (low+high) >>> 1;
+      int mid = low + ((high - low) >> 1);
       // we have to compare in this order, because the comparator order
       // has special logic when the 'left side' is a special key.
       int cmp = comparator.compare(key, arr[mid]);
@@ -2278,7 +2278,7 @@ public class Bytes implements Comparable<Bytes> {
     int high = toIndex - 1;
 
     while (low <= high) {
-      int mid = (low + high) >>> 1;
+      int mid = low + ((high - low) >> 1);
       int midVal = a[mid] & 0xff;
 
       if (midVal < unsignedKey) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3a3855aa/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
index d353bf8..b91b48c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
@@ -745,7 +745,7 @@ public class HFileBlockIndex {
       ByteBufferKeyOnlyKeyValue nonRootIndexkeyOnlyKV = new ByteBufferKeyOnlyKeyValue();
       ObjectIntPair<ByteBuffer> pair = new ObjectIntPair<>();
       while (low <= high) {
-        mid = (low + high) >>> 1;
+        mid = low + ((high - low) >> 1);
 
         // Midkey's offset relative to the end of secondary index
         int midKeyRelOffset = nonRootIndex.getIntAfterPosition(Bytes.SIZEOF_INT * (mid + 1));

http://git-wip-us.apache.org/repos/asf/hbase/blob/3a3855aa/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellFlatMap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellFlatMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellFlatMap.java
index 17e64b0..ae48ea8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellFlatMap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellFlatMap.java
@@ -83,7 +83,7 @@ public abstract class CellFlatMap implements NavigableMap<Cell,Cell> {
     int end = maxCellIdx - 1;
 
     while (begin <= end) {
-      int mid = (begin + end) >>> 1;
+      int mid = begin + ((end - begin) >> 1);
       Cell midCell = getCell(mid);
       int compareRes = comparator.compare(midCell, needle);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/3a3855aa/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BoundedPriorityBlockingQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BoundedPriorityBlockingQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BoundedPriorityBlockingQueue.java
index efad97e..3828606 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BoundedPriorityBlockingQueue.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BoundedPriorityBlockingQueue.java
@@ -119,7 +119,7 @@ public class BoundedPriorityBlockingQueue<E> extends AbstractQueue<E> implements
 
     private int upperBound(int start, int end, E key) {
       while (start < end) {
-        int mid = (start + end) >>> 1;
+        int mid = start + ((end - start) >> 1);
         E mitem = objects[mid];
         int cmp = comparator.compare(mitem, key);
         if (cmp > 0) {


[05/24] hbase git commit: HBASE-20401 Make MAX_WAIT and waitIfNotFinished in CleanerContext configurable (Contributed by Stephen Wu)

Posted by md...@apache.org.
HBASE-20401 Make MAX_WAIT and waitIfNotFinished in CleanerContext configurable (Contributed by Stephen Wu)


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

Branch: refs/heads/HBASE-20749
Commit: 4613f3e8c740e881505fcd2c661a0e28462eb459
Parents: b4759ce
Author: Reid Chan <re...@apache.org>
Authored: Mon Jul 23 10:29:38 2018 +0800
Committer: Reid Chan <re...@apache.org>
Committed: Mon Jul 23 10:30:01 2018 +0800

----------------------------------------------------------------------
 .../hbase/master/cleaner/HFileCleaner.java      | 65 ++++++++++++++++----
 .../hadoop/hbase/master/cleaner/LogCleaner.java | 65 +++++++++++++++-----
 .../hbase/master/cleaner/TestHFileCleaner.java  | 12 ++++
 .../hbase/master/cleaner/TestLogsCleaner.java   | 22 ++++++-
 src/main/asciidoc/_chapters/configuration.adoc  |  4 ++
 5 files changed, 139 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4613f3e8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
index a5e87ae..47b0228 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
@@ -23,6 +23,7 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.conf.Configuration;
@@ -76,6 +77,16 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
       "hbase.regionserver.hfilecleaner.small.thread.count";
   public final static int DEFAULT_SMALL_HFILE_DELETE_THREAD_NUMBER = 1;
 
+  public static final String HFILE_DELETE_THREAD_TIMEOUT_MSEC =
+      "hbase.regionserver.hfilecleaner.thread.timeout.msec";
+  @VisibleForTesting
+  static final long DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC = 60 * 1000L;
+
+  public static final String HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC =
+      "hbase.regionserver.hfilecleaner.thread.check.interval.msec";
+  @VisibleForTesting
+  static final long DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC = 1000L;
+
   private static final Logger LOG = LoggerFactory.getLogger(HFileCleaner.class);
 
   StealJobQueue<HFileDeleteTask> largeFileQueue;
@@ -85,6 +96,8 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
   private int smallQueueInitSize;
   private int largeFileDeleteThreadNumber;
   private int smallFileDeleteThreadNumber;
+  private long cleanerThreadTimeoutMsec;
+  private long cleanerThreadCheckIntervalMsec;
   private List<Thread> threads = new ArrayList<Thread>();
   private boolean running;
 
@@ -115,6 +128,11 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
         conf.getInt(LARGE_HFILE_DELETE_THREAD_NUMBER, DEFAULT_LARGE_HFILE_DELETE_THREAD_NUMBER);
     smallFileDeleteThreadNumber =
         conf.getInt(SMALL_HFILE_DELETE_THREAD_NUMBER, DEFAULT_SMALL_HFILE_DELETE_THREAD_NUMBER);
+    cleanerThreadTimeoutMsec =
+        conf.getLong(HFILE_DELETE_THREAD_TIMEOUT_MSEC, DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC);
+    cleanerThreadCheckIntervalMsec =
+        conf.getLong(HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC,
+            DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC);
     startHFileDeleteThreads();
   }
 
@@ -146,7 +164,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
     }
     // wait for each submitted task to finish
     for (HFileDeleteTask task : tasks) {
-      if (task.getResult()) {
+      if (task.getResult(cleanerThreadCheckIntervalMsec)) {
         deletedFiles++;
       }
     }
@@ -159,7 +177,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
    * @return HFileDeleteTask to track progress
    */
   private HFileDeleteTask deleteFile(FileStatus file) {
-    HFileDeleteTask task = new HFileDeleteTask(file);
+    HFileDeleteTask task = new HFileDeleteTask(file, cleanerThreadTimeoutMsec);
     boolean enqueued = dispatch(task);
     return enqueued ? task : null;
   }
@@ -300,17 +318,17 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
   };
 
   private static final class HFileDeleteTask {
-    private static final long MAX_WAIT = 60 * 1000L;
-    private static final long WAIT_UNIT = 1000L;
 
     boolean done = false;
     boolean result;
     final Path filePath;
     final long fileLength;
+    final long timeoutMsec;
 
-    public HFileDeleteTask(FileStatus file) {
+    public HFileDeleteTask(FileStatus file, long timeoutMsec) {
       this.filePath = file.getPath();
       this.fileLength = file.getLen();
+      this.timeoutMsec = timeoutMsec;
     }
 
     public synchronized void setResult(boolean result) {
@@ -319,17 +337,19 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
       notify();
     }
 
-    public synchronized boolean getResult() {
-      long waitTime = 0;
+    public synchronized boolean getResult(long waitIfNotFinished) {
+      long waitTimeMsec = 0;
       try {
         while (!done) {
-          wait(WAIT_UNIT);
-          waitTime += WAIT_UNIT;
+          long startTimeNanos = System.nanoTime();
+          wait(waitIfNotFinished);
+          waitTimeMsec += TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTimeNanos,
+              TimeUnit.NANOSECONDS);
           if (done) {
             return this.result;
           }
-          if (waitTime > MAX_WAIT) {
-            LOG.warn("Wait more than " + MAX_WAIT + " ms for deleting " + this.filePath
+          if (waitTimeMsec > timeoutMsec) {
+            LOG.warn("Wait more than " + timeoutMsec + " ms for deleting " + this.filePath
                 + ", exit...");
             return false;
           }
@@ -373,6 +393,16 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
     return throttlePoint;
   }
 
+  @VisibleForTesting
+  long getCleanerThreadTimeoutMsec() {
+    return cleanerThreadTimeoutMsec;
+  }
+
+  @VisibleForTesting
+  long getCleanerThreadCheckIntervalMsec() {
+    return cleanerThreadCheckIntervalMsec;
+  }
+
   @Override
   public void onConfigurationChange(Configuration conf) {
     super.onConfigurationChange(conf);
@@ -443,6 +473,19 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
       this.smallFileDeleteThreadNumber = smallFileDeleteThreadNumber;
       updated = true;
     }
+    long cleanerThreadTimeoutMsec =
+        conf.getLong(HFILE_DELETE_THREAD_TIMEOUT_MSEC, DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC);
+    if (cleanerThreadTimeoutMsec != this.cleanerThreadTimeoutMsec) {
+      this.cleanerThreadTimeoutMsec = cleanerThreadTimeoutMsec;
+      updated = true;
+    }
+    long cleanerThreadCheckIntervalMsec =
+        conf.getLong(HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC,
+            DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC);
+    if (cleanerThreadCheckIntervalMsec != this.cleanerThreadCheckIntervalMsec) {
+      this.cleanerThreadCheckIntervalMsec = cleanerThreadCheckIntervalMsec;
+      updated = true;
+    }
     return updated;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4613f3e8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
index bc27991..db098e2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -46,11 +47,24 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
   private static final Logger LOG = LoggerFactory.getLogger(LogCleaner.class.getName());
 
-  public static final String OLD_WALS_CLEANER_SIZE = "hbase.oldwals.cleaner.thread.size";
-  public static final int OLD_WALS_CLEANER_DEFAULT_SIZE = 2;
+  public static final String OLD_WALS_CLEANER_THREAD_SIZE = "hbase.oldwals.cleaner.thread.size";
+  public static final int DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE = 2;
+
+  public static final String OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC =
+      "hbase.oldwals.cleaner.thread.timeout.msec";
+  @VisibleForTesting
+  static final long DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC = 60 * 1000L;
+
+  public static final String OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC =
+      "hbase.oldwals.cleaner.thread.check.interval.msec";
+  @VisibleForTesting
+  static final long DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC = 500L;
+
 
   private final LinkedBlockingQueue<CleanerContext> pendingDelete;
   private List<Thread> oldWALsCleaner;
+  private long cleanerThreadTimeoutMsec;
+  private long cleanerThreadCheckIntervalMsec;
 
   /**
    * @param period the period of time to sleep between each run
@@ -63,8 +77,12 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
       Path oldLogDir) {
     super("LogsCleaner", period, stopper, conf, fs, oldLogDir, HBASE_MASTER_LOGCLEANER_PLUGINS);
     this.pendingDelete = new LinkedBlockingQueue<>();
-    int size = conf.getInt(OLD_WALS_CLEANER_SIZE, OLD_WALS_CLEANER_DEFAULT_SIZE);
+    int size = conf.getInt(OLD_WALS_CLEANER_THREAD_SIZE, DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE);
     this.oldWALsCleaner = createOldWalsCleaner(size);
+    this.cleanerThreadTimeoutMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC,
+        DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC);
+    this.cleanerThreadCheckIntervalMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC,
+        DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC);
   }
 
   @Override
@@ -77,7 +95,7 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
   public void onConfigurationChange(Configuration conf) {
     super.onConfigurationChange(conf);
 
-    int newSize = conf.getInt(OLD_WALS_CLEANER_SIZE, OLD_WALS_CLEANER_DEFAULT_SIZE);
+    int newSize = conf.getInt(OLD_WALS_CLEANER_THREAD_SIZE, DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE);
     if (newSize == oldWALsCleaner.size()) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Size from configuration is the same as previous which is " +
@@ -87,13 +105,18 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
     }
     interruptOldWALsCleaner();
     oldWALsCleaner = createOldWalsCleaner(newSize);
+    cleanerThreadTimeoutMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC,
+        DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC);
+    cleanerThreadCheckIntervalMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC,
+        DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC);
   }
 
   @Override
   protected int deleteFiles(Iterable<FileStatus> filesToDelete) {
     List<CleanerContext> results = new LinkedList<>();
     for (FileStatus toDelete : filesToDelete) {
-      CleanerContext context = CleanerContext.createCleanerContext(toDelete);
+      CleanerContext context = CleanerContext.createCleanerContext(toDelete,
+          cleanerThreadTimeoutMsec);
       if (context != null) {
         pendingDelete.add(context);
         results.add(context);
@@ -102,7 +125,7 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
 
     int deletedFiles = 0;
     for (CleanerContext res : results) {
-      deletedFiles += res.getResult(500) ? 1 : 0;
+      deletedFiles += res.getResult(cleanerThreadCheckIntervalMsec) ? 1 : 0;
     }
     return deletedFiles;
   }
@@ -118,6 +141,16 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
     return oldWALsCleaner.size();
   }
 
+  @VisibleForTesting
+  long getCleanerThreadTimeoutMsec() {
+    return cleanerThreadTimeoutMsec;
+  }
+
+  @VisibleForTesting
+  long getCleanerThreadCheckIntervalMsec() {
+    return cleanerThreadCheckIntervalMsec;
+  }
+
   private List<Thread> createOldWalsCleaner(int size) {
     LOG.info("Creating OldWALs cleaners with size=" + size);
 
@@ -186,20 +219,20 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
   }
 
   private static final class CleanerContext {
-    // At most waits 60 seconds
-    static final long MAX_WAIT = 60 * 1000;
 
     final FileStatus target;
     volatile boolean result;
     volatile boolean setFromCleaner = false;
+    long timeoutMsec;
 
-    static CleanerContext createCleanerContext(FileStatus status) {
-      return status != null ? new CleanerContext(status) : null;
+    static CleanerContext createCleanerContext(FileStatus status, long timeoutMsec) {
+      return status != null ? new CleanerContext(status, timeoutMsec) : null;
     }
 
-    private CleanerContext(FileStatus status) {
+    private CleanerContext(FileStatus status, long timeoutMsec) {
       this.target = status;
       this.result = false;
+      this.timeoutMsec = timeoutMsec;
     }
 
     synchronized void setResult(boolean res) {
@@ -209,13 +242,15 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
     }
 
     synchronized boolean getResult(long waitIfNotFinished) {
-      long totalTime = 0;
+      long totalTimeMsec = 0;
       try {
         while (!setFromCleaner) {
+          long startTimeNanos = System.nanoTime();
           wait(waitIfNotFinished);
-          totalTime += waitIfNotFinished;
-          if (totalTime >= MAX_WAIT) {
-            LOG.warn("Spend too much time to delete oldwals " + target);
+          totalTimeMsec += TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTimeNanos,
+              TimeUnit.NANOSECONDS);
+          if (totalTimeMsec >= timeoutMsec) {
+            LOG.warn("Spend too much time " + totalTimeMsec + " ms to delete oldwals " + target);
             return result;
           }
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4613f3e8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 465e193..9da4df4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -352,6 +352,8 @@ public class TestHFileCleaner {
     final int SMALL_FILE_NUM = 20;
     final int LARGE_THREAD_NUM = 2;
     final int SMALL_THREAD_NUM = 4;
+    final long THREAD_TIMEOUT_MSEC = 30 * 1000L;
+    final long THREAD_CHECK_INTERVAL_MSEC = 500L;
 
     Configuration conf = UTIL.getConfiguration();
     // no cleaner policies = delete all files
@@ -369,6 +371,10 @@ public class TestHFileCleaner {
     Assert.assertEquals(ORIGINAL_THROTTLE_POINT, cleaner.getThrottlePoint());
     Assert.assertEquals(ORIGINAL_QUEUE_INIT_SIZE, cleaner.getLargeQueueInitSize());
     Assert.assertEquals(ORIGINAL_QUEUE_INIT_SIZE, cleaner.getSmallQueueInitSize());
+    Assert.assertEquals(HFileCleaner.DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC,
+        cleaner.getCleanerThreadTimeoutMsec());
+    Assert.assertEquals(HFileCleaner.DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC,
+        cleaner.getCleanerThreadCheckIntervalMsec());
 
     // clean up archive directory and create files for testing
     fs.delete(archivedHfileDir, true);
@@ -396,6 +402,10 @@ public class TestHFileCleaner {
     newConf.setInt(HFileCleaner.SMALL_HFILE_QUEUE_INIT_SIZE, UPDATE_QUEUE_INIT_SIZE);
     newConf.setInt(HFileCleaner.LARGE_HFILE_DELETE_THREAD_NUMBER, LARGE_THREAD_NUM);
     newConf.setInt(HFileCleaner.SMALL_HFILE_DELETE_THREAD_NUMBER, SMALL_THREAD_NUM);
+    newConf.setLong(HFileCleaner.HFILE_DELETE_THREAD_TIMEOUT_MSEC, THREAD_TIMEOUT_MSEC);
+    newConf.setLong(HFileCleaner.HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC,
+        THREAD_CHECK_INTERVAL_MSEC);
+
     LOG.debug("File deleted from large queue: " + cleaner.getNumOfDeletedLargeFiles()
         + "; from small queue: " + cleaner.getNumOfDeletedSmallFiles());
     cleaner.onConfigurationChange(newConf);
@@ -405,6 +415,8 @@ public class TestHFileCleaner {
     Assert.assertEquals(UPDATE_QUEUE_INIT_SIZE, cleaner.getLargeQueueInitSize());
     Assert.assertEquals(UPDATE_QUEUE_INIT_SIZE, cleaner.getSmallQueueInitSize());
     Assert.assertEquals(LARGE_THREAD_NUM + SMALL_THREAD_NUM, cleaner.getCleanerThreads().size());
+    Assert.assertEquals(THREAD_TIMEOUT_MSEC, cleaner.getCleanerThreadTimeoutMsec());
+    Assert.assertEquals(THREAD_CHECK_INTERVAL_MSEC, cleaner.getCleanerThreadCheckIntervalMsec());
 
     // make sure no cost when onConfigurationChange called with no change
     List<Thread> oldThreads = cleaner.getCleanerThreads();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4613f3e8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 0263085..882ea9d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -279,14 +279,23 @@ public class TestLogsCleaner {
   @Test
   public void testOnConfigurationChange() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
-    conf.setInt(LogCleaner.OLD_WALS_CLEANER_SIZE, LogCleaner.OLD_WALS_CLEANER_DEFAULT_SIZE);
+    conf.setInt(LogCleaner.OLD_WALS_CLEANER_THREAD_SIZE,
+        LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE);
+    conf.setLong(LogCleaner.OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC,
+        LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC);
+    conf.setLong(LogCleaner.OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC,
+        LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC);
     // Prepare environments
     Server server = new DummyServer();
     Path oldWALsDir = new Path(TEST_UTIL.getDefaultRootDirPath(),
         HConstants.HREGION_OLDLOGDIR_NAME);
     FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
     LogCleaner cleaner = new LogCleaner(3000, server, conf, fs, oldWALsDir);
-    assertEquals(LogCleaner.OLD_WALS_CLEANER_DEFAULT_SIZE, cleaner.getSizeOfCleaners());
+    assertEquals(LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE, cleaner.getSizeOfCleaners());
+    assertEquals(LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC,
+        cleaner.getCleanerThreadTimeoutMsec());
+    assertEquals(LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC,
+        cleaner.getCleanerThreadCheckIntervalMsec());
     // Create dir and files for test
     fs.delete(oldWALsDir, true);
     fs.mkdirs(oldWALsDir);
@@ -300,9 +309,16 @@ public class TestLogsCleaner {
     thread.start();
     // change size of cleaners dynamically
     int sizeToChange = 4;
-    conf.setInt(LogCleaner.OLD_WALS_CLEANER_SIZE, sizeToChange);
+    long threadTimeoutToChange = 30 * 1000L;
+    long threadCheckIntervalToChange = 250L;
+    conf.setInt(LogCleaner.OLD_WALS_CLEANER_THREAD_SIZE, sizeToChange);
+    conf.setLong(LogCleaner.OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC, threadTimeoutToChange);
+    conf.setLong(LogCleaner.OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC,
+        threadCheckIntervalToChange);
     cleaner.onConfigurationChange(conf);
     assertEquals(sizeToChange, cleaner.getSizeOfCleaners());
+    assertEquals(threadTimeoutToChange, cleaner.getCleanerThreadTimeoutMsec());
+    assertEquals(threadCheckIntervalToChange, cleaner.getCleanerThreadCheckIntervalMsec());
     // Stop chore
     thread.join();
     status = fs.listStatus(oldWALsDir);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4613f3e8/src/main/asciidoc/_chapters/configuration.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/configuration.adoc b/src/main/asciidoc/_chapters/configuration.adoc
index 174aa80..113058c 100644
--- a/src/main/asciidoc/_chapters/configuration.adoc
+++ b/src/main/asciidoc/_chapters/configuration.adoc
@@ -1071,6 +1071,8 @@ Here are those configurations:
 | hbase.regionserver.hfilecleaner.small.queue.size
 | hbase.regionserver.hfilecleaner.large.thread.count
 | hbase.regionserver.hfilecleaner.small.thread.count
+| hbase.regionserver.hfilecleaner.thread.timeout.msec
+| hbase.regionserver.hfilecleaner.thread.check.interval.msec
 | hbase.regionserver.flush.throughput.controller
 | hbase.hstore.compaction.max.size
 | hbase.hstore.compaction.max.size.offpeak
@@ -1091,6 +1093,8 @@ Here are those configurations:
 | hbase.offpeak.start.hour
 | hbase.offpeak.end.hour
 | hbase.oldwals.cleaner.thread.size
+| hbase.oldwals.cleaner.thread.timeout.msec
+| hbase.oldwals.cleaner.thread.check.interval.msec
 | hbase.procedure.worker.keep.alive.time.msec
 | hbase.procedure.worker.add.stuck.percentage
 | hbase.procedure.worker.monitor.interval.msec


[19/24] hbase git commit: HBASE-20927 RSGroupAdminEndpoint doesn't handle clearing dead servers if they are not processed yet.

Posted by md...@apache.org.
HBASE-20927 RSGroupAdminEndpoint doesn't handle clearing dead servers if they are not processed yet.

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-20749
Commit: 973b4ddcfa174f06470ebfe02977f41fb8a02a6e
Parents: 8b8de1f
Author: Sergey Soldatov <ss...@apache.org>
Authored: Wed Jul 25 23:32:36 2018 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Jul 26 09:58:49 2018 -0700

----------------------------------------------------------------------
 .../hbase/rsgroup/RSGroupAdminEndpoint.java     |  4 ++-
 .../hadoop/hbase/rsgroup/TestRSGroupsBase.java  | 28 ++++++++++++++++++++
 2 files changed, 31 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/973b4ddc/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index b67e335..3d1f780 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -539,7 +539,9 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
         filter(server -> !notClearedServers.contains(server)).
         map(ServerName::getAddress).
         collect(Collectors.toSet());
-    groupAdminServer.removeServers(clearedServer);
+    if(!clearedServer.isEmpty()) {
+      groupAdminServer.removeServers(clearedServer);
+    }
   }
 
   public void checkPermission(String request) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/973b4ddc/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
index 199dd98..43099db 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
@@ -1121,4 +1121,32 @@ public abstract class TestRSGroupsBase {
       }
     });
   }
+  @Test
+  public void testClearNotProcessedDeadServer() throws Exception {
+    LOG.info("testClearNotProcessedDeadServer");
+    NUM_DEAD_SERVERS = cluster.getClusterMetrics().getDeadServerNames().size();
+    RSGroupInfo appInfo = addGroup("deadServerGroup", 1);
+    ServerName targetServer =
+        ServerName.parseServerName(appInfo.getServers().iterator().next().toString());
+    AdminProtos.AdminService.BlockingInterface targetRS =
+        ((ClusterConnection) admin.getConnection()).getAdmin(targetServer);
+    try {
+      targetServer = ProtobufUtil.toServerName(targetRS.getServerInfo(null,
+          AdminProtos.GetServerInfoRequest.newBuilder().build()).getServerInfo().getServerName());
+      //stopping may cause an exception
+      //due to the connection loss
+      targetRS.stopServer(null,
+          AdminProtos.StopServerRequest.newBuilder().setReason("Die").build());
+      NUM_DEAD_SERVERS ++;
+    } catch(Exception e) {
+    }
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return cluster.getClusterMetrics().getDeadServerNames().size() == NUM_DEAD_SERVERS;
+      }
+    });
+    List<ServerName> notClearedServers = admin.clearDeadServers(Lists.newArrayList(targetServer));
+    assertEquals(1, notClearedServers.size());
+  }
 }