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.util.ArrayList;
23  import java.util.Collections;
24  import java.util.Comparator;
25  import java.util.LinkedList;
26  import java.util.List;
27  
28  import org.apache.commons.logging.Log;
29  import org.apache.commons.logging.LogFactory;
30  import org.apache.hadoop.classification.InterfaceAudience;
31  import org.apache.hadoop.classification.InterfaceStability;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.conf.Configured;
34  import org.apache.hadoop.fs.FSDataInputStream;
35  import org.apache.hadoop.fs.FSDataOutputStream;
36  import org.apache.hadoop.fs.FileChecksum;
37  import org.apache.hadoop.fs.FileStatus;
38  import org.apache.hadoop.fs.FileSystem;
39  import org.apache.hadoop.fs.FileUtil;
40  import org.apache.hadoop.fs.Path;
41  import org.apache.hadoop.fs.permission.FsPermission;
42  import org.apache.hadoop.hbase.TableName;
43  import org.apache.hadoop.hbase.HBaseConfiguration;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.io.HFileLink;
46  import org.apache.hadoop.hbase.io.HLogLink;
47  import org.apache.hadoop.hbase.mapreduce.JobUtil;
48  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
49  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
50  import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
51  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
52  import org.apache.hadoop.hbase.util.FSUtils;
53  import org.apache.hadoop.hbase.util.Pair;
54  import org.apache.hadoop.io.NullWritable;
55  import org.apache.hadoop.io.SequenceFile;
56  import org.apache.hadoop.io.Text;
57  import org.apache.hadoop.mapreduce.Job;
58  import org.apache.hadoop.mapreduce.Mapper;
59  import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
60  import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
61  import org.apache.hadoop.util.StringUtils;
62  import org.apache.hadoop.util.Tool;
63  import org.apache.hadoop.util.ToolRunner;
64  
65  /**
66   * Export the specified snapshot to a given FileSystem.
67   *
68   * The .snapshot/name folder is copied to the destination cluster
69   * and then all the hfiles/hlogs are copied using a Map-Reduce Job in the .archive/ location.
70   * When everything is done, the second cluster can restore the snapshot.
71   */
72  @InterfaceAudience.Public
73  @InterfaceStability.Evolving
74  public final class ExportSnapshot extends Configured implements Tool {
75    private static final Log LOG = LogFactory.getLog(ExportSnapshot.class);
76  
77    private static final String CONF_FILES_USER = "snapshot.export.files.attributes.user";
78    private static final String CONF_FILES_GROUP = "snapshot.export.files.attributes.group";
79    private static final String CONF_FILES_MODE = "snapshot.export.files.attributes.mode";
80    private static final String CONF_CHECKSUM_VERIFY = "snapshot.export.checksum.verify";
81    private static final String CONF_OUTPUT_ROOT = "snapshot.export.output.root";
82    private static final String CONF_INPUT_ROOT = "snapshot.export.input.root";
83  
84    private static final String INPUT_FOLDER_PREFIX = "export-files.";
85  
86    // Export Map-Reduce Counters, to keep track of the progress
87    public enum Counter { MISSING_FILES, COPY_FAILED, BYTES_EXPECTED, BYTES_COPIED };
88  
89    private static class ExportMapper extends Mapper<Text, NullWritable, NullWritable, NullWritable> {
90      final static int REPORT_SIZE = 1 * 1024 * 1024;
91      final static int BUFFER_SIZE = 64 * 1024;
92  
93      private boolean verifyChecksum;
94      private String filesGroup;
95      private String filesUser;
96      private short filesMode;
97  
98      private FileSystem outputFs;
99      private Path outputArchive;
100     private Path outputRoot;
101 
102     private FileSystem inputFs;
103     private Path inputArchive;
104     private Path inputRoot;
105 
106     @Override
107     public void setup(Context context) {
108       Configuration conf = context.getConfiguration();
109       verifyChecksum = conf.getBoolean(CONF_CHECKSUM_VERIFY, true);
110 
111       filesGroup = conf.get(CONF_FILES_GROUP);
112       filesUser = conf.get(CONF_FILES_USER);
113       filesMode = (short)conf.getInt(CONF_FILES_MODE, 0);
114       outputRoot = new Path(conf.get(CONF_OUTPUT_ROOT));
115       inputRoot = new Path(conf.get(CONF_INPUT_ROOT));
116 
117       inputArchive = new Path(inputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
118       outputArchive = new Path(outputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
119 
120       try {
121         inputFs = FileSystem.get(inputRoot.toUri(), conf);
122       } catch (IOException e) {
123         throw new RuntimeException("Could not get the input FileSystem with root=" + inputRoot, e);
124       }
125 
126       try {
127         outputFs = FileSystem.get(outputRoot.toUri(), conf);
128       } catch (IOException e) {
129         throw new RuntimeException("Could not get the output FileSystem with root="+ outputRoot, e);
130       }
131     }
132 
133     @Override
134     public void map(Text key, NullWritable value, Context context)
135         throws InterruptedException, IOException {
136       Path inputPath = new Path(key.toString());
137       Path outputPath = getOutputPath(inputPath);
138 
139       LOG.info("copy file input=" + inputPath + " output=" + outputPath);
140       if (copyFile(context, inputPath, outputPath)) {
141         LOG.info("copy completed for input=" + inputPath + " output=" + outputPath);
142       }
143     }
144 
145     /**
146      * Returns the location where the inputPath will be copied.
147      *  - hfiles are encoded as hfile links hfile-region-table
148      *  - logs are encoded as serverName/logName
149      */
150     private Path getOutputPath(final Path inputPath) throws IOException {
151       Path path;
152       if (HFileLink.isHFileLink(inputPath) || StoreFileInfo.isReference(inputPath)) {
153         String family = inputPath.getParent().getName();
154         TableName table =
155             HFileLink.getReferencedTableName(inputPath.getName());
156         String region = HFileLink.getReferencedRegionName(inputPath.getName());
157         String hfile = HFileLink.getReferencedHFileName(inputPath.getName());
158         path = new Path(FSUtils.getTableDir(new Path("./"), table),
159             new Path(region, new Path(family, hfile)));
160       } else if (isHLogLinkPath(inputPath)) {
161         String logName = inputPath.getName();
162         path = new Path(new Path(outputRoot, HConstants.HREGION_OLDLOGDIR_NAME), logName);
163       } else {
164         path = inputPath;
165       }
166       return new Path(outputArchive, path);
167     }
168 
169     private boolean copyFile(final Context context, final Path inputPath, final Path outputPath)
170         throws IOException {
171       FSDataInputStream in = openSourceFile(inputPath);
172       if (in == null) {
173         context.getCounter(Counter.MISSING_FILES).increment(1);
174         return false;
175       }
176 
177       try {
178         // Verify if the input file exists
179         FileStatus inputStat = getFileStatus(inputFs, inputPath);
180         if (inputStat == null) return false;
181 
182         // Verify if the output file exists and is the same that we want to copy
183         if (outputFs.exists(outputPath)) {
184           FileStatus outputStat = outputFs.getFileStatus(outputPath);
185           if (sameFile(inputStat, outputStat)) {
186             LOG.info("Skip copy " + inputPath + " to " + outputPath + ", same file.");
187             return true;
188           }
189         }
190 
191         context.getCounter(Counter.BYTES_EXPECTED).increment(inputStat.getLen());
192 
193         // Ensure that the output folder is there and copy the file
194         outputFs.mkdirs(outputPath.getParent());
195         FSDataOutputStream out = outputFs.create(outputPath, true);
196         try {
197           if (!copyData(context, inputPath, in, outputPath, out, inputStat.getLen()))
198             return false;
199         } finally {
200           out.close();
201         }
202 
203         // Preserve attributes
204         return preserveAttributes(outputPath, inputStat);
205       } finally {
206         in.close();
207       }
208     }
209 
210     /**
211      * Preserve the files attribute selected by the user copying them from the source file
212      */
213     private boolean preserveAttributes(final Path path, final FileStatus refStat) {
214       FileStatus stat;
215       try {
216         stat = outputFs.getFileStatus(path);
217       } catch (IOException e) {
218         LOG.warn("Unable to get the status for file=" + path);
219         return false;
220       }
221 
222       try {
223         if (filesMode > 0 && stat.getPermission().toShort() != filesMode) {
224           outputFs.setPermission(path, new FsPermission(filesMode));
225         } else if (!stat.getPermission().equals(refStat.getPermission())) {
226           outputFs.setPermission(path, refStat.getPermission());
227         }
228       } catch (IOException e) {
229         LOG.error("Unable to set the permission for file=" + path, e);
230         return false;
231       }
232 
233       try {
234         String user = (filesUser != null) ? filesUser : refStat.getOwner();
235         String group = (filesGroup != null) ? filesGroup : refStat.getGroup();
236         if (!(user.equals(stat.getOwner()) && group.equals(stat.getGroup()))) {
237           outputFs.setOwner(path, user, group);
238         }
239       } catch (IOException e) {
240         LOG.error("Unable to set the owner/group for file=" + path, e);
241         return false;
242       }
243 
244       return true;
245     }
246 
247     private boolean copyData(final Context context,
248         final Path inputPath, final FSDataInputStream in,
249         final Path outputPath, final FSDataOutputStream out,
250         final long inputFileSize) {
251       final String statusMessage = "copied %s/" + StringUtils.humanReadableInt(inputFileSize) +
252                                    " (%.3f%%)";
253 
254       try {
255         byte[] buffer = new byte[BUFFER_SIZE];
256         long totalBytesWritten = 0;
257         int reportBytes = 0;
258         int bytesRead;
259 
260         while ((bytesRead = in.read(buffer)) > 0) {
261           out.write(buffer, 0, bytesRead);
262           totalBytesWritten += bytesRead;
263           reportBytes += bytesRead;
264 
265           if (reportBytes >= REPORT_SIZE) {
266             context.getCounter(Counter.BYTES_COPIED).increment(reportBytes);
267             context.setStatus(String.format(statusMessage,
268                               StringUtils.humanReadableInt(totalBytesWritten),
269                               totalBytesWritten/(float)inputFileSize) +
270                               " from " + inputPath + " to " + outputPath);
271             reportBytes = 0;
272           }
273         }
274 
275         context.getCounter(Counter.BYTES_COPIED).increment(reportBytes);
276         context.setStatus(String.format(statusMessage,
277                           StringUtils.humanReadableInt(totalBytesWritten),
278                           totalBytesWritten/(float)inputFileSize) +
279                           " from " + inputPath + " to " + outputPath);
280 
281         // Verify that the written size match
282         if (totalBytesWritten != inputFileSize) {
283           LOG.error("number of bytes copied not matching copied=" + totalBytesWritten +
284                     " expected=" + inputFileSize + " for file=" + inputPath);
285           context.getCounter(Counter.COPY_FAILED).increment(1);
286           return false;
287         }
288 
289         return true;
290       } catch (IOException e) {
291         LOG.error("Error copying " + inputPath + " to " + outputPath, e);
292         context.getCounter(Counter.COPY_FAILED).increment(1);
293         return false;
294       }
295     }
296 
297     private FSDataInputStream openSourceFile(final Path path) {
298       try {
299         if (HFileLink.isHFileLink(path) || StoreFileInfo.isReference(path)) {
300           return new HFileLink(inputRoot, inputArchive, path).open(inputFs);
301         } else if (isHLogLinkPath(path)) {
302           String serverName = path.getParent().getName();
303           String logName = path.getName();
304           return new HLogLink(inputRoot, serverName, logName).open(inputFs);
305         }
306         return inputFs.open(path);
307       } catch (IOException e) {
308         LOG.error("Unable to open source file=" + path, e);
309         return null;
310       }
311     }
312 
313     private FileStatus getFileStatus(final FileSystem fs, final Path path) {
314       try {
315         if (HFileLink.isHFileLink(path) || StoreFileInfo.isReference(path)) {
316           HFileLink link = new HFileLink(inputRoot, inputArchive, path);
317           return link.getFileStatus(fs);
318         } else if (isHLogLinkPath(path)) {
319           String serverName = path.getParent().getName();
320           String logName = path.getName();
321           return new HLogLink(inputRoot, serverName, logName).getFileStatus(fs);
322         }
323         return fs.getFileStatus(path);
324       } catch (IOException e) {
325         LOG.warn("Unable to get the status for file=" + path);
326         return null;
327       }
328     }
329 
330     private FileChecksum getFileChecksum(final FileSystem fs, final Path path) {
331       try {
332         return fs.getFileChecksum(path);
333       } catch (IOException e) {
334         LOG.warn("Unable to get checksum for file=" + path, e);
335         return null;
336       }
337     }
338 
339     /**
340      * Check if the two files are equal by looking at the file length,
341      * and at the checksum (if user has specified the verifyChecksum flag).
342      */
343     private boolean sameFile(final FileStatus inputStat, final FileStatus outputStat) {
344       // Not matching length
345       if (inputStat.getLen() != outputStat.getLen()) return false;
346 
347       // Mark files as equals, since user asked for no checksum verification
348       if (!verifyChecksum) return true;
349 
350       // If checksums are not available, files are not the same.
351       FileChecksum inChecksum = getFileChecksum(inputFs, inputStat.getPath());
352       if (inChecksum == null) return false;
353 
354       FileChecksum outChecksum = getFileChecksum(outputFs, outputStat.getPath());
355       if (outChecksum == null) return false;
356 
357       return inChecksum.equals(outChecksum);
358     }
359 
360     /**
361      * HLog files are encoded as serverName/logName
362      * and since all the other files should be in /hbase/table/..path..
363      * we can rely on the depth, for now.
364      */
365     private static boolean isHLogLinkPath(final Path path) {
366       return path.depth() == 2;
367     }
368   }
369 
370   /**
371    * Extract the list of files (HFiles/HLogs) to copy using Map-Reduce.
372    * @return list of files referenced by the snapshot (pair of path and size)
373    */
374   private List<Pair<Path, Long>> getSnapshotFiles(final FileSystem fs, final Path snapshotDir)
375       throws IOException {
376     SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
377 
378     final List<Pair<Path, Long>> files = new ArrayList<Pair<Path, Long>>();
379     final TableName table =
380         TableName.valueOf(snapshotDesc.getTable());
381     final Configuration conf = getConf();
382 
383     // Get snapshot files
384     SnapshotReferenceUtil.visitReferencedFiles(fs, snapshotDir,
385       new SnapshotReferenceUtil.FileVisitor() {
386         public void storeFile (final String region, final String family, final String hfile)
387             throws IOException {
388           Path path = new Path(family, HFileLink.createHFileLinkName(table, region, hfile));
389           long size = new HFileLink(conf, path).getFileStatus(fs).getLen();
390           files.add(new Pair<Path, Long>(path, size));
391         }
392 
393         public void recoveredEdits (final String region, final String logfile)
394             throws IOException {
395           // copied with the snapshot referenecs
396         }
397 
398         public void logFile (final String server, final String logfile)
399             throws IOException {
400           long size = new HLogLink(conf, server, logfile).getFileStatus(fs).getLen();
401           files.add(new Pair<Path, Long>(new Path(server, logfile), size));
402         }
403     });
404 
405     return files;
406   }
407 
408   /**
409    * Given a list of file paths and sizes, create around ngroups in as balanced a way as possible.
410    * The groups created will have similar amounts of bytes.
411    * <p>
412    * The algorithm used is pretty straightforward; the file list is sorted by size,
413    * and then each group fetch the bigger file available, iterating through groups
414    * alternating the direction.
415    */
416   static List<List<Path>> getBalancedSplits(final List<Pair<Path, Long>> files, int ngroups) {
417     // Sort files by size, from small to big
418     Collections.sort(files, new Comparator<Pair<Path, Long>>() {
419       public int compare(Pair<Path, Long> a, Pair<Path, Long> b) {
420         long r = a.getSecond() - b.getSecond();
421         return (r < 0) ? -1 : ((r > 0) ? 1 : 0);
422       }
423     });
424 
425     // create balanced groups
426     List<List<Path>> fileGroups = new LinkedList<List<Path>>();
427     long[] sizeGroups = new long[ngroups];
428     int hi = files.size() - 1;
429     int lo = 0;
430 
431     List<Path> group;
432     int dir = 1;
433     int g = 0;
434 
435     while (hi >= lo) {
436       if (g == fileGroups.size()) {
437         group = new LinkedList<Path>();
438         fileGroups.add(group);
439       } else {
440         group = fileGroups.get(g);
441       }
442 
443       Pair<Path, Long> fileInfo = files.get(hi--);
444 
445       // add the hi one
446       sizeGroups[g] += fileInfo.getSecond();
447       group.add(fileInfo.getFirst());
448 
449       // change direction when at the end or the beginning
450       g += dir;
451       if (g == ngroups) {
452         dir = -1;
453         g = ngroups - 1;
454       } else if (g < 0) {
455         dir = 1;
456         g = 0;
457       }
458     }
459 
460     if (LOG.isDebugEnabled()) {
461       for (int i = 0; i < sizeGroups.length; ++i) {
462         LOG.debug("export split=" + i + " size=" + StringUtils.humanReadableInt(sizeGroups[i]));
463       }
464     }
465 
466     return fileGroups;
467   }
468 
469   private static Path getInputFolderPath(Configuration conf)
470       throws IOException, InterruptedException {
471     Path stagingDir = JobUtil.getStagingDir(conf);
472     return new Path(stagingDir, INPUT_FOLDER_PREFIX +
473       String.valueOf(EnvironmentEdgeManager.currentTimeMillis()));
474   }
475 
476   /**
477    * Create the input files, with the path to copy, for the MR job.
478    * Each input files contains n files, and each input file has a similar amount data to copy.
479    * The number of input files created are based on the number of mappers provided as argument
480    * and the number of the files to copy.
481    */
482   private static Path[] createInputFiles(final Configuration conf,
483       final List<Pair<Path, Long>> snapshotFiles, int mappers)
484       throws IOException, InterruptedException {
485     Path inputFolderPath = getInputFolderPath(conf);
486     FileSystem fs = inputFolderPath.getFileSystem(conf);
487     LOG.debug("Input folder location: " + inputFolderPath);
488 
489     List<List<Path>> splits = getBalancedSplits(snapshotFiles, mappers);
490     Path[] inputFiles = new Path[splits.size()];
491 
492     Text key = new Text();
493     for (int i = 0; i < inputFiles.length; i++) {
494       List<Path> files = splits.get(i);
495       inputFiles[i] = new Path(inputFolderPath, String.format("export-%d.seq", i));
496       SequenceFile.Writer writer = SequenceFile.createWriter(fs, conf, inputFiles[i],
497         Text.class, NullWritable.class);
498       LOG.debug("Input split: " + i);
499       try {
500         for (Path file: files) {
501           LOG.debug(file.toString());
502           key.set(file.toString());
503           writer.append(key, NullWritable.get());
504         }
505       } finally {
506         writer.close();
507       }
508     }
509 
510     return inputFiles;
511   }
512 
513   /**
514    * Run Map-Reduce Job to perform the files copy.
515    */
516   private boolean runCopyJob(final Path inputRoot, final Path outputRoot,
517       final List<Pair<Path, Long>> snapshotFiles, final boolean verifyChecksum,
518       final String filesUser, final String filesGroup, final int filesMode,
519       final int mappers) throws IOException, InterruptedException, ClassNotFoundException {
520     Configuration conf = getConf();
521     if (filesGroup != null) conf.set(CONF_FILES_GROUP, filesGroup);
522     if (filesUser != null) conf.set(CONF_FILES_USER, filesUser);
523     conf.setInt(CONF_FILES_MODE, filesMode);
524     conf.setBoolean(CONF_CHECKSUM_VERIFY, verifyChecksum);
525     conf.set(CONF_OUTPUT_ROOT, outputRoot.toString());
526     conf.set(CONF_INPUT_ROOT, inputRoot.toString());
527     conf.setInt("mapreduce.job.maps", mappers);
528 
529     Job job = new Job(conf);
530     job.setJobName("ExportSnapshot");
531     job.setJarByClass(ExportSnapshot.class);
532     job.setMapperClass(ExportMapper.class);
533     job.setInputFormatClass(SequenceFileInputFormat.class);
534     job.setOutputFormatClass(NullOutputFormat.class);
535     job.setMapSpeculativeExecution(false);
536     job.setNumReduceTasks(0);
537     for (Path path: createInputFiles(conf, snapshotFiles, mappers)) {
538       LOG.debug("Add Input Path=" + path);
539       SequenceFileInputFormat.addInputPath(job, path);
540     }
541 
542     return job.waitForCompletion(true);
543   }
544 
545   /**
546    * Execute the export snapshot by copying the snapshot metadata, hfiles and hlogs.
547    * @return 0 on success, and != 0 upon failure.
548    */
549   @Override
550   public int run(String[] args) throws Exception {
551     boolean verifyChecksum = true;
552     String snapshotName = null;
553     String filesGroup = null;
554     String filesUser = null;
555     Path outputRoot = null;
556     int filesMode = 0;
557     int mappers = getConf().getInt("mapreduce.job.maps", 1);
558 
559     // Process command line args
560     for (int i = 0; i < args.length; i++) {
561       String cmd = args[i];
562       try {
563         if (cmd.equals("-snapshot")) {
564           snapshotName = args[++i];
565         } else if (cmd.equals("-copy-to")) {
566           outputRoot = new Path(args[++i]);
567         } else if (cmd.equals("-no-checksum-verify")) {
568           verifyChecksum = false;
569         } else if (cmd.equals("-mappers")) {
570           mappers = Integer.parseInt(args[++i]);
571         } else if (cmd.equals("-chuser")) {
572           filesUser = args[++i];
573         } else if (cmd.equals("-chgroup")) {
574           filesGroup = args[++i];
575         } else if (cmd.equals("-chmod")) {
576           filesMode = Integer.parseInt(args[++i], 8);
577         } else if (cmd.equals("-h") || cmd.equals("--help")) {
578           printUsageAndExit();
579         } else {
580           System.err.println("UNEXPECTED: " + cmd);
581           printUsageAndExit();
582         }
583       } catch (Exception e) {
584         printUsageAndExit();
585       }
586     }
587 
588     // Check user options
589     if (snapshotName == null) {
590       System.err.println("Snapshot name not provided.");
591       printUsageAndExit();
592     }
593 
594     if (outputRoot == null) {
595       System.err.println("Destination file-system not provided.");
596       printUsageAndExit();
597     }
598 
599     Configuration conf = getConf();
600     Path inputRoot = FSUtils.getRootDir(conf);
601     FileSystem inputFs = FileSystem.get(conf);
602     FileSystem outputFs = FileSystem.get(outputRoot.toUri(), conf);
603 
604     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, inputRoot);
605     Path snapshotTmpDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshotName, outputRoot);
606     Path outputSnapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, outputRoot);
607 
608     // Check if the snapshot already exists
609     if (outputFs.exists(outputSnapshotDir)) {
610       System.err.println("The snapshot '" + snapshotName +
611         "' already exists in the destination: " + outputSnapshotDir);
612       return 1;
613     }
614 
615     // Check if the snapshot already in-progress
616     if (outputFs.exists(snapshotTmpDir)) {
617       System.err.println("A snapshot with the same name '" + snapshotName + "' may be in-progress");
618       System.err.println("Please check " + snapshotTmpDir + ". If the snapshot has completed, ");
619       System.err.println("consider removing " + snapshotTmpDir + " before retrying export"); 
620       return 1;
621     }
622 
623     // Step 0 - Extract snapshot files to copy
624     final List<Pair<Path, Long>> files = getSnapshotFiles(inputFs, snapshotDir);
625 
626     // Step 1 - Copy fs1:/.snapshot/<snapshot> to  fs2:/.snapshot/.tmp/<snapshot>
627     // The snapshot references must be copied before the hfiles otherwise the cleaner
628     // will remove them because they are unreferenced.
629     try {
630       FileUtil.copy(inputFs, snapshotDir, outputFs, snapshotTmpDir, false, false, conf);
631     } catch (IOException e) {
632       System.err.println("Failed to copy the snapshot directory: from=" + snapshotDir +
633         " to=" + snapshotTmpDir);
634       e.printStackTrace(System.err);
635       return 1;
636     }
637 
638     // Step 2 - Start MR Job to copy files
639     // The snapshot references must be copied before the files otherwise the files gets removed
640     // by the HFileArchiver, since they have no references.
641     try {
642       if (files.size() == 0) {
643         LOG.warn("There are 0 store file to be copied. There may be no data in the table.");
644       } else {
645         if (!runCopyJob(inputRoot, outputRoot, files, verifyChecksum,
646             filesUser, filesGroup, filesMode, mappers)) {
647           throw new ExportSnapshotException("Snapshot export failed!");
648         }
649       }
650 
651       // Step 3 - Rename fs2:/.snapshot/.tmp/<snapshot> fs2:/.snapshot/<snapshot>
652       if (!outputFs.rename(snapshotTmpDir, outputSnapshotDir)) {
653         System.err.println("Snapshot export failed!");
654         System.err.println("Unable to rename snapshot directory from=" +
655                            snapshotTmpDir + " to=" + outputSnapshotDir);
656         return 1;
657       }
658 
659       return 0;
660     } catch (Exception e) {
661       System.err.println("Snapshot export failed!");
662       e.printStackTrace(System.err);
663       outputFs.delete(outputSnapshotDir, true);
664       return 1;
665     }
666   }
667 
668   // ExportSnapshot
669   private void printUsageAndExit() {
670     System.err.printf("Usage: bin/hbase %s [options]%n", getClass().getName());
671     System.err.println(" where [options] are:");
672     System.err.println("  -h|-help                Show this help and exit.");
673     System.err.println("  -snapshot NAME          Snapshot to restore.");
674     System.err.println("  -copy-to NAME           Remote destination hdfs://");
675     System.err.println("  -no-checksum-verify     Do not verify checksum.");
676     System.err.println("  -chuser USERNAME        Change the owner of the files to the specified one.");
677     System.err.println("  -chgroup GROUP          Change the group of the files to the specified one.");
678     System.err.println("  -chmod MODE             Change the permission of the files to the specified one.");
679     System.err.println("  -mappers                Number of mappers to use during the copy (mapreduce.job.maps).");
680     System.err.println();
681     System.err.println("Examples:");
682     System.err.println("  hbase " + getClass() + " \\");
683     System.err.println("    -snapshot MySnapshot -copy-to hdfs:///srv2:8082/hbase \\");
684     System.err.println("    -chuser MyUser -chgroup MyGroup -chmod 700 -mappers 16");
685     System.exit(1);
686   }
687 
688   /**
689    * The guts of the {@link #main} method.
690    * Call this method to avoid the {@link #main(String[])} System.exit.
691    * @param args
692    * @return errCode
693    * @throws Exception
694    */
695   static int innerMain(final Configuration conf, final String [] args) throws Exception {
696     return ToolRunner.run(conf, new ExportSnapshot(), args);
697   }
698 
699   public static void main(String[] args) throws Exception {
700      System.exit(innerMain(HBaseConfiguration.create(), args));
701   }
702 }