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