001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.snapshot;
020
021import java.io.FileNotFoundException;
022import java.io.IOException;
023import java.net.URI;
024import java.text.SimpleDateFormat;
025import java.util.ArrayList;
026import java.util.Collections;
027import java.util.Date;
028import java.util.List;
029import java.util.Map;
030import java.util.concurrent.ConcurrentHashMap;
031import java.util.concurrent.ExecutorService;
032import java.util.concurrent.atomic.AtomicInteger;
033import java.util.concurrent.atomic.AtomicLong;
034
035import org.apache.hadoop.conf.Configuration;
036import org.apache.hadoop.fs.FileStatus;
037import org.apache.hadoop.fs.FileSystem;
038import org.apache.hadoop.fs.Path;
039import org.apache.hadoop.hbase.TableName;
040import org.apache.hadoop.hbase.client.RegionInfo;
041import org.apache.hadoop.hbase.client.SnapshotDescription;
042import org.apache.hadoop.hbase.io.HFileLink;
043import org.apache.hadoop.hbase.io.WALLink;
044import org.apache.hadoop.hbase.util.AbstractHBaseTool;
045import org.apache.hadoop.hbase.util.FSUtils;
046import org.apache.hadoop.util.StringUtils;
047import org.apache.yetus.audience.InterfaceAudience;
048import org.slf4j.Logger;
049import org.slf4j.LoggerFactory;
050import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
051import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
052import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
053import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
054import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
055
056/**
057 * Tool for dumping snapshot information.
058 * <ol>
059 * <li> Table Descriptor
060 * <li> Snapshot creation time, type, format version, ...
061 * <li> List of hfiles and wals
062 * <li> Stats about hfiles and logs sizes, percentage of shared with the source table, ...
063 * </ol>
064 */
065@InterfaceAudience.Public
066public final class SnapshotInfo extends AbstractHBaseTool {
067  private static final Logger LOG = LoggerFactory.getLogger(SnapshotInfo.class);
068
069  static final class Options {
070    static final Option SNAPSHOT = new Option(null, "snapshot", true, "Snapshot to examine.");
071    static final Option REMOTE_DIR = new Option(null, "remote-dir", true,
072        "Root directory that contains the snapshots.");
073    static final Option LIST_SNAPSHOTS = new Option(null, "list-snapshots", false,
074        "List all the available snapshots and exit.");
075    static final Option FILES = new Option(null, "files", false, "Files and logs list.");
076    static final Option STATS = new Option(null, "stats", false, "Files and logs stats.");
077    static final Option SCHEMA = new Option(null, "schema", false,
078        "Describe the snapshotted table.");
079    static final Option SIZE_IN_BYTES = new Option(null, "size-in-bytes", false,
080        "Print the size of the files in bytes.");
081  }
082
083  /**
084   * Statistics about the snapshot
085   * <ol>
086   * <li> How many store files and logs are in the archive
087   * <li> How many store files and logs are shared with the table
088   * <li> Total store files and logs size and shared amount
089   * </ol>
090   */
091  public static class SnapshotStats {
092    /** Information about the file referenced by the snapshot */
093    static class FileInfo {
094      private final boolean corrupted;
095      private final boolean inArchive;
096      private final long size;
097
098      FileInfo(final boolean inArchive, final long size, final boolean corrupted) {
099        this.corrupted = corrupted;
100        this.inArchive = inArchive;
101        this.size = size;
102      }
103
104      /** @return true if the file is in the archive */
105      public boolean inArchive() {
106        return this.inArchive;
107      }
108
109      /** @return true if the file is corrupted */
110      public boolean isCorrupted() {
111        return this.corrupted;
112      }
113
114      /** @return true if the file is missing */
115      public boolean isMissing() {
116        return this.size < 0;
117      }
118
119      /** @return the file size */
120      public long getSize() {
121        return this.size;
122      }
123
124      String getStateToString() {
125        if (isCorrupted()) return "CORRUPTED";
126        if (isMissing()) return "NOT FOUND";
127        if (inArchive()) return "archive";
128        return null;
129      }
130    }
131
132    private AtomicInteger hfilesArchiveCount = new AtomicInteger();
133    private AtomicInteger hfilesCorrupted = new AtomicInteger();
134    private AtomicInteger hfilesMissing = new AtomicInteger();
135    private AtomicInteger hfilesCount = new AtomicInteger();
136    private AtomicInteger hfilesMobCount = new AtomicInteger();
137    private AtomicInteger logsMissing = new AtomicInteger();
138    private AtomicInteger logsCount = new AtomicInteger();
139    private AtomicLong hfilesArchiveSize = new AtomicLong();
140    private AtomicLong hfilesSize = new AtomicLong();
141    private AtomicLong hfilesMobSize = new AtomicLong();
142    private AtomicLong nonSharedHfilesArchiveSize = new AtomicLong();
143    private AtomicLong logSize = new AtomicLong();
144
145    private final SnapshotProtos.SnapshotDescription snapshot;
146    private final TableName snapshotTable;
147    private final Configuration conf;
148    private final FileSystem fs;
149
150    SnapshotStats(final Configuration conf, final FileSystem fs,
151        final SnapshotDescription snapshot)
152    {
153      this.snapshot = ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot);
154      this.snapshotTable = snapshot.getTableName();
155      this.conf = conf;
156      this.fs = fs;
157    }
158
159    SnapshotStats(final Configuration conf, final FileSystem fs,
160        final SnapshotProtos.SnapshotDescription snapshot) {
161      this.snapshot = snapshot;
162      this.snapshotTable = TableName.valueOf(snapshot.getTable());
163      this.conf = conf;
164      this.fs = fs;
165    }
166
167
168    /** @return the snapshot descriptor */
169    public SnapshotDescription getSnapshotDescription() {
170      return ProtobufUtil.createSnapshotDesc(this.snapshot);
171    }
172
173    /** @return true if the snapshot is corrupted */
174    public boolean isSnapshotCorrupted() {
175      return hfilesMissing.get() > 0 ||
176             logsMissing.get() > 0 ||
177             hfilesCorrupted.get() > 0;
178    }
179
180    /** @return the number of available store files */
181    public int getStoreFilesCount() {
182      return hfilesCount.get() + hfilesArchiveCount.get() + hfilesMobCount.get();
183    }
184
185    /** @return the number of available store files in the archive */
186    public int getArchivedStoreFilesCount() {
187      return hfilesArchiveCount.get();
188    }
189
190    /** @return the number of available store files in the mob dir */
191    public int getMobStoreFilesCount() { return hfilesMobCount.get(); }
192
193    /** @return the number of available log files */
194    public int getLogsCount() {
195      return logsCount.get();
196    }
197
198    /** @return the number of missing store files */
199    public int getMissingStoreFilesCount() {
200      return hfilesMissing.get();
201    }
202
203    /** @return the number of corrupted store files */
204    public int getCorruptedStoreFilesCount() {
205      return hfilesCorrupted.get();
206    }
207
208    /** @return the number of missing log files */
209    public int getMissingLogsCount() {
210      return logsMissing.get();
211    }
212
213    /** @return the total size of the store files referenced by the snapshot */
214    public long getStoreFilesSize() {
215      return hfilesSize.get() + hfilesArchiveSize.get() + hfilesMobSize.get();
216    }
217
218    /** @return the total size of the store files shared */
219    public long getSharedStoreFilesSize() {
220      return hfilesSize.get();
221    }
222
223    /** @return the total size of the store files in the archive */
224    public long getArchivedStoreFileSize() {
225      return hfilesArchiveSize.get();
226    }
227
228    /** @return the total size of the store files in the mob store*/
229    public long getMobStoreFilesSize() { return hfilesMobSize.get(); }
230
231    /** @return the total size of the store files in the archive which is not shared
232     *    with other snapshots and tables
233     *
234     *    This is only calculated when
235     *  {@link #getSnapshotStats(Configuration, SnapshotProtos.SnapshotDescription, Map)}
236     *    is called with a non-null Map
237     */
238    public long getNonSharedArchivedStoreFilesSize() {
239      return nonSharedHfilesArchiveSize.get();
240    }
241
242    /** @return the percentage of the shared store files */
243    public float getSharedStoreFilePercentage() {
244      return ((float) hfilesSize.get() / (getStoreFilesSize())) * 100;
245    }
246
247    /** @return the percentage of the mob store files */
248    public float getMobStoreFilePercentage() {
249      return ((float) hfilesMobSize.get() / (getStoreFilesSize())) * 100;
250    }
251
252    /** @return the total log size */
253    public long getLogsSize() {
254      return logSize.get();
255    }
256
257    /** Check if for a give file in archive, if there are other snapshots/tables still
258     * reference it.
259     * @param filePath file path in archive
260     * @param snapshotFilesMap a map for store files in snapshots about how many snapshots refer
261     *                         to it.
262     * @return true or false
263     */
264    private boolean isArchivedFileStillReferenced(final Path filePath,
265        final Map<Path, Integer> snapshotFilesMap) {
266
267      Integer c = snapshotFilesMap.get(filePath);
268
269      // Check if there are other snapshots or table from clone_snapshot() (via back-reference)
270      // still reference to it.
271      if ((c != null) && (c == 1)) {
272        Path parentDir = filePath.getParent();
273        Path backRefDir = HFileLink.getBackReferencesDir(parentDir, filePath.getName());
274        try {
275          if (FSUtils.listStatus(fs, backRefDir) == null) {
276            return false;
277          }
278        } catch (IOException e) {
279          // For the purpose of this function, IOException is ignored and treated as
280          // the file is still being referenced.
281        }
282      }
283      return true;
284    }
285
286    /**
287     * Add the specified store file to the stats
288     * @param region region encoded Name
289     * @param family family name
290     * @param storeFile store file name
291     * @param filesMap store files map for all snapshots, it may be null
292     * @return the store file information
293     */
294    FileInfo addStoreFile(final RegionInfo region, final String family,
295        final SnapshotRegionManifest.StoreFile storeFile,
296        final Map<Path, Integer> filesMap) throws IOException {
297      HFileLink link = HFileLink.build(conf, snapshotTable, region.getEncodedName(),
298              family, storeFile.getName());
299      boolean isCorrupted = false;
300      boolean inArchive = false;
301      long size = -1;
302      try {
303        if (fs.exists(link.getArchivePath())) {
304          inArchive = true;
305          size = fs.getFileStatus(link.getArchivePath()).getLen();
306          hfilesArchiveSize.addAndGet(size);
307          hfilesArchiveCount.incrementAndGet();
308
309          // If store file is not shared with other snapshots and tables,
310          // increase nonSharedHfilesArchiveSize
311          if ((filesMap != null) &&
312              !isArchivedFileStillReferenced(link.getArchivePath(), filesMap)) {
313            nonSharedHfilesArchiveSize.addAndGet(size);
314          }
315        } else if (fs.exists(link.getMobPath())) {
316          inArchive = true;
317          size = fs.getFileStatus(link.getMobPath()).getLen();
318          hfilesMobSize.addAndGet(size);
319          hfilesMobCount.incrementAndGet();
320        } else {
321          size = link.getFileStatus(fs).getLen();
322          hfilesSize.addAndGet(size);
323          hfilesCount.incrementAndGet();
324        }
325        isCorrupted = (storeFile.hasFileSize() && storeFile.getFileSize() != size);
326        if (isCorrupted) hfilesCorrupted.incrementAndGet();
327      } catch (FileNotFoundException e) {
328        hfilesMissing.incrementAndGet();
329      }
330      return new FileInfo(inArchive, size, isCorrupted);
331    }
332
333    /**
334     * Add the specified log file to the stats
335     * @param server server name
336     * @param logfile log file name
337     * @return the log information
338     */
339    FileInfo addLogFile(final String server, final String logfile) throws IOException {
340      WALLink logLink = new WALLink(conf, server, logfile);
341      long size = -1;
342      try {
343        size = logLink.getFileStatus(fs).getLen();
344        logSize.addAndGet(size);
345        logsCount.incrementAndGet();
346      } catch (FileNotFoundException e) {
347        logsMissing.incrementAndGet();
348      }
349      return new FileInfo(false, size, false);
350    }
351  }
352
353  private FileSystem fs;
354  private Path rootDir;
355
356  private SnapshotManifest snapshotManifest;
357
358  private boolean listSnapshots = false;
359  private String snapshotName;
360  private Path remoteDir;
361  private boolean showSchema = false;
362  private boolean showFiles = false;
363  private boolean showStats = false;
364  private boolean printSizeInBytes = false;
365
366  @Override
367  public int doWork() throws IOException, InterruptedException {
368    if (remoteDir != null) {
369      URI defaultFs = remoteDir.getFileSystem(conf).getUri();
370      FSUtils.setFsDefault(conf, new Path(defaultFs));
371      FSUtils.setRootDir(conf, remoteDir);
372    }
373
374    // List Available Snapshots
375    if (listSnapshots) {
376      SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss");
377      System.out.printf("%-20s | %-20s | %s%n", "SNAPSHOT", "CREATION TIME", "TABLE NAME");
378      for (SnapshotDescription desc: getSnapshotList(conf)) {
379        System.out.printf("%-20s | %20s | %s%n",
380                          desc.getName(),
381                          df.format(new Date(desc.getCreationTime())),
382                          desc.getTableNameAsString());
383      }
384      return 0;
385    }
386
387    rootDir = FSUtils.getRootDir(conf);
388    fs = FileSystem.get(rootDir.toUri(), conf);
389    LOG.debug("fs=" + fs.getUri().toString() + " root=" + rootDir);
390
391    // Load snapshot information
392    if (!loadSnapshotInfo(snapshotName)) {
393      System.err.println("Snapshot '" + snapshotName + "' not found!");
394      return 1;
395    }
396
397    printInfo();
398    if (showSchema) printSchema();
399    printFiles(showFiles, showStats);
400
401    return 0;
402  }
403
404  /**
405   * Load snapshot info and table descriptor for the specified snapshot
406   * @param snapshotName name of the snapshot to load
407   * @return false if snapshot is not found
408   */
409  private boolean loadSnapshotInfo(final String snapshotName) throws IOException {
410    Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
411    if (!fs.exists(snapshotDir)) {
412      LOG.warn("Snapshot '" + snapshotName + "' not found in: " + snapshotDir);
413      return false;
414    }
415
416    SnapshotProtos.SnapshotDescription snapshotDesc =
417        SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
418    snapshotManifest = SnapshotManifest.open(getConf(), fs, snapshotDir, snapshotDesc);
419    return true;
420  }
421
422  /**
423   * Dump the {@link SnapshotDescription}
424   */
425  private void printInfo() {
426    SnapshotProtos.SnapshotDescription snapshotDesc = snapshotManifest.getSnapshotDescription();
427    SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss");
428    System.out.println("Snapshot Info");
429    System.out.println("----------------------------------------");
430    System.out.println("   Name: " + snapshotDesc.getName());
431    System.out.println("   Type: " + snapshotDesc.getType());
432    System.out.println("  Table: " + snapshotDesc.getTable());
433    System.out.println(" Format: " + snapshotDesc.getVersion());
434    System.out.println("Created: " + df.format(new Date(snapshotDesc.getCreationTime())));
435    System.out.println("  Owner: " + snapshotDesc.getOwner());
436    System.out.println();
437  }
438
439  /**
440   * Dump the {@link org.apache.hadoop.hbase.client.TableDescriptor}
441   */
442  private void printSchema() {
443    System.out.println("Table Descriptor");
444    System.out.println("----------------------------------------");
445    System.out.println(snapshotManifest.getTableDescriptor().toString());
446    System.out.println();
447  }
448
449  /**
450   * Collect the hfiles and logs statistics of the snapshot and
451   * dump the file list if requested and the collected information.
452   */
453  private void printFiles(final boolean showFiles, final boolean showStats) throws IOException {
454    if (showFiles) {
455      System.out.println("Snapshot Files");
456      System.out.println("----------------------------------------");
457    }
458
459    // Collect information about hfiles and logs in the snapshot
460    final SnapshotProtos.SnapshotDescription snapshotDesc = snapshotManifest.getSnapshotDescription();
461    final String table = snapshotDesc.getTable();
462    final SnapshotDescription desc = ProtobufUtil.createSnapshotDesc(snapshotDesc);
463    final SnapshotStats stats = new SnapshotStats(this.getConf(), this.fs, desc);
464    SnapshotReferenceUtil.concurrentVisitReferencedFiles(getConf(), fs, snapshotManifest,
465        "SnapshotInfo",
466      new SnapshotReferenceUtil.SnapshotVisitor() {
467        @Override
468        public void storeFile(final RegionInfo regionInfo, final String family,
469            final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
470          if (storeFile.hasReference()) return;
471
472          SnapshotStats.FileInfo info = stats.addStoreFile(regionInfo, family, storeFile, null);
473          if (showFiles) {
474            String state = info.getStateToString();
475            System.out.printf("%8s %s/%s/%s/%s %s%n",
476              (info.isMissing() ? "-" : fileSizeToString(info.getSize())),
477              table, regionInfo.getEncodedName(), family, storeFile.getName(),
478              state == null ? "" : "(" + state + ")");
479          }
480        }
481    });
482
483    // Dump the stats
484    System.out.println();
485    if (stats.isSnapshotCorrupted()) {
486      System.out.println("**************************************************************");
487      System.out.printf("BAD SNAPSHOT: %d hfile(s) and %d log(s) missing.%n",
488        stats.getMissingStoreFilesCount(), stats.getMissingLogsCount());
489      System.out.printf("              %d hfile(s) corrupted.%n",
490        stats.getCorruptedStoreFilesCount());
491      System.out.println("**************************************************************");
492    }
493
494    if (showStats) {
495      System.out.printf("%d HFiles (%d in archive, %d in mob storage), total size %s " +
496              "(%.2f%% %s shared with the source table, %.2f%% %s in mob dir)%n",
497        stats.getStoreFilesCount(), stats.getArchivedStoreFilesCount(),
498        stats.getMobStoreFilesCount(),
499        fileSizeToString(stats.getStoreFilesSize()),
500        stats.getSharedStoreFilePercentage(),
501        fileSizeToString(stats.getSharedStoreFilesSize()),
502        stats.getMobStoreFilePercentage(),
503        fileSizeToString(stats.getMobStoreFilesSize())
504      );
505      System.out.printf("%d Logs, total size %s%n",
506        stats.getLogsCount(), fileSizeToString(stats.getLogsSize()));
507      System.out.println();
508    }
509  }
510
511  private String fileSizeToString(long size) {
512    return printSizeInBytes ? Long.toString(size) : StringUtils.humanReadableInt(size);
513  }
514
515  @Override
516  protected void addOptions() {
517    addRequiredOption(Options.SNAPSHOT);
518    addOption(Options.REMOTE_DIR);
519    addOption(Options.LIST_SNAPSHOTS);
520    addOption(Options.FILES);
521    addOption(Options.STATS);
522    addOption(Options.SCHEMA);
523    addOption(Options.SIZE_IN_BYTES);
524  }
525
526  @Override
527  protected void processOptions(CommandLine cmd) {
528    snapshotName = cmd.getOptionValue(Options.SNAPSHOT.getLongOpt());
529    showFiles = cmd.hasOption(Options.FILES.getLongOpt());
530    showStats = cmd.hasOption(Options.FILES.getLongOpt())
531        || cmd.hasOption(Options.STATS.getLongOpt());
532    showSchema = cmd.hasOption(Options.SCHEMA.getLongOpt());
533    listSnapshots = cmd.hasOption(Options.LIST_SNAPSHOTS.getLongOpt());
534    printSizeInBytes = cmd.hasOption(Options.SIZE_IN_BYTES.getLongOpt());
535    if (cmd.hasOption(Options.REMOTE_DIR.getLongOpt())) {
536      remoteDir = new Path(cmd.getOptionValue(Options.REMOTE_DIR.getLongOpt()));
537    }
538  }
539
540  @Override
541  protected void printUsage() {
542    printUsage("hbase snapshot info [options]", "Options:", "");
543    System.err.println("Examples:");
544    System.err.println("  hbase snapshot info --snapshot MySnapshot --files");
545  }
546
547  /**
548   * Returns the snapshot stats
549   * @param conf the {@link Configuration} to use
550   * @param snapshot {@link SnapshotDescription} to get stats from
551   * @return the snapshot stats
552   */
553  public static SnapshotStats getSnapshotStats(final Configuration conf,
554      final SnapshotDescription snapshot) throws IOException {
555    SnapshotProtos.SnapshotDescription snapshotDesc =
556      ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot);
557    return getSnapshotStats(conf, snapshotDesc, null);
558  }
559
560  /**
561   * Returns the snapshot stats
562   * @param conf the {@link Configuration} to use
563   * @param snapshotDesc  HBaseProtos.SnapshotDescription to get stats from
564   * @param filesMap {@link Map} store files map for all snapshots, it may be null
565   * @return the snapshot stats
566   */
567  public static SnapshotStats getSnapshotStats(final Configuration conf,
568      final SnapshotProtos.SnapshotDescription snapshotDesc,
569      final Map<Path, Integer> filesMap) throws IOException {
570    Path rootDir = FSUtils.getRootDir(conf);
571    FileSystem fs = FileSystem.get(rootDir.toUri(), conf);
572    Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotDesc, rootDir);
573    SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
574    final SnapshotStats stats = new SnapshotStats(conf, fs, snapshotDesc);
575    SnapshotReferenceUtil.concurrentVisitReferencedFiles(conf, fs, manifest,
576        "SnapshotsStatsAggregation", new SnapshotReferenceUtil.SnapshotVisitor() {
577          @Override
578          public void storeFile(final RegionInfo regionInfo, final String family,
579              final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
580            if (!storeFile.hasReference()) {
581              stats.addStoreFile(regionInfo, family, storeFile, filesMap);
582            }
583          }});
584    return stats;
585  }
586
587  /**
588   * Returns the list of available snapshots in the specified location
589   * @param conf the {@link Configuration} to use
590   * @return the list of snapshots
591   */
592  public static List<SnapshotDescription> getSnapshotList(final Configuration conf)
593      throws IOException {
594    Path rootDir = FSUtils.getRootDir(conf);
595    FileSystem fs = FileSystem.get(rootDir.toUri(), conf);
596    Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
597    FileStatus[] snapshots = fs.listStatus(snapshotDir,
598        new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
599    List<SnapshotDescription> snapshotLists = new ArrayList<>(snapshots.length);
600    for (FileStatus snapshotDirStat: snapshots) {
601      SnapshotProtos.SnapshotDescription snapshotDesc =
602          SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDirStat.getPath());
603      snapshotLists.add(ProtobufUtil.createSnapshotDesc(snapshotDesc));
604    }
605    return snapshotLists;
606  }
607
608  /**
609   * Gets the store files map for snapshot
610   * @param conf the {@link Configuration} to use
611   * @param snapshot {@link SnapshotDescription} to get stats from
612   * @param exec the {@link ExecutorService} to use
613   * @param filesMap {@link Map} the map to put the mapping entries
614   * @param uniqueHFilesArchiveSize {@link AtomicLong} the accumulated store file size in archive
615   * @param uniqueHFilesSize {@link AtomicLong} the accumulated store file size shared
616   * @param uniqueHFilesMobSize {@link AtomicLong} the accumulated mob store file size shared
617   */
618  private static void getSnapshotFilesMap(final Configuration conf,
619      final SnapshotDescription snapshot, final ExecutorService exec,
620      final ConcurrentHashMap<Path, Integer> filesMap,
621      final AtomicLong uniqueHFilesArchiveSize, final AtomicLong uniqueHFilesSize,
622      final AtomicLong uniqueHFilesMobSize) throws IOException {
623    SnapshotProtos.SnapshotDescription snapshotDesc =
624        ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot);
625    Path rootDir = FSUtils.getRootDir(conf);
626    final FileSystem fs = FileSystem.get(rootDir.toUri(), conf);
627
628    Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotDesc, rootDir);
629    SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
630    SnapshotReferenceUtil.concurrentVisitReferencedFiles(conf, fs, manifest, exec,
631        new SnapshotReferenceUtil.SnapshotVisitor() {
632          @Override public void storeFile(final RegionInfo regionInfo, final String family,
633              final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
634            if (!storeFile.hasReference()) {
635              HFileLink link = HFileLink.build(conf, snapshot.getTableName(),
636                  regionInfo.getEncodedName(), family, storeFile.getName());
637              long size;
638              Integer count;
639              Path p;
640              AtomicLong al;
641              int c = 0;
642
643              if (fs.exists(link.getArchivePath())) {
644                p = link.getArchivePath();
645                al = uniqueHFilesArchiveSize;
646                size = fs.getFileStatus(p).getLen();
647              } else if (fs.exists(link.getMobPath())) {
648                p = link.getMobPath();
649                al = uniqueHFilesMobSize;
650                size = fs.getFileStatus(p).getLen();
651              } else {
652                p = link.getOriginPath();
653                al = uniqueHFilesSize;
654                size = link.getFileStatus(fs).getLen();
655              }
656
657              // If it has been counted, do not double count
658              count = filesMap.get(p);
659              if (count != null) {
660                c = count.intValue();
661              } else {
662                al.addAndGet(size);
663              }
664
665              filesMap.put(p, ++c);
666            }
667          }
668        });
669  }
670
671  /**
672   * Returns the map of store files based on path for all snapshots
673   * @param conf the {@link Configuration} to use
674   * @param uniqueHFilesArchiveSize pass out the size for store files in archive
675   * @param uniqueHFilesSize pass out the size for store files shared
676   * @param uniqueHFilesMobSize pass out the size for mob store files shared
677   * @return the map of store files
678   */
679  public static Map<Path, Integer> getSnapshotsFilesMap(final Configuration conf,
680      AtomicLong uniqueHFilesArchiveSize, AtomicLong uniqueHFilesSize,
681      AtomicLong uniqueHFilesMobSize) throws IOException {
682    List<SnapshotDescription> snapshotList = getSnapshotList(conf);
683
684
685    if (snapshotList.isEmpty()) {
686      return Collections.emptyMap();
687    }
688
689    ConcurrentHashMap<Path, Integer> fileMap = new ConcurrentHashMap<>();
690
691    ExecutorService exec = SnapshotManifest.createExecutor(conf, "SnapshotsFilesMapping");
692
693    try {
694      for (final SnapshotDescription snapshot : snapshotList) {
695        getSnapshotFilesMap(conf, snapshot, exec, fileMap, uniqueHFilesArchiveSize,
696            uniqueHFilesSize, uniqueHFilesMobSize);
697      }
698    } finally {
699      exec.shutdown();
700    }
701
702    return fileMap;
703  }
704
705
706  public static void main(String[] args) {
707    new SnapshotInfo().doStaticMain(args);
708  }
709}