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.assertFalse;
22  import static org.junit.Assert.assertTrue;
23  
24  import java.io.IOException;
25  import java.util.ArrayList;
26  import java.util.Arrays;
27  import java.util.Collection;
28  import java.util.List;
29  import java.util.Set;
30  import java.util.HashSet;
31  import java.util.TreeSet;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.fs.FileStatus;
36  import org.apache.hadoop.fs.FileSystem;
37  import org.apache.hadoop.fs.Path;
38  import org.apache.hadoop.fs.PathFilter;
39  import org.apache.hadoop.hbase.HBaseTestingUtility;
40  import org.apache.hadoop.hbase.HColumnDescriptor;
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.TableNotEnabledException;
45  import org.apache.hadoop.hbase.client.Durability;
46  import org.apache.hadoop.hbase.client.HBaseAdmin;
47  import org.apache.hadoop.hbase.client.HTable;
48  import org.apache.hadoop.hbase.client.Put;
49  import org.apache.hadoop.hbase.master.HMaster;
50  import org.apache.hadoop.hbase.master.MasterFileSystem;
51  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
52  import org.apache.hadoop.hbase.regionserver.HRegion;
53  import org.apache.hadoop.hbase.regionserver.HRegionServer;
54  import org.apache.hadoop.hbase.util.Bytes;
55  import org.apache.hadoop.hbase.util.FSTableDescriptors;
56  import org.apache.hadoop.hbase.util.FSUtils;
57  import org.apache.hadoop.hbase.util.FSVisitor;
58  import org.apache.hadoop.hbase.util.MD5Hash;
59  import org.junit.Assert;
60  
61  /**
62   * Utilities class for snapshots
63   */
64  public class SnapshotTestingUtils {
65  
66    private static final Log LOG = LogFactory.getLog(SnapshotTestingUtils.class);
67  
68    /**
69     * Assert that we don't have any snapshots lists
70     * @throws IOException if the admin operation fails
71     */
72    public static void assertNoSnapshots(HBaseAdmin admin) throws IOException {
73      assertEquals("Have some previous snapshots", 0, admin.listSnapshots().size());
74    }
75  
76    /**
77     * Make sure that there is only one snapshot returned from the master and its
78     * name and table match the passed in parameters.
79     */
80    public static List<SnapshotDescription> assertExistsMatchingSnapshot(
81        HBaseAdmin admin, String snapshotName, String tableName)
82        throws IOException {
83      // list the snapshot
84      List<SnapshotDescription> snapshots = admin.listSnapshots();
85  
86      List<SnapshotDescription> returnedSnapshots = new ArrayList<SnapshotDescription>();
87      for (SnapshotDescription sd : snapshots) {
88        if (snapshotName.equals(sd.getName()) &&
89            tableName.equals(sd.getTable())) {
90          returnedSnapshots.add(sd);
91        }
92      }
93  
94      Assert.assertTrue("No matching snapshots found.", returnedSnapshots.size()>0);
95      return returnedSnapshots;
96    }
97  
98    /**
99     * Make sure that there is only one snapshot returned from the master
100    */
101   public static void assertOneSnapshotThatMatches(HBaseAdmin admin,
102       HSnapshotDescription snapshot) throws IOException {
103     assertOneSnapshotThatMatches(admin, snapshot.getName(),
104         snapshot.getTable());
105   }
106 
107   /**
108    * Make sure that there is only one snapshot returned from the master and its name and table match
109    * the passed in parameters.
110    */
111   public static void assertOneSnapshotThatMatches(HBaseAdmin admin, SnapshotDescription snapshot)
112       throws IOException {
113     assertOneSnapshotThatMatches(admin, snapshot.getName(), snapshot.getTable());
114   }
115 
116   /**
117    * Make sure that there is only one snapshot returned from the master and its
118    * name and table match the passed in parameters.
119    */
120   public static List<SnapshotDescription> assertOneSnapshotThatMatches(
121       HBaseAdmin admin, String snapshotName, String tableName)
122       throws IOException {
123     // list the snapshot
124     List<SnapshotDescription> snapshots = admin.listSnapshots();
125 
126     assertEquals("Should only have 1 snapshot", 1, snapshots.size());
127     assertEquals(snapshotName, snapshots.get(0).getName());
128     assertEquals(tableName, snapshots.get(0).getTable());
129 
130     return snapshots;
131   }
132 
133   /**
134    * Make sure that there is only one snapshot returned from the master and its
135    * name and table match the passed in parameters.
136    */
137   public static List<SnapshotDescription> assertOneSnapshotThatMatches(
138       HBaseAdmin admin, byte[] snapshot, byte[] tableName) throws IOException {
139     return assertOneSnapshotThatMatches(admin, Bytes.toString(snapshot),
140         Bytes.toString(tableName));
141   }
142 
143   /**
144    * Confirm that the snapshot contains references to all the files that should
145    * be in the snapshot.
146    */
147   public static void confirmSnapshotValid(
148       SnapshotDescription snapshotDescriptor, byte[] tableName,
149       byte[] testFamily, Path rootDir, HBaseAdmin admin, FileSystem fs,
150       boolean requireLogs, Path logsDir, Set<String> snapshotServers)
151       throws IOException {
152     ArrayList nonEmptyTestFamilies = new ArrayList(1);
153     nonEmptyTestFamilies.add(testFamily);
154     confirmSnapshotValid(snapshotDescriptor, Bytes.toString(tableName),
155       nonEmptyTestFamilies, null, rootDir, admin, fs, requireLogs,
156       logsDir, snapshotServers);
157   }
158 
159   /**
160    * Confirm that the snapshot has no references files but only metadata.
161    */
162   public static void confirmEmptySnapshotValid(
163       SnapshotDescription snapshotDescriptor, byte[] tableName,
164       byte[] testFamily, Path rootDir, HBaseAdmin admin, FileSystem fs,
165       boolean requireLogs, Path logsDir, Set<String> snapshotServers)
166       throws IOException {
167     ArrayList emptyTestFamilies = new ArrayList(1);
168     emptyTestFamilies.add(testFamily);
169     confirmSnapshotValid(snapshotDescriptor, Bytes.toString(tableName),
170       null, emptyTestFamilies, rootDir, admin, fs, requireLogs,
171       logsDir, snapshotServers);
172   }
173 
174   /**
175    * Confirm that the snapshot contains references to all the files that should
176    * be in the snapshot. This method also perform some redundant check like
177    * the existence of the snapshotinfo or the regioninfo which are done always
178    * by the MasterSnapshotVerifier, at the end of the snapshot operation.
179    */
180   public static void confirmSnapshotValid(
181       SnapshotDescription snapshotDescriptor, String tableName,
182       List<byte[]> nonEmptyTestFamilies, List<byte[]> emptyTestFamilies,
183       Path rootDir, HBaseAdmin admin, FileSystem fs, boolean requireLogs,
184       Path logsDir, Set<String> snapshotServers) throws IOException {
185     // check snapshot dir
186     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(
187         snapshotDescriptor, rootDir);
188     assertTrue(fs.exists(snapshotDir));
189 
190     // check snapshot info
191     Path snapshotinfo = new Path(snapshotDir, SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
192     assertTrue(fs.exists(snapshotinfo));
193 
194     // check the logs dir
195     if (requireLogs) {
196       TakeSnapshotUtils.verifyAllLogsGotReferenced(fs, logsDir,
197           snapshotServers, snapshotDescriptor, new Path(snapshotDir,
198               HConstants.HREGION_LOGDIR_NAME));
199     }
200 
201     // check the table info
202     HTableDescriptor desc = FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, tableName);
203     HTableDescriptor snapshotDesc = FSTableDescriptors.getTableDescriptorFromFs(fs, snapshotDir);
204     assertEquals(desc, snapshotDesc);
205 
206     // Extract regions and families with store files
207     final Set<String> snapshotRegions = new HashSet<String>();
208     final Set<byte[]> snapshotFamilies = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
209     FSVisitor.visitTableStoreFiles(fs, snapshotDir, new FSVisitor.StoreFileVisitor() {
210       public void storeFile(final String region, final String family, final String hfileName)
211           throws IOException {
212         snapshotRegions.add(region);
213         snapshotFamilies.add(Bytes.toBytes(family));
214       }
215     });
216 
217     // Verify that there are store files in the specified families
218     if (nonEmptyTestFamilies != null) {
219       for (final byte[] familyName: nonEmptyTestFamilies) {
220         assertTrue(snapshotFamilies.contains(familyName));
221       }
222     }
223 
224     // Verify that there are no store files in the specified families
225     if (emptyTestFamilies != null) {
226       for (final byte[] familyName: emptyTestFamilies) {
227         assertFalse(snapshotFamilies.contains(familyName));
228       }
229     }
230 
231     // Avoid checking regions if the request is for an empty snapshot
232     if ((nonEmptyTestFamilies == null || nonEmptyTestFamilies.size() == 0) &&
233         (emptyTestFamilies != null && emptyTestFamilies.size() > 0)) {
234       assertEquals(0, snapshotRegions.size());
235       return;
236     }
237 
238     // check the region snapshot for all the regions
239     List<HRegionInfo> regions = admin.getTableRegions(Bytes.toBytes(tableName));
240     assertEquals(regions.size(), snapshotRegions.size());
241 
242     // Verify Regions
243     for (HRegionInfo info : regions) {
244       String regionName = info.getEncodedName();
245       assertTrue(snapshotRegions.contains(regionName));
246 
247       Path regionDir = new Path(snapshotDir, regionName);
248       HRegionInfo snapshotRegionInfo = HRegion.loadDotRegionInfoFileContent(fs, regionDir);
249       assertEquals(info, snapshotRegionInfo);
250     }
251   }
252 
253   /**
254    * Helper method for testing async snapshot operations. Just waits for the given snapshot to
255    * complete on the server by repeatedly checking the master.
256    * @param master running the snapshot
257    * @param snapshot to check
258    * @param sleep amount to sleep between checks to see if the snapshot is done
259    * @throws IOException if the snapshot fails
260    */
261   public static void waitForSnapshotToComplete(HMaster master, HSnapshotDescription snapshot,
262       long sleep) throws IOException {
263     boolean done = false;
264     while (!done) {
265       done = master.isSnapshotDone(snapshot);
266       try {
267         Thread.sleep(sleep);
268       } catch (InterruptedException e) {
269         throw new IOException(e);
270       }
271     }
272   }
273 
274   public static void cleanupSnapshot(HBaseAdmin admin, byte[] tableName)
275       throws IOException {
276     SnapshotTestingUtils.cleanupSnapshot(admin, Bytes.toString(tableName));
277   }
278 
279   public static void cleanupSnapshot(HBaseAdmin admin, String snapshotName)
280       throws IOException {
281     // delete the taken snapshot
282     admin.deleteSnapshot(snapshotName);
283     assertNoSnapshots(admin);
284   }
285 
286   /**
287    * Expect the snapshot to throw an error when checking if the snapshot is complete
288    * @param master master to check
289    * @param snapshot the {@link HSnapshotDescription} request to pass to the master
290    * @param clazz expected exception from the master
291    */
292   public static void expectSnapshotDoneException(HMaster master, HSnapshotDescription snapshot,
293       Class<? extends HBaseSnapshotException> clazz) {
294     try {
295       boolean res = master.isSnapshotDone(snapshot);
296       Assert.fail("didn't fail to lookup a snapshot: res=" + res);
297     } catch (HBaseSnapshotException e) {
298       assertEquals("Threw wrong snapshot exception!", clazz, e.getClass());
299     } catch (Throwable t) {
300       Assert.fail("Threw an unexpected exception:" + t);
301     }
302   }
303 
304   /**
305    * List all the HFiles in the given table
306    *
307    * @param fs: FileSystem where the table lives
308    * @param tableDir directory of the table
309    * @return array of the current HFiles in the table (could be a zero-length array)
310    * @throws IOException on unexecpted error reading the FS
311    */
312   public static Path[] listHFiles(final FileSystem fs, final Path tableDir)
313       throws IOException {
314     final ArrayList<Path> hfiles = new ArrayList<Path>();
315     FSVisitor.visitTableStoreFiles(fs, tableDir, new FSVisitor.StoreFileVisitor() {
316       public void storeFile(final String region, final String family, final String hfileName)
317           throws IOException {
318         hfiles.add(new Path(tableDir, new Path(region, new Path(family, hfileName))));
319       }
320     });
321     return hfiles.toArray(new Path[hfiles.size()]);
322   }
323 
324   /**
325    * Take a snapshot of the specified table and verify that the given family is
326    * not empty. Note that this will leave the table disabled
327    * in the case of an offline snapshot.
328    */
329   public static void createSnapshotAndValidate(HBaseAdmin admin,
330       String tableName, String familyName, String snapshotNameString,
331       Path rootDir, FileSystem fs, boolean onlineSnapshot)
332       throws Exception {
333     ArrayList<byte[]> nonEmptyFamilyNames = new ArrayList<byte[]>(1);
334     nonEmptyFamilyNames.add(Bytes.toBytes(familyName));
335     createSnapshotAndValidate(admin, tableName, nonEmptyFamilyNames, /* emptyFamilyNames= */ null,
336                               snapshotNameString, rootDir, fs, onlineSnapshot);
337   }
338 
339   /**
340    * Take a snapshot of the specified table and verify the given families.
341    * Note that this will leave the table disabled in the case of an offline snapshot.
342    */
343   public static void createSnapshotAndValidate(HBaseAdmin admin,
344       String tableName, List<byte[]> nonEmptyFamilyNames, List<byte[]> emptyFamilyNames,
345       String snapshotNameString, Path rootDir, FileSystem fs, boolean onlineSnapshot)
346         throws Exception {
347     if (!onlineSnapshot) {
348       try {
349         admin.disableTable(tableName);
350       } catch (TableNotEnabledException tne) {
351         LOG.info("In attempting to disable " + tableName + " it turns out that the this table is " +
352             "already disabled.");
353       }
354     }
355     admin.snapshot(snapshotNameString, tableName);
356 
357     List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertExistsMatchingSnapshot(admin,
358       snapshotNameString, tableName);
359     if (snapshots == null || snapshots.size() != 1) {
360       Assert.fail("Incorrect number of snapshots for table " + tableName);
361     }
362 
363     SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), tableName, nonEmptyFamilyNames,
364       emptyFamilyNames, rootDir, admin, fs, false,
365       new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), null);
366   }
367 
368   // ==========================================================================
369   //  Table Helpers
370   // ==========================================================================
371   public static void waitForTableToBeOnline(final HBaseTestingUtility util, final byte[] tableName)
372       throws IOException, InterruptedException {
373     HRegionServer rs = util.getRSForFirstRegionInTable(tableName);
374     List<HRegion> onlineRegions = rs.getOnlineRegions(tableName);
375     for (HRegion region : onlineRegions) {
376       region.waitForFlushesAndCompactions();
377     }
378     util.getHBaseAdmin().isTableAvailable(tableName);
379   }
380 
381   public static void createTable(final HBaseTestingUtility util, final byte[] tableName,
382       final byte[]... families) throws IOException, InterruptedException {
383     HTableDescriptor htd = new HTableDescriptor(tableName);
384     for (byte[] family: families) {
385       HColumnDescriptor hcd = new HColumnDescriptor(family);
386       htd.addFamily(hcd);
387     }
388     byte[][] splitKeys = new byte[14][];
389     byte[] hex = Bytes.toBytes("123456789abcde");
390     for (int i = 0; i < splitKeys.length; ++i) {
391       splitKeys[i] = new byte[] { hex[i] };
392     }
393     util.getHBaseAdmin().createTable(htd, splitKeys);
394     waitForTableToBeOnline(util, tableName);
395     assertEquals(15, util.getHBaseAdmin().getTableRegions(tableName).size());
396   }
397 
398   public static void loadData(final HBaseTestingUtility util, final byte[] tableName, int rows,
399       byte[]... families) throws IOException, InterruptedException {
400     loadData(util, new HTable(util.getConfiguration(), tableName), rows, families);
401   }
402 
403   public static void loadData(final HBaseTestingUtility util, final HTable table, int rows,
404       byte[]... families) throws IOException, InterruptedException {
405     table.setAutoFlush(false);
406 
407     // Ensure one row per region
408     assertTrue(rows >= 16);
409     for (byte k0: Bytes.toBytes("0123456789abcdef")) {
410       byte[] k = new byte[] { k0 };
411       byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), k);
412       byte[] key = Bytes.add(k, Bytes.toBytes(MD5Hash.getMD5AsHex(value)));
413       putData(table, families, key, value);
414       rows--;
415     }
416 
417     // Add other extra rows. more rows, more files
418     while (rows-- > 0) {
419       byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), Bytes.toBytes(rows));
420       byte[] key = Bytes.toBytes(MD5Hash.getMD5AsHex(value));
421       putData(table, families, key, value);
422     }
423     table.flushCommits();
424 
425     waitForTableToBeOnline(util, table.getTableName());
426   }
427 
428   private static void putData(final HTable table, final byte[][] families,
429       final byte[] key, final byte[] value) throws IOException {
430     byte[] q = Bytes.toBytes("q");
431     Put put = new Put(key);
432     put.setDurability(Durability.SKIP_WAL);
433     for (byte[] family: families) {
434       put.add(family, q, value);
435     }
436     table.put(put);
437   }
438 
439   public static void deleteAllSnapshots(final HBaseAdmin admin)
440       throws IOException {
441     // Delete all the snapshots
442     for (SnapshotDescription snapshot: admin.listSnapshots()) {
443       admin.deleteSnapshot(snapshot.getName());
444     }
445     SnapshotTestingUtils.assertNoSnapshots(admin);
446   }
447 
448   public static void deleteArchiveDirectory(final HBaseTestingUtility util)
449       throws IOException {
450     // Ensure the archiver to be empty
451     MasterFileSystem mfs = util.getMiniHBaseCluster().getMaster().getMasterFileSystem();
452     Path archiveDir = new Path(mfs.getRootDir(), HConstants.HFILE_ARCHIVE_DIRECTORY);
453     mfs.getFileSystem().delete(archiveDir, true);
454   }
455 
456   public static void verifyRowCount(final HBaseTestingUtility util, final byte[] tableName,
457       long expectedRows) throws IOException {
458     HTable table = new HTable(util.getConfiguration(), tableName);
459     try {
460       assertEquals(expectedRows, util.countRows(table));
461     } finally {
462       table.close();
463     }
464   }
465 }