View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase;
20  
21  import java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.List;
24  import java.util.Set;
25  import java.util.concurrent.ConcurrentHashMap;
26  import java.util.concurrent.atomic.AtomicBoolean;
27  
28  import org.apache.commons.lang.RandomStringUtils;
29  import org.apache.commons.lang.math.RandomUtils;
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.hbase.client.Admin;
34  import org.apache.hadoop.hbase.client.Connection;
35  import org.apache.hadoop.hbase.client.ConnectionFactory;
36  import org.apache.hadoop.hbase.client.Put;
37  import org.apache.hadoop.hbase.client.Table;
38  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
39  import org.apache.hadoop.hbase.testclassification.IntegrationTests;
40  import org.apache.hadoop.hbase.util.Bytes;
41  import org.apache.hadoop.hbase.util.HBaseFsck;
42  import org.apache.hadoop.hbase.util.Threads;
43  import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
44  import org.apache.hadoop.util.ToolRunner;
45  import org.junit.Assert;
46  import org.junit.Test;
47  import org.junit.experimental.categories.Category;
48  
49  /**
50   *
51   * Integration test that verifies Procedure V2. <br/><br/>
52   *
53   * DDL operations should go through (rollforward or rollback) when primary master is killed by
54   * ChaosMonkey (default MASTER_KILLING)<br/><br/>
55   *
56   * Multiple Worker threads are started to randomly do the following Actions in loops:<br/>
57   * Actions generating and populating tables:
58   * <ul>
59   *     <li>CreateTableAction</li>
60   *     <li>DisableTableAction</li>
61   *     <li>EnableTableAction</li>
62   *     <li>DeleteTableAction</li>
63   *     <li>AddRowAction</li>
64   * </ul>
65   * Actions performing DDL operations:
66   * <ul>
67   *     <li>AddColumnFamilyAction</li>
68   *     <li>AlterColumnFamilyVersionsAction</li>
69   *     <li>AlterColumnFamilyEncodingAction</li>
70   *     <li>DeleteColumnFamilyAction</li>
71   * </ul>
72   * <br/>
73   *
74   * The threads run for a period of time (default 20 minutes) then are stopped at the end of
75   * runtime. Verification is performed towards those checkpoints:
76   * <ol>
77   *     <li>No Actions throw Exceptions.</li>
78   *     <li>No inconsistencies are detected in hbck.</li>
79   * </ol>
80   *
81   * <p>
82   * This test should be run by the hbase user since it invokes hbck at the end
83   * </p><p>
84   * Usage:
85   *  hbase org.apache.hadoop.hbase.IntegrationTestDDLMasterFailover
86   *    -Dhbase.IntegrationTestDDLMasterFailover.runtime=1200000
87   *    -Dhbase.IntegrationTestDDLMasterFailover.numThreads=20
88   *    -Dhbase.IntegrationTestDDLMasterFailover.numRegions=50 --monkey masterKilling
89   */
90  
91  @Category(IntegrationTests.class)
92  public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
93  
94    private static final Log LOG = LogFactory.getLog(IntegrationTestDDLMasterFailover.class);
95  
96    private static final int SERVER_COUNT = 1; // number of slaves for the smallest cluster
97  
98    protected static final long DEFAULT_RUN_TIME = 20 * 60 * 1000;
99  
100   protected static final int DEFAULT_NUM_THREADS = 20;
101 
102   protected static final int DEFAULT_NUM_REGIONS = 50; // number of regions in pre-split tables
103 
104   protected HBaseCluster cluster;
105 
106   protected Connection connection;
107 
108   /**
109    * A soft limit on how long we should run
110    */
111   protected static final String RUN_TIME_KEY = "hbase.%s.runtime";
112   protected static final String NUM_THREADS_KEY = "hbase.%s.numThreads";
113   protected static final String NUM_REGIONS_KEY = "hbase.%s.numRegions";
114 
115   protected AtomicBoolean running = new AtomicBoolean(true);
116 
117   protected AtomicBoolean create_table = new AtomicBoolean(true);
118 
119   protected int numThreads, numRegions;
120 
121   ConcurrentHashMap<TableName, HTableDescriptor> enabledTables =
122       new ConcurrentHashMap<TableName, HTableDescriptor>();
123 
124   ConcurrentHashMap<TableName, HTableDescriptor> disabledTables =
125       new ConcurrentHashMap<TableName, HTableDescriptor>();
126 
127   ConcurrentHashMap<TableName, HTableDescriptor> deletedTables =
128       new ConcurrentHashMap<TableName, HTableDescriptor>();
129 
130   @Override
131   public void setUpCluster() throws Exception {
132     util = getTestingUtil(getConf());
133     LOG.debug("Initializing/checking cluster has " + SERVER_COUNT + " servers");
134     util.initializeCluster(getMinServerCount());
135     LOG.debug("Done initializing/checking cluster");
136     cluster = util.getHBaseClusterInterface();
137   }
138 
139   @Override
140   public void cleanUpCluster() throws Exception {
141     Admin admin = util.getHBaseAdmin();
142     admin.disableTables("ittable-\\d+");
143     admin.deleteTables("ittable-\\d+");
144     Connection connection = getConnection();
145     connection.close();
146     super.cleanUpCluster();
147   }
148 
149   protected int getMinServerCount() {
150     return SERVER_COUNT;
151   }
152 
153   protected synchronized void setConnection(Connection connection){
154     this.connection = connection;
155   }
156 
157   protected synchronized Connection getConnection(){
158     if (this.connection == null) {
159       try {
160         Connection connection = ConnectionFactory.createConnection(getConf());
161         setConnection(connection);
162       } catch (IOException e) {
163         LOG.fatal("Failed to establish connection.", e);
164       }
165     }
166     return connection;
167   }
168 
169   protected void verifyTables() throws  IOException{
170     Connection connection = getConnection();
171     Admin admin = connection.getAdmin();
172     // iterating concurrent map
173     for (TableName tableName : enabledTables.keySet()){
174       Assert.assertTrue("Table: " + tableName + " in enabledTables is not enabled",
175           admin.isTableEnabled(tableName));
176     }
177     for (TableName tableName : disabledTables.keySet()){
178       Assert.assertTrue("Table: " + tableName + " in disabledTables is not disabled",
179           admin.isTableDisabled(tableName));
180     }
181     for (TableName tableName : deletedTables.keySet()){
182       Assert.assertFalse("Table: " + tableName + " in deletedTables is not deleted",
183           admin.tableExists(tableName));
184     }
185     admin.close();
186   }
187 
188   @Test
189   public void testAsUnitTest() throws Exception {
190     runTest();
191   }
192 
193   @Override
194   public int runTestFromCommandLine() throws Exception {
195     int ret = runTest();
196     return ret;
197   }
198 
199   private abstract class MasterAction{
200     Connection connection = getConnection();
201 
202     abstract void perform() throws IOException;
203   }
204 
205   private abstract class TableAction extends  MasterAction{
206     // TableAction has implemented selectTable() shared by multiple table Actions
207     protected HTableDescriptor selectTable(ConcurrentHashMap<TableName, HTableDescriptor> tableMap)
208     {
209       // synchronization to prevent removal from multiple threads
210       synchronized (tableMap){
211         // randomly select table from tableMap
212         if (tableMap.isEmpty()) {
213           return null;
214         }
215         ArrayList<TableName> tableList = new ArrayList<TableName>(tableMap.keySet());
216         TableName randomKey = tableList.get(RandomUtils.nextInt(tableList.size()));
217         HTableDescriptor randomHtd = tableMap.get(randomKey);
218         // remove from tableMap
219         tableMap.remove(randomKey);
220         return randomHtd;
221       }
222     }
223   }
224 
225   private class CreateTableAction extends TableAction {
226 
227     @Override
228     void perform() throws IOException {
229       Admin admin = connection.getAdmin();
230       try {
231         HTableDescriptor htd = createTableDesc();
232         TableName tableName = htd.getTableName();
233         if ( admin.tableExists(tableName)){
234           return;
235         }
236         String numRegionKey = String.format(NUM_REGIONS_KEY, this.getClass().getSimpleName());
237         numRegions = getConf().getInt(numRegionKey, DEFAULT_NUM_REGIONS);
238         byte[] startKey = Bytes.toBytes("row-0000000000");
239         byte[] endKey = Bytes.toBytes("row-" + Integer.MAX_VALUE);
240         LOG.info("Creating table:" + htd);
241         admin.createTable(htd, startKey, endKey, numRegions);
242         Assert.assertTrue("Table: " + htd + " was not created", admin.tableExists(tableName));
243         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
244         enabledTables.put(tableName, freshTableDesc);
245         LOG.info("Created table:" + freshTableDesc);
246       } catch (Exception e){
247         LOG.warn("Caught exception in action: " + this.getClass());
248         // TODO workaround
249         // when master failover happens during CREATE_TABLE, client will do RPC retry and get TableExistsException
250         // ignore for now till better resolution
251         if (e instanceof TableExistsException) {
252           LOG.warn("Caught TableExistsException in action: " + this.getClass(), e);
253         } else {
254           throw e;
255         }
256       } finally {
257         admin.close();
258       }
259       verifyTables();
260     }
261 
262     private HTableDescriptor createTableDesc() {
263       String tableName = "ittable-" + String.format("%010d",
264         RandomUtils.nextInt(Integer.MAX_VALUE));
265       String familyName = "cf-" + Math.abs(RandomUtils.nextInt());
266       HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
267       // add random column family
268       htd.addFamily(new HColumnDescriptor(familyName));
269       return htd;
270     }
271   }
272 
273   private class DisableTableAction extends TableAction {
274 
275     @Override
276     void perform() throws IOException {
277 
278       HTableDescriptor selected = selectTable(enabledTables);
279       if (selected == null) {
280         return;
281       }
282 
283       Admin admin = connection.getAdmin();
284       try {
285         TableName tableName = selected.getTableName();
286         LOG.info("Disabling table :" + selected);
287         admin.disableTable(tableName);
288         Assert.assertTrue("Table: " + selected + " was not disabled",
289             admin.isTableDisabled(tableName));
290         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
291         disabledTables.put(tableName, freshTableDesc);
292         LOG.info("Disabled table :" + freshTableDesc);
293       } catch (Exception e){
294         LOG.warn("Caught exception in action: " + this.getClass());
295         // TODO workaround
296         // loose restriction for TableNotDisabledException/TableNotEnabledException thrown in sync
297         // operations
298         // 1) when enable/disable starts, the table state is changed to ENABLING/DISABLING (ZK node
299         // in 1.x), which will be further changed to ENABLED/DISABLED once the operation completes
300         // 2) if master failover happens in the middle of the enable/disable operation, the new
301         // master will try to recover the tables in ENABLING/DISABLING state, as programmed in
302         // AssignmentManager#recoverTableInEnablingState() and
303         // AssignmentManager#recoverTableInDisablingState()
304         // 3) after the new master initialization completes, the procedure tries to re-do the
305         // enable/disable operation, which was already done. Ignore those exceptions before change
306         // of behaviors of AssignmentManager in presence of PV2
307         if (e instanceof TableNotEnabledException) {
308           LOG.warn("Caught TableNotEnabledException in action: " + this.getClass());
309           e.printStackTrace();
310         } else {
311           throw e;
312         }
313       } finally {
314         admin.close();
315       }
316       verifyTables();
317     }
318   }
319 
320   private class EnableTableAction extends TableAction {
321 
322     @Override
323     void perform() throws IOException {
324 
325       HTableDescriptor selected = selectTable(disabledTables);
326       if (selected == null ) {
327         return;
328       }
329 
330       Admin admin = connection.getAdmin();
331       try {
332         TableName tableName = selected.getTableName();
333         LOG.info("Enabling table :" + selected);
334         admin.enableTable(tableName);
335         Assert.assertTrue("Table: " + selected + " was not enabled",
336             admin.isTableEnabled(tableName));
337         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
338         enabledTables.put(tableName, freshTableDesc);
339         LOG.info("Enabled table :" + freshTableDesc);
340       } catch (Exception e){
341         LOG.warn("Caught exception in action: " + this.getClass());
342         // TODO workaround
343         // loose restriction for TableNotDisabledException/TableNotEnabledException thrown in sync
344         // operations 1) when enable/disable starts, the table state is changed to
345         // ENABLING/DISABLING (ZK node in 1.x), which will be further changed to ENABLED/DISABLED
346         // once the operation completes 2) if master failover happens in the middle of the
347         // enable/disable operation, the new master will try to recover the tables in
348         // ENABLING/DISABLING state, as programmed in
349         // AssignmentManager#recoverTableInEnablingState() and
350         // AssignmentManager#recoverTableInDisablingState()
351         // 3) after the new master initialization completes, the procedure tries to re-do the
352         // enable/disable operation, which was already done. Ignore those exceptions before
353         // change of behaviors of AssignmentManager in presence of PV2
354         if (e instanceof TableNotDisabledException) {
355           LOG.warn("Caught TableNotDisabledException in action: " + this.getClass());
356           e.printStackTrace();
357         } else {
358           throw e;
359         }
360       } finally {
361         admin.close();
362       }
363       verifyTables();
364     }
365   }
366 
367   private class DeleteTableAction extends TableAction {
368 
369     @Override
370     void perform() throws IOException {
371 
372       HTableDescriptor selected = selectTable(disabledTables);
373       if (selected == null) {
374         return;
375       }
376 
377       Admin admin = connection.getAdmin();
378       try {
379         TableName tableName = selected.getTableName();
380         LOG.info("Deleting table :" + selected);
381         admin.deleteTable(tableName);
382         Assert.assertFalse("Table: " + selected + " was not deleted",
383                 admin.tableExists(tableName));
384         deletedTables.put(tableName, selected);
385         LOG.info("Deleted table :" + selected);
386       } catch (Exception e){
387         LOG.warn("Caught exception in action: " + this.getClass());
388         // TODO workaround
389         // when master failover happens during DELETE_TABLE, client will do RPC retry and get
390         // TableNotFoundException ignore for now till better resolution
391         if (e instanceof TableNotFoundException) {
392           LOG.warn("Caught TableNotFoundException in action: " + this.getClass());
393           e.printStackTrace();
394         } else {
395           throw e;
396         }
397       } finally {
398         admin.close();
399       }
400       verifyTables();
401     }
402   }
403 
404 
405   private abstract class ColumnAction extends TableAction{
406     // ColumnAction has implemented selectFamily() shared by multiple family Actions
407     protected HColumnDescriptor selectFamily(HTableDescriptor htd) {
408       if (htd == null) {
409         return null;
410       }
411       HColumnDescriptor[] families = htd.getColumnFamilies();
412       if (families.length == 0){
413         LOG.info("No column families in table: " + htd);
414         return null;
415       }
416       HColumnDescriptor randomCfd = families[RandomUtils.nextInt(families.length)];
417       return randomCfd;
418     }
419   }
420 
421   private class AddColumnFamilyAction extends ColumnAction {
422 
423     @Override
424     void perform() throws IOException {
425       HTableDescriptor selected = selectTable(disabledTables);
426       if (selected == null) {
427         return;
428       }
429 
430       Admin admin = connection.getAdmin();
431       try {
432         HColumnDescriptor cfd = createFamilyDesc();
433         if (selected.hasFamily(cfd.getName())){
434           LOG.info(new String(cfd.getName()) + " already exists in table "
435               + selected.getTableName());
436           return;
437         }
438         TableName tableName = selected.getTableName();
439         LOG.info("Adding column family: " + cfd + " to table: " + tableName);
440         admin.addColumn(tableName, cfd);
441         // assertion
442         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
443         Assert.assertTrue("Column family: " + cfd + " was not added",
444             freshTableDesc.hasFamily(cfd.getName()));
445         LOG.info("Added column family: " + cfd + " to table: " + tableName);
446         disabledTables.put(tableName, freshTableDesc);
447       } catch (Exception e){
448         LOG.warn("Caught exception in action: " + this.getClass());
449         // TODO HBASE-13415
450         // loose restriction for InvalidFamilyOperationException thrown in async operations before
451         // HBASE-13415 completes when failover happens, multiple procids may be created from the
452         // same request when 1 procedure succeeds, the others would complain about family already
453         // exists
454         if (e instanceof InvalidFamilyOperationException) {
455           LOG.warn("Caught InvalidFamilyOperationException in action: " + this.getClass());
456           e.printStackTrace();
457         } else {
458           throw e;
459         }
460       } finally {
461         admin.close();
462       }
463       verifyTables();
464     }
465 
466     private HColumnDescriptor createFamilyDesc() {
467       String familyName = "cf-" + String.format("%010d", RandomUtils.nextInt(Integer.MAX_VALUE));
468       HColumnDescriptor cfd = new HColumnDescriptor(familyName);
469       return cfd;
470     }
471   }
472 
473   private class AlterFamilyVersionsAction extends ColumnAction {
474 
475     @Override
476     void perform() throws IOException {
477       HTableDescriptor selected = selectTable(disabledTables);
478       if (selected == null) {
479         return;
480       }
481       HColumnDescriptor columnDesc = selectFamily(selected);
482       if (columnDesc == null){
483         return;
484       }
485 
486       Admin admin = connection.getAdmin();
487       int versions = RandomUtils.nextInt(10) + 3;
488       try {
489         TableName tableName = selected.getTableName();
490         LOG.info("Altering versions of column family: " + columnDesc + " to: " + versions +
491             " in table: " + tableName);
492         columnDesc.setMinVersions(versions);
493         columnDesc.setMaxVersions(versions);
494         admin.modifyTable(tableName, selected);
495         // assertion
496         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
497         HColumnDescriptor freshColumnDesc = freshTableDesc.getFamily(columnDesc.getName());
498         Assert.assertEquals("Column family: " + columnDesc + " was not altered",
499             freshColumnDesc.getMaxVersions(), versions);
500         Assert.assertEquals("Column family: " + freshColumnDesc + " was not altered",
501             freshColumnDesc.getMinVersions(), versions);
502         LOG.info("Altered versions of column family: " + columnDesc + " to: " + versions +
503             " in table: " + tableName);
504         disabledTables.put(tableName, freshTableDesc);
505       } catch (Exception e) {
506         LOG.warn("Caught exception in action: " + this.getClass());
507         throw e;
508       } finally {
509         admin.close();
510       }
511       verifyTables();
512     }
513   }
514 
515   private class AlterFamilyEncodingAction extends ColumnAction {
516 
517     @Override
518     void perform() throws IOException {
519       HTableDescriptor selected = selectTable(disabledTables);
520       if (selected == null) {
521         return;
522       }
523       HColumnDescriptor columnDesc = selectFamily(selected);
524       if (columnDesc == null){
525         return;
526       }
527 
528       Admin admin = connection.getAdmin();
529       try {
530         TableName tableName = selected.getTableName();
531         // possible DataBlockEncoding ids
532         int[] possibleIds = {0, 2, 3, 4, 6};
533         short id = (short) possibleIds[RandomUtils.nextInt(possibleIds.length)];
534         LOG.info("Altering encoding of column family: " + columnDesc + " to: " + id +
535             " in table: " + tableName);
536         columnDesc.setDataBlockEncoding(DataBlockEncoding.getEncodingById(id));
537         admin.modifyTable(tableName, selected);
538         // assertion
539         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
540         HColumnDescriptor freshColumnDesc = freshTableDesc.getFamily(columnDesc.getName());
541         Assert.assertEquals("Encoding of column family: " + columnDesc + " was not altered",
542             freshColumnDesc.getDataBlockEncoding().getId(), id);
543         LOG.info("Altered encoding of column family: " + freshColumnDesc + " to: " + id +
544             " in table: " + tableName);
545         disabledTables.put(tableName, freshTableDesc);
546       } catch (Exception e) {
547         LOG.warn("Caught exception in action: " + this.getClass());
548         throw e;
549       } finally {
550         admin.close();
551       }
552       verifyTables();
553     }
554   }
555 
556   private class DeleteColumnFamilyAction extends ColumnAction {
557 
558     @Override
559     void perform() throws IOException {
560       HTableDescriptor selected = selectTable(disabledTables);
561       HColumnDescriptor cfd = selectFamily(selected);
562       if (selected == null || cfd == null) {
563         return;
564       }
565 
566       Admin admin = connection.getAdmin();
567       try {
568         if (selected.getColumnFamilies().length < 2) {
569           LOG.info("No enough column families to delete in table " + selected.getTableName());
570           return;
571         }
572         TableName tableName = selected.getTableName();
573         LOG.info("Deleting column family: " + cfd + " from table: " + tableName);
574         admin.deleteColumn(tableName, cfd.getName());
575         // assertion
576         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
577         Assert.assertFalse("Column family: " + cfd + " was not added",
578             freshTableDesc.hasFamily(cfd.getName()));
579         LOG.info("Deleted column family: " + cfd + " from table: " + tableName);
580         disabledTables.put(tableName, freshTableDesc);
581       } catch (Exception e) {
582         LOG.warn("Caught exception in action: " + this.getClass());
583         // TODO HBASE-13415
584         // loose restriction for InvalidFamilyOperationException thrown in async operations before
585         // HBASE-13415 completes when failover happens, multiple procids may be created from the
586         //  same request when 1 procedure succeeds, the others would complain about family not
587         // exists
588         if (e instanceof InvalidFamilyOperationException) {
589           LOG.warn("Caught InvalidFamilyOperationException in action: " + this.getClass());
590           e.printStackTrace();
591         } else {
592           throw e;
593         }
594       } finally {
595         admin.close();
596       }
597       verifyTables();
598     }
599   }
600 
601   private class AddRowAction extends ColumnAction {
602     // populate tables
603     @Override
604     void perform() throws IOException {
605       HTableDescriptor selected = selectTable(enabledTables);
606       if (selected == null ) {
607         return;
608       }
609 
610       Admin admin = connection.getAdmin();
611       TableName tableName = selected.getTableName();
612       try (Table table = connection.getTable(tableName)){
613         ArrayList<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>(admin.getTableRegions(
614             selected.getTableName()));
615         int numRegions = regionInfos.size();
616         // average number of rows to be added per action to each region
617         int average_rows = 1;
618         int numRows = average_rows * numRegions;
619         LOG.info("Adding " + numRows + " rows to table: " + selected);
620         for (int i = 0; i < numRows; i++){
621           // nextInt(Integer.MAX_VALUE)) to return positive numbers only
622           byte[] rowKey = Bytes.toBytes(
623               "row-" + String.format("%010d", RandomUtils.nextInt(Integer.MAX_VALUE)));
624           HColumnDescriptor cfd = selectFamily(selected);
625           if (cfd == null){
626             return;
627           }
628           byte[] family = cfd.getName();
629           byte[] qualifier = Bytes.toBytes("col-" + RandomUtils.nextInt(Integer.MAX_VALUE) % 10);
630           byte[] value = Bytes.toBytes("val-" + RandomStringUtils.randomAlphanumeric(10));
631           Put put = new Put(rowKey);
632           put.addColumn(family, qualifier, value);
633           table.put(put);
634         }
635         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
636         enabledTables.put(tableName, freshTableDesc);
637         LOG.info("Added " + numRows + " rows to table: " + selected);
638       } catch (Exception e) {
639         LOG.warn("Caught exception in action: " + this.getClass());
640         throw e;
641       } finally {
642         admin.close();
643       }
644       verifyTables();
645     }
646   }
647 
648   private enum ACTION {
649     CREATE_TABLE,
650     DISABLE_TABLE,
651     ENABLE_TABLE,
652     DELETE_TABLE,
653     ADD_COLUMNFAMILY,
654     DELETE_COLUMNFAMILY,
655     ALTER_FAMILYVERSIONS,
656     ALTER_FAMILYENCODING,
657     ADD_ROW
658   }
659 
660   private class Worker extends Thread {
661 
662     private Exception savedException;
663 
664     private ACTION action;
665 
666     @Override
667     public void run() {
668       while (running.get()) {
669         // select random action
670         ACTION selectedAction = ACTION.values()[RandomUtils.nextInt() % ACTION.values().length];
671         this.action = selectedAction;
672         LOG.info("Performing Action: " + selectedAction);
673 
674         try {
675           switch (selectedAction) {
676           case CREATE_TABLE:
677             // stop creating new tables in the later stage of the test to avoid too many empty
678             // tables
679             if (create_table.get()) {
680               new CreateTableAction().perform();
681             }
682             break;
683           case ADD_ROW:
684             new AddRowAction().perform();
685             break;
686           case DISABLE_TABLE:
687             new DisableTableAction().perform();
688             break;
689           case ENABLE_TABLE:
690             new EnableTableAction().perform();
691             break;
692           case DELETE_TABLE:
693             // reduce probability of deleting table to 20%
694             if (RandomUtils.nextInt(100) < 20) {
695               new DeleteTableAction().perform();
696             }
697             break;
698           case ADD_COLUMNFAMILY:
699             new AddColumnFamilyAction().perform();
700             break;
701           case DELETE_COLUMNFAMILY:
702             // reduce probability of deleting column family to 20%
703             if (RandomUtils.nextInt(100) < 20) {
704               new DeleteColumnFamilyAction().perform();
705             }
706             break;
707           case ALTER_FAMILYVERSIONS:
708             new AlterFamilyVersionsAction().perform();
709             break;
710           case ALTER_FAMILYENCODING:
711             new AlterFamilyEncodingAction().perform();
712             break;
713           }
714         } catch (Exception ex) {
715           this.savedException = ex;
716           return;
717         }
718       }
719       LOG.info(this.getName() + " stopped");
720     }
721 
722     public Exception getSavedException(){
723       return this.savedException;
724     }
725 
726     public ACTION getAction(){
727       return this.action;
728     }
729   }
730 
731   private void checkException(List<Worker> workers){
732     if(workers == null || workers.isEmpty())
733       return;
734     for (Worker worker : workers){
735       Exception e = worker.getSavedException();
736       if (e != null) {
737         LOG.error("Found exception in thread: " + worker.getName());
738         e.printStackTrace();
739       }
740       Assert.assertNull("Action failed: " + worker.getAction() + " in thread: "
741           + worker.getName(), e);
742     }
743   }
744 
745   private int runTest() throws Exception {
746     LOG.info("Starting the test");
747 
748     String runtimeKey = String.format(RUN_TIME_KEY, this.getClass().getSimpleName());
749     long runtime = util.getConfiguration().getLong(runtimeKey, DEFAULT_RUN_TIME);
750 
751     String numThreadKey = String.format(NUM_THREADS_KEY, this.getClass().getSimpleName());
752     numThreads = util.getConfiguration().getInt(numThreadKey, DEFAULT_NUM_THREADS);
753 
754     ArrayList<Worker> workers = new ArrayList<>();
755     for (int i = 0; i < numThreads; i++) {
756       checkException(workers);
757       Worker worker = new Worker();
758       LOG.info("Launching worker thread " + worker.getName());
759       workers.add(worker);
760       worker.start();
761     }
762 
763     Threads.sleep(runtime / 2);
764     LOG.info("Stopping creating new tables");
765     create_table.set(false);
766     Threads.sleep(runtime / 2);
767     LOG.info("Runtime is up");
768     running.set(false);
769 
770     checkException(workers);
771 
772     for (Worker worker : workers) {
773       worker.join();
774     }
775     LOG.info("All Worker threads stopped");
776 
777     // verify
778     LOG.info("Verify actions of all threads succeeded");
779     checkException(workers);
780     LOG.info("Verify states of all tables");
781     verifyTables();
782 
783     // RUN HBCK
784 
785     HBaseFsck hbck = null;
786     try {
787       LOG.info("Running hbck");
788       hbck = HbckTestingUtil.doFsck(util.getConfiguration(), false);
789       HbckTestingUtil.assertNoErrors(hbck);
790       LOG.info("Finished hbck");
791     } finally {
792       if (hbck != null) {
793         hbck.close();
794       }
795     }
796      return 0;
797   }
798 
799   @Override
800   public TableName getTablename() {
801     return null;
802   }
803 
804   @Override
805   protected Set<String> getColumnFamilies() {
806     return null;
807   }
808 
809   public static void main(String[] args) throws Exception {
810     Configuration conf = HBaseConfiguration.create();
811     IntegrationTestingUtility.setUseDistributedCluster(conf);
812     IntegrationTestDDLMasterFailover masterFailover = new IntegrationTestDDLMasterFailover();
813     Connection connection = null;
814     int ret = 1;
815     try {
816       // Initialize connection once, then pass to Actions
817       LOG.debug("Setting up connection ...");
818       connection = ConnectionFactory.createConnection(conf);
819       masterFailover.setConnection(connection);
820       ret = ToolRunner.run(conf, masterFailover, args);
821     } catch (IOException e){
822       LOG.fatal("Failed to establish connection. Aborting test ...", e);
823     } finally {
824       connection = masterFailover.getConnection();
825       if (connection != null){
826         connection.close();
827       }
828       System.exit(ret);
829     }
830   }
831 }