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.snapshot;
20  
21  import java.io.IOException;
22  import java.io.FileNotFoundException;
23  import java.net.URI;
24  import java.text.SimpleDateFormat;
25  import java.util.ArrayList;
26  import java.util.Date;
27  import java.util.List;
28  import java.util.concurrent.atomic.AtomicInteger;
29  import java.util.concurrent.atomic.AtomicLong;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  
34  import org.apache.hadoop.fs.Path;
35  import org.apache.hadoop.fs.FileStatus;
36  import org.apache.hadoop.fs.FileSystem;
37  import org.apache.hadoop.hbase.classification.InterfaceAudience;
38  import org.apache.hadoop.hbase.classification.InterfaceStability;
39  import org.apache.hadoop.conf.Configured;
40  import org.apache.hadoop.hbase.HRegionInfo;
41  import org.apache.hadoop.hbase.TableName;
42  import org.apache.hadoop.util.StringUtils;
43  import org.apache.hadoop.util.Tool;
44  import org.apache.hadoop.util.ToolRunner;
45  
46  import org.apache.hadoop.conf.Configuration;
47  import org.apache.hadoop.hbase.HBaseConfiguration;
48  import org.apache.hadoop.hbase.io.HFileLink;
49  import org.apache.hadoop.hbase.io.WALLink;
50  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
51  import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
52  import org.apache.hadoop.hbase.util.FSUtils;
53  
54  /**
55   * Tool for dumping snapshot information.
56   * <ol>
57   * <li> Table Descriptor
58   * <li> Snapshot creation time, type, format version, ...
59   * <li> List of hfiles and wals
60   * <li> Stats about hfiles and logs sizes, percentage of shared with the source table, ...
61   * </ol>
62   */
63  @InterfaceAudience.Public
64  @InterfaceStability.Evolving
65  public final class SnapshotInfo extends Configured implements Tool {
66    private static final Log LOG = LogFactory.getLog(SnapshotInfo.class);
67  
68    /**
69     * Statistics about the snapshot
70     * <ol>
71     * <li> How many store files and logs are in the archive
72     * <li> How many store files and logs are shared with the table
73     * <li> Total store files and logs size and shared amount
74     * </ol>
75     */
76    public static class SnapshotStats {
77      /** Information about the file referenced by the snapshot */
78      static class FileInfo {
79        private final boolean corrupted;
80        private final boolean inArchive;
81        private final long size;
82  
83        FileInfo(final boolean inArchive, final long size, final boolean corrupted) {
84          this.corrupted = corrupted;
85          this.inArchive = inArchive;
86          this.size = size;
87        }
88  
89        /** @return true if the file is in the archive */
90        public boolean inArchive() {
91          return this.inArchive;
92        }
93  
94        /** @return true if the file is corrupted */
95        public boolean isCorrupted() {
96          return this.corrupted;
97        }
98  
99        /** @return true if the file is missing */
100       public boolean isMissing() {
101         return this.size < 0;
102       }
103 
104       /** @return the file size */
105       public long getSize() {
106         return this.size;
107       }
108 
109       String getStateToString() {
110         if (isCorrupted()) return "CORRUPTED";
111         if (isMissing()) return "NOT FOUND";
112         if (inArchive()) return "archive";
113         return null;
114       }
115     }
116 
117     private AtomicInteger hfileArchiveCount = new AtomicInteger();
118     private AtomicInteger hfilesCorrupted = new AtomicInteger();
119     private AtomicInteger hfilesMissing = new AtomicInteger();
120     private AtomicInteger hfilesCount = new AtomicInteger();
121     private AtomicInteger logsMissing = new AtomicInteger();
122     private AtomicInteger logsCount = new AtomicInteger();
123     private AtomicLong hfileArchiveSize = new AtomicLong();
124     private AtomicLong hfileSize = new AtomicLong();
125     private AtomicLong logSize = new AtomicLong();
126 
127     private final SnapshotDescription snapshot;
128     private final TableName snapshotTable;
129     private final Configuration conf;
130     private final FileSystem fs;
131 
132     SnapshotStats(final Configuration conf, final FileSystem fs, final SnapshotDescription snapshot)
133     {
134       this.snapshot = snapshot;
135       this.snapshotTable = TableName.valueOf(snapshot.getTable());
136       this.conf = conf;
137       this.fs = fs;
138     }
139 
140     /** @return the snapshot descriptor */
141     public SnapshotDescription getSnapshotDescription() {
142       return this.snapshot;
143     }
144 
145     /** @return true if the snapshot is corrupted */
146     public boolean isSnapshotCorrupted() {
147       return hfilesMissing.get() > 0 ||
148              logsMissing.get() > 0 ||
149              hfilesCorrupted.get() > 0;
150     }
151 
152     /** @return the number of available store files */
153     public int getStoreFilesCount() {
154       return hfilesCount.get() + hfileArchiveCount.get();
155     }
156 
157     /** @return the number of available store files in the archive */
158     public int getArchivedStoreFilesCount() {
159       return hfileArchiveCount.get();
160     }
161 
162     /** @return the number of available log files */
163     public int getLogsCount() {
164       return logsCount.get();
165     }
166 
167     /** @return the number of missing store files */
168     public int getMissingStoreFilesCount() {
169       return hfilesMissing.get();
170     }
171 
172     /** @return the number of corrupted store files */
173     public int getCorruptedStoreFilesCount() {
174       return hfilesCorrupted.get();
175     }
176 
177     /** @return the number of missing log files */
178     public int getMissingLogsCount() {
179       return logsMissing.get();
180     }
181 
182     /** @return the total size of the store files referenced by the snapshot */
183     public long getStoreFilesSize() {
184       return hfileSize.get() + hfileArchiveSize.get();
185     }
186 
187     /** @return the total size of the store files shared */
188     public long getSharedStoreFilesSize() {
189       return hfileSize.get();
190     }
191 
192     /** @return the total size of the store files in the archive */
193     public long getArchivedStoreFileSize() {
194       return hfileArchiveSize.get();
195     }
196 
197     /** @return the percentage of the shared store files */
198     public float getSharedStoreFilePercentage() {
199       return ((float)hfileSize.get() / (hfileSize.get() + hfileArchiveSize.get())) * 100;
200     }
201 
202     /** @return the total log size */
203     public long getLogsSize() {
204       return logSize.get();
205     }
206 
207     /**
208      * Add the specified store file to the stats
209      * @param region region encoded Name
210      * @param family family name
211      * @param storeFile store file name
212      * @return the store file information
213      */
214     FileInfo addStoreFile(final HRegionInfo region, final String family,
215         final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
216       HFileLink link = HFileLink.build(conf, snapshotTable, region.getEncodedName(),
217               family, storeFile.getName());
218       boolean isCorrupted = false;
219       boolean inArchive = false;
220       long size = -1;
221       try {
222         if ((inArchive = fs.exists(link.getArchivePath()))) {
223           size = fs.getFileStatus(link.getArchivePath()).getLen();
224           hfileArchiveSize.addAndGet(size);
225           hfileArchiveCount.incrementAndGet();
226         } else {
227           size = link.getFileStatus(fs).getLen();
228           hfileSize.addAndGet(size);
229           hfilesCount.incrementAndGet();
230         }
231         isCorrupted = (storeFile.hasFileSize() && storeFile.getFileSize() != size);
232         if (isCorrupted) hfilesCorrupted.incrementAndGet();
233       } catch (FileNotFoundException e) {
234         hfilesMissing.incrementAndGet();
235       }
236       return new FileInfo(inArchive, size, isCorrupted);
237     }
238 
239     /**
240      * Add the specified log file to the stats
241      * @param server server name
242      * @param logfile log file name
243      * @return the log information
244      */
245     FileInfo addLogFile(final String server, final String logfile) throws IOException {
246       WALLink logLink = new WALLink(conf, server, logfile);
247       long size = -1;
248       try {
249         size = logLink.getFileStatus(fs).getLen();
250         logSize.addAndGet(size);
251         logsCount.incrementAndGet();
252       } catch (FileNotFoundException e) {
253         logsMissing.incrementAndGet();
254       }
255       return new FileInfo(false, size, false);
256     }
257   }
258 
259   private boolean printSizeInBytes = false;
260   private FileSystem fs;
261   private Path rootDir;
262 
263   private SnapshotManifest snapshotManifest;
264 
265   @Override
266   public int run(String[] args) throws IOException, InterruptedException {
267     final Configuration conf = getConf();
268     boolean listSnapshots = false;
269     String snapshotName = null;
270     boolean showSchema = false;
271     boolean showFiles = false;
272     boolean showStats = false;
273 
274     // Process command line args
275     for (int i = 0; i < args.length; i++) {
276       String cmd = args[i];
277       try {
278         if (cmd.equals("-snapshot")) {
279           snapshotName = args[++i];
280         } else if (cmd.equals("-files")) {
281           showFiles = true;
282           showStats = true;
283         } else if (cmd.equals("-stats")) {
284           showStats = true;
285         } else if (cmd.equals("-schema")) {
286           showSchema = true;
287         } else if (cmd.equals("-remote-dir")) {
288           Path sourceDir = new Path(args[++i]);
289           URI defaultFs = sourceDir.getFileSystem(conf).getUri();
290           FSUtils.setFsDefault(conf, new Path(defaultFs));
291           FSUtils.setRootDir(conf, sourceDir);
292         } else if (cmd.equals("-list-snapshots")) {
293           listSnapshots = true;
294         } else if (cmd.equals("-size-in-bytes")) {
295           printSizeInBytes = true;
296         } else if (cmd.equals("-h") || cmd.equals("--help")) {
297           printUsageAndExit();
298         } else {
299           System.err.println("UNEXPECTED: " + cmd);
300           printUsageAndExit();
301         }
302       } catch (Exception e) {
303         printUsageAndExit();
304       }
305     }
306 
307     // List Available Snapshots
308     if (listSnapshots) {
309       SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss");
310       System.out.printf("%-20s | %-20s | %s%n", "SNAPSHOT", "CREATION TIME", "TABLE NAME");
311       for (SnapshotDescription desc: getSnapshotList(conf)) {
312         System.out.printf("%-20s | %20s | %s%n",
313                           desc.getName(),
314                           df.format(new Date(desc.getCreationTime())),
315                           desc.getTable());
316       }
317       return 0;
318     }
319 
320     if (snapshotName == null) {
321       System.err.println("Missing snapshot name!");
322       printUsageAndExit();
323       return 1;
324     }
325 
326     rootDir = FSUtils.getRootDir(conf);
327     fs = FileSystem.get(rootDir.toUri(), conf);
328     LOG.debug("fs=" + fs.getUri().toString() + " root=" + rootDir);
329 
330     // Load snapshot information
331     if (!loadSnapshotInfo(snapshotName)) {
332       System.err.println("Snapshot '" + snapshotName + "' not found!");
333       return 1;
334     }
335 
336     printInfo();
337     if (showSchema) printSchema();
338     printFiles(showFiles, showStats);
339 
340     return 0;
341   }
342 
343   /**
344    * Load snapshot info and table descriptor for the specified snapshot
345    * @param snapshotName name of the snapshot to load
346    * @return false if snapshot is not found
347    */
348   private boolean loadSnapshotInfo(final String snapshotName) throws IOException {
349     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
350     if (!fs.exists(snapshotDir)) {
351       LOG.warn("Snapshot '" + snapshotName + "' not found in: " + snapshotDir);
352       return false;
353     }
354 
355     SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
356     snapshotManifest = SnapshotManifest.open(getConf(), fs, snapshotDir, snapshotDesc);
357     return true;
358   }
359 
360   /**
361    * Dump the {@link SnapshotDescription}
362    */
363   private void printInfo() {
364     SnapshotDescription snapshotDesc = snapshotManifest.getSnapshotDescription();
365     SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss");
366     System.out.println("Snapshot Info");
367     System.out.println("----------------------------------------");
368     System.out.println("   Name: " + snapshotDesc.getName());
369     System.out.println("   Type: " + snapshotDesc.getType());
370     System.out.println("  Table: " + snapshotDesc.getTable());
371     System.out.println(" Format: " + snapshotDesc.getVersion());
372     System.out.println("Created: " + df.format(new Date(snapshotDesc.getCreationTime())));
373     System.out.println();
374   }
375 
376   /**
377    * Dump the {@link HTableDescriptor}
378    */
379   private void printSchema() {
380     System.out.println("Table Descriptor");
381     System.out.println("----------------------------------------");
382     System.out.println(snapshotManifest.getTableDescriptor().toString());
383     System.out.println();
384   }
385 
386   /**
387    * Collect the hfiles and logs statistics of the snapshot and
388    * dump the file list if requested and the collected information.
389    */
390   private void printFiles(final boolean showFiles, final boolean showStats) throws IOException {
391     if (showFiles) {
392       System.out.println("Snapshot Files");
393       System.out.println("----------------------------------------");
394     }
395 
396     // Collect information about hfiles and logs in the snapshot
397     final SnapshotDescription snapshotDesc = snapshotManifest.getSnapshotDescription();
398     final String table = snapshotDesc.getTable();
399     final SnapshotStats stats = new SnapshotStats(this.getConf(), this.fs, snapshotDesc);
400     SnapshotReferenceUtil.concurrentVisitReferencedFiles(getConf(), fs, snapshotManifest,
401       new SnapshotReferenceUtil.SnapshotVisitor() {
402         @Override
403         public void storeFile(final HRegionInfo regionInfo, final String family,
404             final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
405           if (storeFile.hasReference()) return;
406 
407           SnapshotStats.FileInfo info = stats.addStoreFile(regionInfo, family, storeFile);
408           if (showFiles) {
409             String state = info.getStateToString();
410             System.out.printf("%8s %s/%s/%s/%s %s%n",
411               (info.isMissing() ? "-" : fileSizeToString(info.getSize())),
412               table, regionInfo.getEncodedName(), family, storeFile.getName(),
413               state == null ? "" : "(" + state + ")");
414           }
415         }
416 
417         @Override
418         public void logFile (final String server, final String logfile)
419             throws IOException {
420           SnapshotStats.FileInfo info = stats.addLogFile(server, logfile);
421 
422           if (showFiles) {
423             String state = info.getStateToString();
424             System.out.printf("%8s log %s on server %s (%s)%n",
425               (info.isMissing() ? "-" : fileSizeToString(info.getSize())),
426               logfile, server,
427               state == null ? "" : "(" + state + ")");
428           }
429         }
430     });
431 
432     // Dump the stats
433     System.out.println();
434     if (stats.isSnapshotCorrupted()) {
435       System.out.println("**************************************************************");
436       System.out.printf("BAD SNAPSHOT: %d hfile(s) and %d log(s) missing.%n",
437         stats.getMissingStoreFilesCount(), stats.getMissingLogsCount());
438       System.out.printf("              %d hfile(s) corrupted.%n",
439         stats.getCorruptedStoreFilesCount());
440       System.out.println("**************************************************************");
441     }
442 
443     if (showStats) {
444       System.out.printf("%d HFiles (%d in archive), total size %s (%.2f%% %s shared with the source table)%n",
445         stats.getStoreFilesCount(), stats.getArchivedStoreFilesCount(),
446         fileSizeToString(stats.getStoreFilesSize()),
447         stats.getSharedStoreFilePercentage(),
448         fileSizeToString(stats.getSharedStoreFilesSize())
449       );
450       System.out.printf("%d Logs, total size %s%n",
451         stats.getLogsCount(), fileSizeToString(stats.getLogsSize()));
452       System.out.println();
453     }
454   }
455 
456   private String fileSizeToString(long size) {
457     return printSizeInBytes ? Long.toString(size) : StringUtils.humanReadableInt(size);
458   }
459 
460   private void printUsageAndExit() {
461     System.err.printf("Usage: bin/hbase %s [options]%n", getClass().getName());
462     System.err.println(" where [options] are:");
463     System.err.println("  -h|-help                Show this help and exit.");
464     System.err.println("  -remote-dir             Root directory that contains the snapshots.");
465     System.err.println("  -list-snapshots         List all the available snapshots and exit.");
466     System.err.println("  -size-in-bytes          Print the size of the files in bytes.");
467     System.err.println("  -snapshot NAME          Snapshot to examine.");
468     System.err.println("  -files                  Files and logs list.");
469     System.err.println("  -stats                  Files and logs stats.");
470     System.err.println("  -schema                 Describe the snapshotted table.");
471     System.err.println();
472     System.err.println("Examples:");
473     System.err.println("  hbase " + getClass() + " \\");
474     System.err.println("    -snapshot MySnapshot -files");
475     System.exit(1);
476   }
477 
478   /**
479    * Returns the snapshot stats
480    * @param conf the {@link Configuration} to use
481    * @param snapshot {@link SnapshotDescription} to get stats from
482    * @return the snapshot stats
483    */
484   public static SnapshotStats getSnapshotStats(final Configuration conf,
485       final SnapshotDescription snapshot) throws IOException {
486     Path rootDir = FSUtils.getRootDir(conf);
487     FileSystem fs = FileSystem.get(rootDir.toUri(), conf);
488     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
489     SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshot);
490     final SnapshotStats stats = new SnapshotStats(conf, fs, snapshot);
491     SnapshotReferenceUtil.concurrentVisitReferencedFiles(conf, fs, manifest,
492       new SnapshotReferenceUtil.SnapshotVisitor() {
493         @Override
494         public void storeFile(final HRegionInfo regionInfo, final String family,
495             final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
496           if (!storeFile.hasReference()) {
497             stats.addStoreFile(regionInfo, family, storeFile);
498           }
499         }
500 
501         @Override
502         public void logFile (final String server, final String logfile) throws IOException {
503           stats.addLogFile(server, logfile);
504         }
505     });
506     return stats;
507   }
508 
509   /**
510    * Returns the list of available snapshots in the specified location
511    * @param conf the {@link Configuration} to use
512    * @return the list of snapshots
513    */
514   public static List<SnapshotDescription> getSnapshotList(final Configuration conf)
515       throws IOException {
516     Path rootDir = FSUtils.getRootDir(conf);
517     FileSystem fs = FileSystem.get(rootDir.toUri(), conf);
518     Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
519     FileStatus[] snapshots = fs.listStatus(snapshotDir,
520       new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
521     List<SnapshotDescription> snapshotLists =
522       new ArrayList<SnapshotDescription>(snapshots.length);
523     for (FileStatus snapshotDirStat: snapshots) {
524       snapshotLists.add(SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDirStat.getPath()));
525     }
526     return snapshotLists;
527   }
528 
529   /**
530    * The guts of the {@link #main} method.
531    * Call this method to avoid the {@link #main(String[])} System.exit.
532    * @param args
533    * @return errCode
534    * @throws Exception
535    */
536   static int innerMain(final String [] args) throws Exception {
537     return ToolRunner.run(HBaseConfiguration.create(), new SnapshotInfo(), args);
538   }
539 
540   public static void main(String[] args) throws Exception {
541      System.exit(innerMain(args));
542   }
543 }