HBASE-20642 Clients should re-use the same nonce across DDL operations

Also changes modify table operations to help the case where a MTP spans
two master, avoiding the sanity-checks propagating back to the client
unnecessarily.

Signed-off-by: Josh Elser <els...@apache.org>
Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-19064
Commit: 72784c2d836a4b977667449d3adec5e8d15453f5
Parents: e989a99
Author: Ankit Singhal <ankitsingha...@gmail.com>
Authored: Wed Jun 20 14:52:26 2018 -0700
Committer: Josh Elser <els...@apache.org>
Committed: Wed Jun 20 14:56:10 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  53 ++++--
 .../org/apache/hadoop/hbase/master/HMaster.java | 168 ++++++++++---------
 .../MasterProcedureTestingUtility.java          |  24 ++-
 .../procedure/TestModifyTableProcedure.java     |  71 ++++++++
 4 files changed, 224 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/72784c2d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 172db5b..07e1be4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -348,11 +348,13 @@ public class HBaseAdmin implements Admin {
   public Future<Void> modifyTableAsync(TableDescriptor td) throws IOException {
     ModifyTableResponse response = executeCallable(
       new MasterCallable<ModifyTableResponse>(getConnection(), 
getRpcControllerFactory()) {
+        Long nonceGroup = ng.getNonceGroup();
+        Long nonce = ng.newNonce();
         @Override
         protected ModifyTableResponse rpcCall() throws Exception {
           setPriority(td.getTableName());
           ModifyTableRequest request = 
RequestConverter.buildModifyTableRequest(
-            td.getTableName(), td, ng.getNonceGroup(), ng.newNonce());
+            td.getTableName(), td, nonceGroup, nonce);
           return master.modifyTable(getRpcController(), request);
         }
       });
@@ -644,11 +646,13 @@ public class HBaseAdmin implements Admin {
 
     CreateTableResponse response = executeCallable(
       new MasterCallable<CreateTableResponse>(getConnection(), 
getRpcControllerFactory()) {
+        Long nonceGroup = ng.getNonceGroup();
+        Long nonce = ng.newNonce();
         @Override
         protected CreateTableResponse rpcCall() throws Exception {
           setPriority(desc.getTableName());
           CreateTableRequest request = 
RequestConverter.buildCreateTableRequest(
-            desc, splitKeys, ng.getNonceGroup(), ng.newNonce());
+            desc, splitKeys, nonceGroup, nonce);
           return master.createTable(getRpcController(), request);
         }
       });
@@ -694,11 +698,13 @@ public class HBaseAdmin implements Admin {
   public Future<Void> deleteTableAsync(final TableName tableName) throws 
IOException {
     DeleteTableResponse response = executeCallable(
       new MasterCallable<DeleteTableResponse>(getConnection(), 
getRpcControllerFactory()) {
+        Long nonceGroup = ng.getNonceGroup();
+        Long nonce = ng.newNonce();
         @Override
         protected DeleteTableResponse rpcCall() throws Exception {
           setPriority(tableName);
           DeleteTableRequest req =
-              RequestConverter.buildDeleteTableRequest(tableName, 
ng.getNonceGroup(),ng.newNonce());
+              RequestConverter.buildDeleteTableRequest(tableName, 
nonceGroup,nonce);
           return master.deleteTable(getRpcController(), req);
         }
       });
@@ -775,12 +781,14 @@ public class HBaseAdmin implements Admin {
     TruncateTableResponse response =
         executeCallable(new 
MasterCallable<TruncateTableResponse>(getConnection(),
             getRpcControllerFactory()) {
+          Long nonceGroup = ng.getNonceGroup();
+          Long nonce = ng.newNonce();
           @Override
           protected TruncateTableResponse rpcCall() throws Exception {
             setPriority(tableName);
             LOG.info("Started truncating " + tableName);
             TruncateTableRequest req = 
RequestConverter.buildTruncateTableRequest(
-              tableName, preserveSplits, ng.getNonceGroup(), ng.newNonce());
+              tableName, preserveSplits, nonceGroup, nonce);
             return master.truncateTable(getRpcController(), req);
           }
         });
@@ -838,12 +846,14 @@ public class HBaseAdmin implements Admin {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     EnableTableResponse response = executeCallable(
       new MasterCallable<EnableTableResponse>(getConnection(), 
getRpcControllerFactory()) {
+        Long nonceGroup = ng.getNonceGroup();
+        Long nonce = ng.newNonce();
         @Override
         protected EnableTableResponse rpcCall() throws Exception {
           setPriority(tableName);
           LOG.info("Started enable of " + tableName);
           EnableTableRequest req =
-              RequestConverter.buildEnableTableRequest(tableName, 
ng.getNonceGroup(),ng.newNonce());
+              RequestConverter.buildEnableTableRequest(tableName, nonceGroup, 
nonce);
           return master.enableTable(getRpcController(),req);
         }
       });
@@ -901,13 +911,15 @@ public class HBaseAdmin implements Admin {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     DisableTableResponse response = executeCallable(
       new MasterCallable<DisableTableResponse>(getConnection(), 
getRpcControllerFactory()) {
+        Long nonceGroup = ng.getNonceGroup();
+        Long nonce = ng.newNonce();
         @Override
         protected DisableTableResponse rpcCall() throws Exception {
           setPriority(tableName);
           LOG.info("Started disable of " + tableName);
           DisableTableRequest req =
               RequestConverter.buildDisableTableRequest(
-                tableName, ng.getNonceGroup(), ng.newNonce());
+                tableName, nonceGroup, nonce);
           return master.disableTable(getRpcController(), req);
         }
       });
@@ -1019,12 +1031,13 @@ public class HBaseAdmin implements Admin {
     AddColumnResponse response =
         executeCallable(new MasterCallable<AddColumnResponse>(getConnection(),
             getRpcControllerFactory()) {
+          Long nonceGroup = ng.getNonceGroup();
+          Long nonce = ng.newNonce();
           @Override
           protected AddColumnResponse rpcCall() throws Exception {
             setPriority(tableName);
             AddColumnRequest req =
-                RequestConverter.buildAddColumnRequest(tableName, 
columnFamily, ng.getNonceGroup(),
-                  ng.newNonce());
+                RequestConverter.buildAddColumnRequest(tableName, 
columnFamily, nonceGroup, nonce);
             return master.addColumn(getRpcController(), req);
           }
         });
@@ -1068,12 +1081,14 @@ public class HBaseAdmin implements Admin {
     DeleteColumnResponse response =
         executeCallable(new 
MasterCallable<DeleteColumnResponse>(getConnection(),
             getRpcControllerFactory()) {
+          Long nonceGroup = ng.getNonceGroup();
+          Long nonce = ng.newNonce();
           @Override
           protected DeleteColumnResponse rpcCall() throws Exception {
             setPriority(tableName);
             DeleteColumnRequest req =
                 RequestConverter.buildDeleteColumnRequest(tableName, 
columnFamily,
-                  ng.getNonceGroup(), ng.newNonce());
+                  nonceGroup, nonce);
             return master.deleteColumn(getRpcController(), req);
           }
         });
@@ -1105,12 +1120,14 @@ public class HBaseAdmin implements Admin {
     ModifyColumnResponse response =
         executeCallable(new 
MasterCallable<ModifyColumnResponse>(getConnection(),
             getRpcControllerFactory()) {
+          Long nonceGroup = ng.getNonceGroup();
+          Long nonce = ng.newNonce();
           @Override
           protected ModifyColumnResponse rpcCall() throws Exception {
             setPriority(tableName);
             ModifyColumnRequest req =
                 RequestConverter.buildModifyColumnRequest(tableName, 
columnFamily,
-                  ng.getNonceGroup(), ng.newNonce());
+                  nonceGroup, nonce);
             return master.modifyColumn(getRpcController(), req);
           }
         });
@@ -1719,14 +1736,16 @@ public class HBaseAdmin implements Admin {
     MergeTableRegionsResponse response =
         executeCallable(new 
MasterCallable<MergeTableRegionsResponse>(getConnection(),
             getRpcControllerFactory()) {
+          Long nonceGroup = ng.getNonceGroup();
+          Long nonce = ng.newNonce();
       @Override
       protected MergeTableRegionsResponse rpcCall() throws Exception {
         MergeTableRegionsRequest request = RequestConverter
             .buildMergeTableRegionsRequest(
                 encodedNameofRegionsToMerge,
                 forcible,
-                ng.getNonceGroup(),
-                ng.newNonce());
+                nonceGroup,
+                nonce);
         return master.mergeTableRegions(getRpcController(), request);
       }
     });
@@ -1813,11 +1832,13 @@ public class HBaseAdmin implements Admin {
 
     SplitTableRegionResponse response = executeCallable(
         new MasterCallable<SplitTableRegionResponse>(getConnection(), 
getRpcControllerFactory()) {
+          Long nonceGroup = ng.getNonceGroup();
+          Long nonce = ng.newNonce();
           @Override
           protected SplitTableRegionResponse rpcCall() throws Exception {
             setPriority(tableName);
             SplitTableRegionRequest request = RequestConverter
-                .buildSplitTableRegionRequest(hri, splitPoint, 
ng.getNonceGroup(), ng.newNonce());
+                .buildSplitTableRegionRequest(hri, splitPoint, nonceGroup, 
nonce);
             return master.splitRegion(getRpcController(), request);
           }
         });
@@ -2847,12 +2868,14 @@ public class HBaseAdmin implements Admin {
 
     RestoreSnapshotResponse response = executeCallable(
         new MasterCallable<RestoreSnapshotResponse>(getConnection(), 
getRpcControllerFactory()) {
+          Long nonceGroup = ng.getNonceGroup();
+          Long nonce = ng.newNonce();
       @Override
       protected RestoreSnapshotResponse rpcCall() throws Exception {
         final RestoreSnapshotRequest request = 
RestoreSnapshotRequest.newBuilder()
             .setSnapshot(snapshot)
-            .setNonceGroup(ng.getNonceGroup())
-            .setNonce(ng.newNonce())
+            .setNonceGroup(nonceGroup)
+            .setNonce(nonce)
             .setRestoreACL(restoreAcl)
             .build();
         return master.restoreSnapshot(getRpcController(), request);

http://git-wip-us.apache.org/repos/asf/hbase/blob/72784c2d/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 d6b793a..530ed17 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
@@ -2241,74 +2241,76 @@ public class HMaster extends HRegionServer implements 
MasterServices {
   }
 
   @Override
-  public long addColumn(
-      final TableName tableName,
-      final ColumnFamilyDescriptor column,
-      final long nonceGroup,
-      final long nonce)
-      throws IOException {
+  public long addColumn(final TableName tableName, final 
ColumnFamilyDescriptor column,
+      final long nonceGroup, final long nonce) throws IOException {
     checkInitialized();
     checkTableExists(tableName);
 
-    TableDescriptor old = getTableDescriptors().get(tableName);
-    if (old.hasColumnFamily(column.getName())) {
-      throw new InvalidFamilyOperationException("Column family '" + 
column.getNameAsString()
-          + "' in table '" + tableName + "' already exists so cannot be 
added");
-    }
+    return modifyTable(tableName, new TableDescriptorGetter() {
+
+      @Override
+      public TableDescriptor get() throws IOException {
+        TableDescriptor old = getTableDescriptors().get(tableName);
+        if (old.hasColumnFamily(column.getName())) {
+          throw new InvalidFamilyOperationException("Column family '" + 
column.getNameAsString()
+              + "' in table '" + tableName + "' already exists so cannot be 
added");
+        }
 
-    TableDescriptor newDesc = TableDescriptorBuilder
-        .newBuilder(old).setColumnFamily(column).build();
-    return modifyTable(tableName, newDesc, nonceGroup, nonce);
+        return 
TableDescriptorBuilder.newBuilder(old).setColumnFamily(column).build();
+      }
+    }, nonceGroup, nonce);
+  }
+
+  /**
+   * Implement to return TableDescriptor after pre-checks
+   */
+  protected interface TableDescriptorGetter {
+    TableDescriptor get() throws IOException;
   }
 
   @Override
-  public long modifyColumn(
-      final TableName tableName,
-      final ColumnFamilyDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce)
-      throws IOException {
+  public long modifyColumn(final TableName tableName, final 
ColumnFamilyDescriptor descriptor,
+      final long nonceGroup, final long nonce) throws IOException {
     checkInitialized();
     checkTableExists(tableName);
+    return modifyTable(tableName, new TableDescriptorGetter() {
 
-    TableDescriptor old = getTableDescriptors().get(tableName);
-    if (! old.hasColumnFamily(descriptor.getName())) {
-      throw new InvalidFamilyOperationException("Family '" + 
descriptor.getNameAsString()
-          + "' does not exist, so it cannot be modified");
-    }
-
-    TableDescriptor td = TableDescriptorBuilder
-        .newBuilder(old)
-        .modifyColumnFamily(descriptor)
-        .build();
+      @Override
+      public TableDescriptor get() throws IOException {
+        TableDescriptor old = getTableDescriptors().get(tableName);
+        if (!old.hasColumnFamily(descriptor.getName())) {
+          throw new InvalidFamilyOperationException("Family '" + 
descriptor.getNameAsString()
+              + "' does not exist, so it cannot be modified");
+        }
 
-    return modifyTable(tableName, td, nonceGroup, nonce);
+        return 
TableDescriptorBuilder.newBuilder(old).modifyColumnFamily(descriptor).build();
+      }
+    }, nonceGroup, nonce);
   }
 
   @Override
-  public long deleteColumn(
-      final TableName tableName,
-      final byte[] columnName,
-      final long nonceGroup,
-      final long nonce)
-      throws IOException {
+  public long deleteColumn(final TableName tableName, final byte[] columnName,
+      final long nonceGroup, final long nonce) throws IOException {
     checkInitialized();
     checkTableExists(tableName);
 
-    TableDescriptor old = getTableDescriptors().get(tableName);
+    return modifyTable(tableName, new TableDescriptorGetter() {
 
-    if (! old.hasColumnFamily(columnName)) {
-      throw new InvalidFamilyOperationException("Family '" + 
Bytes.toString(columnName)
-          + "' does not exist, so it cannot be deleted");
-    }
-    if (old.getColumnFamilyCount() == 1) {
-      throw new InvalidFamilyOperationException("Family '" + 
Bytes.toString(columnName)
-          + "' is the only column family in the table, so it cannot be 
deleted");
-    }
+      @Override
+      public TableDescriptor get() throws IOException {
+        TableDescriptor old = getTableDescriptors().get(tableName);
 
-    TableDescriptor td = TableDescriptorBuilder
-        .newBuilder(old).removeColumnFamily(columnName).build();
-    return modifyTable(tableName, td, nonceGroup, nonce);
+        if (!old.hasColumnFamily(columnName)) {
+          throw new InvalidFamilyOperationException("Family '" + 
Bytes.toString(columnName)
+              + "' does not exist, so it cannot be deleted");
+        }
+        if (old.getColumnFamilyCount() == 1) {
+          throw new InvalidFamilyOperationException("Family '" + 
Bytes.toString(columnName)
+              + "' is the only column family in the table, so it cannot be 
deleted");
+        }
+        return 
TableDescriptorBuilder.newBuilder(old).removeColumnFamily(columnName).build();
+      }
+    }, nonceGroup, nonce);
   }
 
   @Override
@@ -2433,40 +2435,54 @@ public class HMaster extends HRegionServer implements 
MasterServices {
     return result.get();
   }
 
+  private long modifyTable(final TableName tableName,
+      final TableDescriptorGetter newDescriptorGetter, final long nonceGroup, 
final long nonce)
+      throws IOException {
+    return MasterProcedureUtil
+        .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, 
nonceGroup, nonce) {
+          @Override
+          protected void run() throws IOException {
+            TableDescriptor newDescriptor = newDescriptorGetter.get();
+            sanityCheckTableDescriptor(newDescriptor);
+            TableDescriptor oldDescriptor = 
getMaster().getTableDescriptors().get(tableName);
+            getMaster().getMasterCoprocessorHost().preModifyTable(tableName, 
oldDescriptor,
+              newDescriptor);
+
+            LOG.info(getClientIdAuditPrefix() + " modify " + tableName);
+
+            // Execute the operation synchronously - wait for the operation 
completes before
+            // continuing.
+            //
+            // We need to wait for the procedure to potentially fail due to 
"prepare" sanity
+            // checks. This will block only the beginning of the procedure. 
See HBASE-19953.
+            ProcedurePrepareLatch latch = 
ProcedurePrepareLatch.createBlockingLatch();
+            submitProcedure(
+              new ModifyTableProcedure(procedureExecutor.getEnvironment(), 
newDescriptor, latch));
+            latch.await();
+
+            getMaster().getMasterCoprocessorHost().postModifyTable(tableName, 
oldDescriptor,
+              newDescriptor);
+          }
+
+          @Override
+          protected String getDescription() {
+            return "ModifyTableProcedure";
+          }
+        });
+
+  }
+
   @Override
   public long modifyTable(final TableName tableName, final TableDescriptor 
newDescriptor,
       final long nonceGroup, final long nonce) throws IOException {
     checkInitialized();
-    sanityCheckTableDescriptor(newDescriptor);
-
-    return MasterProcedureUtil.submitProcedure(
-        new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, 
nonce) {
+    return modifyTable(tableName, new TableDescriptorGetter() {
       @Override
-      protected void run() throws IOException {
-        TableDescriptor oldDescriptor = 
getMaster().getTableDescriptors().get(tableName);
-        getMaster().getMasterCoprocessorHost()
-          .preModifyTable(tableName, oldDescriptor, newDescriptor);
-
-        LOG.info(getClientIdAuditPrefix() + " modify " + tableName);
-
-        // Execute the operation synchronously - wait for the operation 
completes before continuing.
-        //
-        // We need to wait for the procedure to potentially fail due to 
"prepare" sanity
-        // checks. This will block only the beginning of the procedure. See 
HBASE-19953.
-        ProcedurePrepareLatch latch = 
ProcedurePrepareLatch.createBlockingLatch();
-        submitProcedure(
-          new ModifyTableProcedure(procedureExecutor.getEnvironment(), 
newDescriptor, latch));
-        latch.await();
-
-        getMaster().getMasterCoprocessorHost()
-          .postModifyTable(tableName, oldDescriptor, newDescriptor);
+      public TableDescriptor get() throws IOException {
+        return newDescriptor;
       }
+    }, nonceGroup, nonce);
 
-      @Override
-      protected String getDescription() {
-        return "ModifyTableProcedure";
-      }
-    });
   }
 
   public long restoreSnapshot(final SnapshotDescription snapshotDesc,

http://git-wip-us.apache.org/repos/asf/hbase/blob/72784c2d/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index 1c1a36e..26e0956 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -405,6 +405,7 @@ public class MasterProcedureTestingUtility {
    *<p>It does
    * <ol><li>Execute step N - kill the executor before store update
    * <li>Restart executor/store
+   * <li>Executes hook for each step twice
    * <li>Execute step N - and then save to store
    * </ol>
    *
@@ -415,11 +416,15 @@ public class MasterProcedureTestingUtility {
    * @see #testRecoveryAndDoubleExecution(ProcedureExecutor, long, int, 
boolean)
    */
   public static void testRecoveryAndDoubleExecution(
-      final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId) 
throws Exception {
+      final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId, 
final StepHook hook)
+      throws Exception {
     ProcedureTestingUtility.waitProcedure(procExec, procId);
     assertEquals(false, procExec.isRunning());
     for (int i = 0; !procExec.isFinished(procId); ++i) {
       LOG.info("Restart " + i + " exec state=" + 
procExec.getProcedure(procId));
+      if (hook != null) {
+        assertTrue(hook.execute(i));
+      }
       restartMasterProcedureExecutor(procExec);
       ProcedureTestingUtility.waitProcedure(procExec, procId);
     }
@@ -427,6 +432,23 @@ public class MasterProcedureTestingUtility {
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
   }
 
+  public static void testRecoveryAndDoubleExecution(
+      final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId) 
throws Exception {
+    testRecoveryAndDoubleExecution(procExec, procId, null);
+  }
+
+  /**
+   * Hook which will be executed on each step
+   */
+  public interface StepHook{
+    /**
+     * @param step Step no. at which this will be executed
+     * @return false if test should fail otherwise true
+     * @throws IOException
+     */
+    boolean execute(int step) throws IOException;
+  }
+
   /**
    * Execute the procedure up to "lastStep" and then the ProcedureExecutor
    * is restarted and an abort() is injected.

http://git-wip-us.apache.org/repos/asf/hbase/blob/72784c2d/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
index 51d7d2b..9428ff0 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
@@ -21,21 +21,28 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
+
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.PerClientRandomNonceGenerator;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import 
org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility.StepHook;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 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.NonceKey;
+import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -269,6 +276,70 @@ public class TestModifyTableProcedure extends 
TestTableDDLProcedureBase {
   }
 
   @Test
+  public void testColumnFamilyAdditionTwiceWithNonce() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final String cf2 = "cf2";
+    final String cf3 = "cf3";
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+
+    // create the table
+    RegionInfo[] regions =
+        MasterProcedureTestingUtility.createTable(procExec, tableName, null, 
"cf1", cf3);
+
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+    // Modify multiple properties of the table.
+    final HTableDescriptor htd =
+        new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+    boolean newCompactionEnableOption = htd.isCompactionEnabled() ? false : 
true;
+    htd.setCompactionEnabled(newCompactionEnableOption);
+    htd.addFamily(new HColumnDescriptor(cf2));
+
+    PerClientRandomNonceGenerator nonceGenerator = 
PerClientRandomNonceGenerator.get();
+    long nonceGroup = nonceGenerator.getNonceGroup();
+    long newNonce = nonceGenerator.newNonce();
+    NonceKey nonceKey = new NonceKey(nonceGroup, newNonce);
+    procExec.registerNonce(nonceKey);
+
+    // Start the Modify procedure && kill the executor
+    final long procId = procExec
+        .submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), 
htd), nonceKey);
+
+    // Restart the executor after MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR and try 
to add column family
+    // as nonce are there , we should not fail
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, 
procId, new StepHook() {
+      @Override
+      public boolean execute(int step) throws IOException {
+        if (step == 3) {
+          return procId == 
UTIL.getHBaseCluster().getMaster().addColumn(tableName,
+            
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(cf2)).build(), 
nonceGroup,
+            newNonce);
+        }
+        return true;
+      }
+    });
+
+    //Try with different nonce, now it should fail the checks
+    try {
+      UTIL.getHBaseCluster().getMaster().addColumn(tableName,
+        ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(cf2)).build(), 
nonceGroup,
+        nonceGenerator.newNonce());
+      Assert.fail();
+    } catch (InvalidFamilyOperationException e) {
+    }
+
+    // Validate descriptor
+    HTableDescriptor currentHtd = 
UTIL.getAdmin().getTableDescriptor(tableName);
+    assertEquals(newCompactionEnableOption, currentHtd.isCompactionEnabled());
+    assertEquals(3, currentHtd.getFamiliesKeys().size());
+    assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf2)));
+    assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf3)));
+
+    // cf2 should be added
+    
MasterProcedureTestingUtility.validateTableCreation(UTIL.getHBaseCluster().getMaster(),
+      tableName, regions, "cf1", cf2, cf3);
+  }
+
+  @Test
   public void testRollbackAndDoubleExecutionOnline() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final String familyName = "cf2";

Reply via email to