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  package org.apache.hadoop.hbase.snapshot;
19  
20  import static org.junit.Assert.assertEquals;
21  import static org.junit.Assert.assertTrue;
22  import static org.junit.Assert.fail;
23  
24  import java.io.IOException;
25  import java.util.Collections;
26  import java.util.Comparator;
27  import java.util.HashMap;
28  import java.util.HashSet;
29  import java.util.List;
30  import java.util.Map;
31  import java.util.Set;
32  import java.util.concurrent.CountDownLatch;
33  
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.commons.logging.impl.Log4JLogger;
37  import org.apache.hadoop.conf.Configuration;
38  import org.apache.hadoop.fs.FileSystem;
39  import org.apache.hadoop.fs.Path;
40  import org.apache.hadoop.hbase.HBaseTestingUtility;
41  import org.apache.hadoop.hbase.HConstants;
42  import org.apache.hadoop.hbase.HRegionInfo;
43  import org.apache.hadoop.hbase.HTableDescriptor;
44  import org.apache.hadoop.hbase.LargeTests;
45  import org.apache.hadoop.hbase.TableName;
46  import org.apache.hadoop.hbase.TableNotFoundException;
47  import org.apache.hadoop.hbase.client.HBaseAdmin;
48  import org.apache.hadoop.hbase.client.HTable;
49  import org.apache.hadoop.hbase.client.ScannerCallable;
50  import org.apache.hadoop.hbase.ipc.RpcClient;
51  import org.apache.hadoop.hbase.ipc.RpcServer;
52  import org.apache.hadoop.hbase.master.HMaster;
53  import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
54  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
55  import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
56  import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
57  import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
58  import org.apache.hadoop.hbase.util.Bytes;
59  import org.apache.hadoop.hbase.util.FSTableDescriptors;
60  import org.apache.hadoop.hbase.util.FSUtils;
61  import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
62  import org.apache.log4j.Level;
63  import org.junit.After;
64  import org.junit.AfterClass;
65  import org.junit.Before;
66  import org.junit.BeforeClass;
67  import org.junit.Test;
68  import org.junit.experimental.categories.Category;
69  
70  /**
71   * Test creating/using/deleting snapshots from the client
72   * <p>
73   * This is an end-to-end test for the snapshot utility
74   *
75   * TODO This is essentially a clone of TestSnapshotFromClient.  This is worth refactoring this
76   * because there will be a few more flavors of snapshots that need to run these tests.
77   */
78  @Category(LargeTests.class)
79  public class TestFlushSnapshotFromClient {
80    private static final Log LOG = LogFactory.getLog(TestFlushSnapshotFromClient.class);
81    private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
82    private static final int NUM_RS = 2;
83    private static final String STRING_TABLE_NAME = "test";
84    private static final byte[] TEST_FAM = Bytes.toBytes("fam");
85    private static final byte[] TEST_QUAL = Bytes.toBytes("q");
86    private static final TableName TABLE_NAME =
87        TableName.valueOf(STRING_TABLE_NAME);
88    private final int DEFAULT_NUM_ROWS = 100;
89  
90    /**
91     * Setup the config for the cluster
92     * @throws Exception on failure
93     */
94    @BeforeClass
95    public static void setupCluster() throws Exception {
96      ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
97      ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
98      ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
99      setupConf(UTIL.getConfiguration());
100     UTIL.startMiniCluster(NUM_RS);
101   }
102 
103   private static void setupConf(Configuration conf) {
104     // disable the ui
105     conf.setInt("hbase.regionsever.info.port", -1);
106     // change the flush size to a small amount, regulating number of store files
107     conf.setInt("hbase.hregion.memstore.flush.size", 25000);
108     // so make sure we get a compaction when doing a load, but keep around some
109     // files in the store
110     conf.setInt("hbase.hstore.compaction.min", 10);
111     conf.setInt("hbase.hstore.compactionThreshold", 10);
112     // block writes if we get to 12 store files
113     conf.setInt("hbase.hstore.blockingStoreFiles", 12);
114     // Enable snapshot
115     conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
116     conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
117       ConstantSizeRegionSplitPolicy.class.getName());
118   }
119 
120   @Before
121   public void setup() throws Exception {
122     SnapshotTestingUtils.createTable(UTIL, TABLE_NAME, TEST_FAM);
123   }
124 
125   @After
126   public void tearDown() throws Exception {
127     UTIL.deleteTable(TABLE_NAME);
128 
129     SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
130     SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
131   }
132 
133   @AfterClass
134   public static void cleanupTest() throws Exception {
135     try {
136       UTIL.shutdownMiniCluster();
137     } catch (Exception e) {
138       LOG.warn("failure shutting down cluster", e);
139     }
140   }
141 
142   /**
143    * Test simple flush snapshotting a table that is online
144    * @throws Exception
145    */
146   @Test (timeout=300000)
147   public void testFlushTableSnapshot() throws Exception {
148     HBaseAdmin admin = UTIL.getHBaseAdmin();
149     // make sure we don't fail on listing snapshots
150     SnapshotTestingUtils.assertNoSnapshots(admin);
151 
152     // put some stuff in the table
153     HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME);
154     SnapshotTestingUtils.loadData(UTIL, table, DEFAULT_NUM_ROWS, TEST_FAM);
155 
156     // get the name of all the regionservers hosting the snapshotted table
157     Set<String> snapshotServers = new HashSet<String>();
158     List<RegionServerThread> servers = UTIL.getMiniHBaseCluster().getLiveRegionServerThreads();
159     for (RegionServerThread server : servers) {
160       if (server.getRegionServer().getOnlineRegions(TABLE_NAME).size() > 0) {
161         snapshotServers.add(server.getRegionServer().getServerName().toString());
162       }
163     }
164 
165     LOG.debug("FS state before snapshot:");
166     FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
167         FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
168 
169     // take a snapshot of the enabled table
170     String snapshotString = "offlineTableSnapshot";
171     byte[] snapshot = Bytes.toBytes(snapshotString);
172     admin.snapshot(snapshotString, STRING_TABLE_NAME, SnapshotDescription.Type.FLUSH);
173     LOG.debug("Snapshot completed.");
174 
175     // make sure we have the snapshot
176     List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
177       snapshot, TABLE_NAME);
178 
179     // make sure its a valid snapshot
180     FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
181     Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
182     LOG.debug("FS state after snapshot:");
183     FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
184         FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
185 
186     SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, TEST_FAM, rootDir,
187         admin, fs, false, new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), snapshotServers);
188   }
189 
190 
191   /**
192    * Test simple flush snapshotting a table that is online
193    * @throws Exception
194    */
195   @Test (timeout=300000)
196   public void testFlushTableSnapshotWithProcedure() throws Exception {
197     HBaseAdmin admin = UTIL.getHBaseAdmin();
198     // make sure we don't fail on listing snapshots
199     SnapshotTestingUtils.assertNoSnapshots(admin);
200 
201     // put some stuff in the table
202     HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME);
203     SnapshotTestingUtils.loadData(UTIL, table, DEFAULT_NUM_ROWS, TEST_FAM);
204 
205     // get the name of all the regionservers hosting the snapshotted table
206     Set<String> snapshotServers = new HashSet<String>();
207     List<RegionServerThread> servers = UTIL.getMiniHBaseCluster().getLiveRegionServerThreads();
208     for (RegionServerThread server : servers) {
209       if (server.getRegionServer().getOnlineRegions(TABLE_NAME).size() > 0) {
210         snapshotServers.add(server.getRegionServer().getServerName().toString());
211       }
212     }
213 
214     LOG.debug("FS state before snapshot:");
215     FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
216         FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
217 
218     // take a snapshot of the enabled table
219     String snapshotString = "offlineTableSnapshot";
220     byte[] snapshot = Bytes.toBytes(snapshotString);
221     Map<String, String> props = new HashMap<String, String>();
222     props.put("table", STRING_TABLE_NAME);
223     admin.execProcedure(SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION,
224         snapshotString, props);
225 
226 
227     LOG.debug("Snapshot completed.");
228 
229     // make sure we have the snapshot
230     List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
231       snapshot, TABLE_NAME);
232 
233     // make sure its a valid snapshot
234     FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
235     Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
236     LOG.debug("FS state after snapshot:");
237     FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
238         FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
239 
240     SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, TEST_FAM, rootDir,
241         admin, fs, false, new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), snapshotServers);
242   }
243 
244   @Test (timeout=300000)
245   public void testSnapshotFailsOnNonExistantTable() throws Exception {
246     HBaseAdmin admin = UTIL.getHBaseAdmin();
247     // make sure we don't fail on listing snapshots
248     SnapshotTestingUtils.assertNoSnapshots(admin);
249     String tableName = "_not_a_table";
250 
251     // make sure the table doesn't exist
252     boolean fail = false;
253     do {
254     try {
255       admin.getTableDescriptor(Bytes.toBytes(tableName));
256       fail = true;
257       LOG.error("Table:" + tableName + " already exists, checking a new name");
258       tableName = tableName+"!";
259     } catch (TableNotFoundException e) {
260       fail = false;
261       }
262     } while (fail);
263 
264     // snapshot the non-existant table
265     try {
266       admin.snapshot("fail", tableName, SnapshotDescription.Type.FLUSH);
267       fail("Snapshot succeeded even though there is not table.");
268     } catch (SnapshotCreationException e) {
269       LOG.info("Correctly failed to snapshot a non-existant table:" + e.getMessage());
270     }
271   }
272 
273   @Test(timeout = 300000)
274   public void testAsyncFlushSnapshot() throws Exception {
275     HBaseAdmin admin = UTIL.getHBaseAdmin();
276     SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("asyncSnapshot")
277         .setTable(TABLE_NAME.getNameAsString())
278         .setType(SnapshotDescription.Type.FLUSH)
279         .build();
280 
281     // take the snapshot async
282     admin.takeSnapshotAsync(snapshot);
283 
284     // constantly loop, looking for the snapshot to complete
285     HMaster master = UTIL.getMiniHBaseCluster().getMaster();
286     SnapshotTestingUtils.waitForSnapshotToComplete(master, snapshot, 200);
287     LOG.info(" === Async Snapshot Completed ===");
288     FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
289       FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
290     // make sure we get the snapshot
291     SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, snapshot);
292   }
293 
294   @Test (timeout=300000)
295   public void testSnapshotStateAfterMerge() throws Exception {
296     int numRows = DEFAULT_NUM_ROWS;
297     HBaseAdmin admin = UTIL.getHBaseAdmin();
298     // make sure we don't fail on listing snapshots
299     SnapshotTestingUtils.assertNoSnapshots(admin);
300     // load the table so we have some data
301     SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, numRows, TEST_FAM);
302 
303     // Take a snapshot
304     String snapshotBeforeMergeName = "snapshotBeforeMerge";
305     admin.snapshot(snapshotBeforeMergeName, STRING_TABLE_NAME, SnapshotDescription.Type.FLUSH);
306 
307     // Clone the table
308     String cloneBeforeMergeName = "cloneBeforeMerge";
309     admin.cloneSnapshot(snapshotBeforeMergeName, cloneBeforeMergeName);
310     SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TableName.valueOf(cloneBeforeMergeName));
311 
312     // Merge two regions
313     List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
314     Collections.sort(regions, new Comparator<HRegionInfo>() {
315       public int compare(HRegionInfo r1, HRegionInfo r2) {
316         return Bytes.compareTo(r1.getStartKey(), r2.getStartKey());
317       }
318     });
319 
320     int numRegions = admin.getTableRegions(TABLE_NAME).size();
321     int numRegionsAfterMerge = numRegions - 2;
322     admin.mergeRegions(regions.get(1).getEncodedNameAsBytes(),
323         regions.get(2).getEncodedNameAsBytes(), true);
324     admin.mergeRegions(regions.get(5).getEncodedNameAsBytes(),
325         regions.get(6).getEncodedNameAsBytes(), true);
326 
327     // Verify that there's one region less
328     waitRegionsAfterMerge(numRegionsAfterMerge);
329     assertEquals(numRegionsAfterMerge, admin.getTableRegions(TABLE_NAME).size());
330 
331     // Clone the table
332     String cloneAfterMergeName = "cloneAfterMerge";
333     admin.cloneSnapshot(snapshotBeforeMergeName, cloneAfterMergeName);
334     SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TableName.valueOf(cloneAfterMergeName));
335 
336     SnapshotTestingUtils.verifyRowCount(UTIL, TABLE_NAME, numRows);
337     SnapshotTestingUtils.verifyRowCount(UTIL, TableName.valueOf(cloneBeforeMergeName), numRows);
338     SnapshotTestingUtils.verifyRowCount(UTIL, TableName.valueOf(cloneAfterMergeName), numRows);
339 
340     // test that we can delete the snapshot
341     UTIL.deleteTable(cloneAfterMergeName);
342     UTIL.deleteTable(cloneBeforeMergeName);
343   }
344 
345   @Test (timeout=300000)
346   public void testTakeSnapshotAfterMerge() throws Exception {
347     int numRows = DEFAULT_NUM_ROWS;
348     HBaseAdmin admin = UTIL.getHBaseAdmin();
349     // make sure we don't fail on listing snapshots
350     SnapshotTestingUtils.assertNoSnapshots(admin);
351     // load the table so we have some data
352     SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, numRows, TEST_FAM);
353 
354     // Merge two regions
355     List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
356     Collections.sort(regions, new Comparator<HRegionInfo>() {
357       public int compare(HRegionInfo r1, HRegionInfo r2) {
358         return Bytes.compareTo(r1.getStartKey(), r2.getStartKey());
359       }
360     });
361 
362     int numRegions = admin.getTableRegions(TABLE_NAME).size();
363     int numRegionsAfterMerge = numRegions - 2;
364     admin.mergeRegions(regions.get(1).getEncodedNameAsBytes(),
365         regions.get(2).getEncodedNameAsBytes(), true);
366     admin.mergeRegions(regions.get(5).getEncodedNameAsBytes(),
367         regions.get(6).getEncodedNameAsBytes(), true);
368 
369     waitRegionsAfterMerge(numRegionsAfterMerge);
370     assertEquals(numRegionsAfterMerge, admin.getTableRegions(TABLE_NAME).size());
371 
372     // Take a snapshot
373     String snapshotName = "snapshotAfterMerge";
374     SnapshotTestingUtils.snapshot(admin, snapshotName, STRING_TABLE_NAME,
375       SnapshotDescription.Type.FLUSH, 3);
376 
377     // Clone the table
378     String cloneName = "cloneMerge";
379     admin.cloneSnapshot(snapshotName, cloneName);
380     SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TableName.valueOf(cloneName));
381 
382     SnapshotTestingUtils.verifyRowCount(UTIL, TABLE_NAME, numRows);
383     SnapshotTestingUtils.verifyRowCount(UTIL, TableName.valueOf(cloneName), numRows);
384 
385     // test that we can delete the snapshot
386     UTIL.deleteTable(cloneName);
387   }
388 
389   /**
390    * Basic end-to-end test of simple-flush-based snapshots
391    */
392   @Test (timeout=300000)
393   public void testFlushCreateListDestroy() throws Exception {
394     LOG.debug("------- Starting Snapshot test -------------");
395     HBaseAdmin admin = UTIL.getHBaseAdmin();
396     // make sure we don't fail on listing snapshots
397     SnapshotTestingUtils.assertNoSnapshots(admin);
398     // load the table so we have some data
399     SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, DEFAULT_NUM_ROWS, TEST_FAM);
400 
401     String snapshotName = "flushSnapshotCreateListDestroy";
402     // test creating the snapshot
403     admin.snapshot(snapshotName, STRING_TABLE_NAME, SnapshotDescription.Type.FLUSH);
404     logFSTree(FSUtils.getRootDir(UTIL.getConfiguration()));
405 
406     // make sure we only have 1 matching snapshot
407     List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
408       snapshotName, TABLE_NAME);
409 
410     // check the directory structure
411     FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
412     Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
413     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshots.get(0), rootDir);
414     assertTrue(fs.exists(snapshotDir));
415     FSUtils.logFileSystemState(UTIL.getTestFileSystem(), snapshotDir, LOG);
416     Path snapshotinfo = new Path(snapshotDir, SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
417     assertTrue(fs.exists(snapshotinfo));
418 
419     // check the table info
420     HTableDescriptor desc = FSTableDescriptors.getTableDescriptorFromFs(fs,
421         rootDir, TABLE_NAME);
422     HTableDescriptor snapshotDesc = FSTableDescriptors.getTableDescriptorFromFs(fs,
423         new Path(SnapshotDescriptionUtils.getSnapshotsDir(rootDir), snapshotName));
424     assertEquals(desc, snapshotDesc);
425 
426     // check the region snapshot for all the regions
427     List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
428     assertTrue(regions.size() > 1);
429     for (HRegionInfo info : regions) {
430       String regionName = info.getEncodedName();
431       Path regionDir = new Path(snapshotDir, regionName);
432       HRegionInfo snapshotRegionInfo = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
433       assertEquals(info, snapshotRegionInfo);
434       // check to make sure we have the family
435       Path familyDir = new Path(regionDir, Bytes.toString(TEST_FAM));
436       assertTrue("Missing region " + Bytes.toString(snapshotRegionInfo.getStartKey()),
437                  fs.exists(familyDir));
438 
439       // make sure we have some file references
440       assertTrue(fs.listStatus(familyDir).length > 0);
441     }
442   }
443 
444   /**
445    * Demonstrate that we reject snapshot requests if there is a snapshot already running on the
446    * same table currently running and that concurrent snapshots on different tables can both
447    * succeed concurretly.
448    */
449   @Test(timeout=300000)
450   public void testConcurrentSnapshottingAttempts() throws IOException, InterruptedException {
451     final String STRING_TABLE2_NAME = STRING_TABLE_NAME + "2";
452     final TableName TABLE2_NAME =
453         TableName.valueOf(STRING_TABLE2_NAME);
454 
455     int ssNum = 20;
456     HBaseAdmin admin = UTIL.getHBaseAdmin();
457     // make sure we don't fail on listing snapshots
458     SnapshotTestingUtils.assertNoSnapshots(admin);
459     // create second testing table
460     SnapshotTestingUtils.createTable(UTIL, TABLE2_NAME, TEST_FAM);
461     // load the table so we have some data
462     SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, DEFAULT_NUM_ROWS, TEST_FAM);
463     SnapshotTestingUtils.loadData(UTIL, TABLE2_NAME, DEFAULT_NUM_ROWS, TEST_FAM);
464 
465     final CountDownLatch toBeSubmitted = new CountDownLatch(ssNum);
466     // We'll have one of these per thread
467     class SSRunnable implements Runnable {
468       SnapshotDescription ss;
469       SSRunnable(SnapshotDescription ss) {
470         this.ss = ss;
471       }
472 
473       @Override
474       public void run() {
475         try {
476           HBaseAdmin admin = UTIL.getHBaseAdmin();
477           LOG.info("Submitting snapshot request: " + ClientSnapshotDescriptionUtils.toString(ss));
478           admin.takeSnapshotAsync(ss);
479         } catch (Exception e) {
480           LOG.info("Exception during snapshot request: " + ClientSnapshotDescriptionUtils.toString(
481               ss)
482               + ".  This is ok, we expect some", e);
483         }
484         LOG.info("Submitted snapshot request: " + ClientSnapshotDescriptionUtils.toString(ss));
485         toBeSubmitted.countDown();
486       }
487     };
488 
489     // build descriptions
490     SnapshotDescription[] descs = new SnapshotDescription[ssNum];
491     for (int i = 0; i < ssNum; i++) {
492       SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
493       builder.setTable(((i % 2) == 0 ? TABLE_NAME : TABLE2_NAME).getNameAsString());
494       builder.setName("ss"+i);
495       builder.setType(SnapshotDescription.Type.FLUSH);
496       descs[i] = builder.build();
497     }
498 
499     // kick each off its own thread
500     for (int i=0 ; i < ssNum; i++) {
501       new Thread(new SSRunnable(descs[i])).start();
502     }
503 
504     // wait until all have been submitted
505     toBeSubmitted.await();
506 
507     // loop until all are done.
508     while (true) {
509       int doneCount = 0;
510       for (SnapshotDescription ss : descs) {
511         try {
512           if (admin.isSnapshotFinished(ss)) {
513             doneCount++;
514           }
515         } catch (Exception e) {
516           LOG.warn("Got an exception when checking for snapshot " + ss.getName(), e);
517           doneCount++;
518         }
519       }
520       if (doneCount == descs.length) {
521         break;
522       }
523       Thread.sleep(100);
524     }
525 
526     // dump for debugging
527     logFSTree(FSUtils.getRootDir(UTIL.getConfiguration()));
528 
529     List<SnapshotDescription> taken = admin.listSnapshots();
530     int takenSize = taken.size();
531     LOG.info("Taken " + takenSize + " snapshots:  " + taken);
532     assertTrue("We expect at least 1 request to be rejected because of we concurrently" +
533         " issued many requests", takenSize < ssNum && takenSize > 0);
534 
535     // Verify that there's at least one snapshot per table
536     int t1SnapshotsCount = 0;
537     int t2SnapshotsCount = 0;
538     for (SnapshotDescription ss : taken) {
539       if (TableName.valueOf(ss.getTable()).equals(TABLE_NAME)) {
540         t1SnapshotsCount++;
541       } else if (TableName.valueOf(ss.getTable()).equals(TABLE2_NAME)) {
542         t2SnapshotsCount++;
543       }
544     }
545     assertTrue("We expect at least 1 snapshot of table1 ", t1SnapshotsCount > 0);
546     assertTrue("We expect at least 1 snapshot of table2 ", t2SnapshotsCount > 0);
547 
548     UTIL.deleteTable(TABLE2_NAME);
549   }
550 
551   private void logFSTree(Path root) throws IOException {
552     FSUtils.logFileSystemState(UTIL.getDFSCluster().getFileSystem(), root, LOG);
553   }
554 
555   private void waitRegionsAfterMerge(final long numRegionsAfterMerge)
556       throws IOException, InterruptedException {
557     HBaseAdmin admin = UTIL.getHBaseAdmin();
558     // Verify that there's one region less
559     long startTime = System.currentTimeMillis();
560     while (admin.getTableRegions(TABLE_NAME).size() != numRegionsAfterMerge) {
561       // This may be flaky... if after 15sec the merge is not complete give up
562       // it will fail in the assertEquals(numRegionsAfterMerge).
563       if ((System.currentTimeMillis() - startTime) > 15000)
564         break;
565       Thread.sleep(100);
566     }
567     SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TABLE_NAME);
568   }
569 }