1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.snapshot;
20
21 import java.io.FileNotFoundException;
22 import java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.Collections;
25 import java.util.Comparator;
26 import java.util.LinkedList;
27 import java.util.List;
28 import java.util.Random;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.classification.InterfaceAudience;
33 import org.apache.hadoop.classification.InterfaceStability;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.conf.Configured;
36 import org.apache.hadoop.fs.FSDataInputStream;
37 import org.apache.hadoop.fs.FSDataOutputStream;
38 import org.apache.hadoop.fs.FileChecksum;
39 import org.apache.hadoop.fs.FileStatus;
40 import org.apache.hadoop.fs.FileSystem;
41 import org.apache.hadoop.fs.FileUtil;
42 import org.apache.hadoop.fs.Path;
43 import org.apache.hadoop.fs.permission.FsPermission;
44 import org.apache.hadoop.hbase.HBaseConfiguration;
45 import org.apache.hadoop.hbase.HConstants;
46 import org.apache.hadoop.hbase.io.HFileLink;
47 import org.apache.hadoop.hbase.io.HLogLink;
48 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
49 import org.apache.hadoop.hbase.regionserver.StoreFile;
50 import org.apache.hadoop.hbase.snapshot.ExportSnapshotException;
51 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
52 import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
53 import org.apache.hadoop.hbase.util.Bytes;
54 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
55 import org.apache.hadoop.hbase.util.FSUtils;
56 import org.apache.hadoop.hbase.util.Pair;
57 import org.apache.hadoop.io.NullWritable;
58 import org.apache.hadoop.io.SequenceFile;
59 import org.apache.hadoop.io.Text;
60 import org.apache.hadoop.mapreduce.Job;
61 import org.apache.hadoop.mapreduce.Mapper;
62 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
63 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
64 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
65 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
66 import org.apache.hadoop.util.StringUtils;
67 import org.apache.hadoop.util.Tool;
68 import org.apache.hadoop.util.ToolRunner;
69
70
71
72
73
74
75
76
77 @InterfaceAudience.Public
78 @InterfaceStability.Evolving
79 public final class ExportSnapshot extends Configured implements Tool {
80 private static final Log LOG = LogFactory.getLog(ExportSnapshot.class);
81
82 private static final String CONF_FILES_USER = "snapshot.export.files.attributes.user";
83 private static final String CONF_FILES_GROUP = "snapshot.export.files.attributes.group";
84 private static final String CONF_FILES_MODE = "snapshot.export.files.attributes.mode";
85 private static final String CONF_CHECKSUM_VERIFY = "snapshot.export.checksum.verify";
86 private static final String CONF_OUTPUT_ROOT = "snapshot.export.output.root";
87 private static final String CONF_INPUT_ROOT = "snapshot.export.input.root";
88 private static final String CONF_STAGING_ROOT = "snapshot.export.staging.root";
89 private static final String CONF_BUFFER_SIZE = "snapshot.export.buffer.size";
90 private static final String CONF_MAP_GROUP = "snapshot.export.default.map.group";
91
92 static final String CONF_TEST_FAILURE = "test.snapshot.export.failure";
93 static final String CONF_TEST_RETRY = "test.snapshot.export.failure.retry";
94
95 private static final String INPUT_FOLDER_PREFIX = "export-files.";
96
97
98 public enum Counter { MISSING_FILES, COPY_FAILED, BYTES_EXPECTED, BYTES_COPIED, FILES_COPIED };
99
100 private static class ExportMapper extends Mapper<Text, NullWritable, NullWritable, NullWritable> {
101 final static int REPORT_SIZE = 1 * 1024 * 1024;
102 final static int BUFFER_SIZE = 64 * 1024;
103
104 private boolean testFailures;
105 private Random random;
106
107 private boolean verifyChecksum;
108 private String filesGroup;
109 private String filesUser;
110 private short filesMode;
111 private int bufferSize;
112
113 private FileSystem outputFs;
114 private Path outputArchive;
115 private Path outputRoot;
116
117 private FileSystem inputFs;
118 private Path inputArchive;
119 private Path inputRoot;
120
121 @Override
122 public void setup(Context context) throws IOException {
123 Configuration conf = context.getConfiguration();
124 verifyChecksum = conf.getBoolean(CONF_CHECKSUM_VERIFY, true);
125
126 filesGroup = conf.get(CONF_FILES_GROUP);
127 filesUser = conf.get(CONF_FILES_USER);
128 filesMode = (short)conf.getInt(CONF_FILES_MODE, 0);
129 outputRoot = new Path(conf.get(CONF_OUTPUT_ROOT));
130 inputRoot = new Path(conf.get(CONF_INPUT_ROOT));
131
132 inputArchive = new Path(inputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
133 outputArchive = new Path(outputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
134
135 testFailures = conf.getBoolean(CONF_TEST_FAILURE, false);
136
137 try {
138 inputFs = FileSystem.get(inputRoot.toUri(), conf);
139 } catch (IOException e) {
140 throw new IOException("Could not get the input FileSystem with root=" + inputRoot, e);
141 }
142
143 try {
144 outputFs = FileSystem.get(outputRoot.toUri(), conf);
145 } catch (IOException e) {
146 throw new IOException("Could not get the output FileSystem with root="+ outputRoot, e);
147 }
148
149
150 int defaultBlockSize = Math.max((int) outputFs.getDefaultBlockSize(), BUFFER_SIZE);
151 bufferSize = conf.getInt(CONF_BUFFER_SIZE, defaultBlockSize);
152 LOG.info("Using bufferSize=" + StringUtils.humanReadableInt(bufferSize));
153 }
154
155 @Override
156 public void map(Text key, NullWritable value, Context context)
157 throws InterruptedException, IOException {
158 Path inputPath = new Path(key.toString());
159 Path outputPath = getOutputPath(inputPath);
160
161 LOG.info("copy file input=" + inputPath + " output=" + outputPath);
162 copyFile(context, inputPath, outputPath);
163 }
164
165
166
167
168
169
170 private Path getOutputPath(final Path inputPath) throws IOException {
171 Path path;
172 if (HFileLink.isHFileLink(inputPath) || StoreFile.isReference(inputPath)) {
173 String family = inputPath.getParent().getName();
174 String table = HFileLink.getReferencedTableName(inputPath.getName());
175 String region = HFileLink.getReferencedRegionName(inputPath.getName());
176 String hfile = HFileLink.getReferencedHFileName(inputPath.getName());
177 path = new Path(table, new Path(region, new Path(family, hfile)));
178 } else if (isHLogLinkPath(inputPath)) {
179 String logName = inputPath.getName();
180 path = new Path(new Path(outputRoot, HConstants.HREGION_OLDLOGDIR_NAME), logName);
181 } else {
182 path = inputPath;
183 }
184 return new Path(outputArchive, path);
185 }
186
187
188
189
190 private void injectTestFailure(final Context context, final Path inputPath)
191 throws IOException {
192 if (testFailures) {
193 if (context.getConfiguration().getBoolean(CONF_TEST_RETRY, false)) {
194 if (random == null) {
195 random = new Random();
196 }
197
198
199
200
201 if (random.nextFloat() < 0.03) {
202 throw new IOException("TEST RETRY FAILURE: Unable to copy input=" + inputPath
203 + " time=" + System.currentTimeMillis());
204 }
205 } else {
206 context.getCounter(Counter.COPY_FAILED).increment(1);
207 throw new IOException("TEST FAILURE: Unable to copy input=" + inputPath);
208 }
209 }
210 }
211
212 private void copyFile(final Context context, final Path inputPath, final Path outputPath)
213 throws IOException {
214 injectTestFailure(context, inputPath);
215
216
217 FileStatus inputStat = getSourceFileStatus(context, inputPath);
218
219
220 if (outputFs.exists(outputPath)) {
221 FileStatus outputStat = outputFs.getFileStatus(outputPath);
222 if (outputStat != null && sameFile(inputStat, outputStat)) {
223 LOG.info("Skip copy " + inputPath + " to " + outputPath + ", same file.");
224 return;
225 }
226 }
227
228 FSDataInputStream in = openSourceFile(context, inputPath);
229 try {
230 context.getCounter(Counter.BYTES_EXPECTED).increment(inputStat.getLen());
231
232
233 outputFs.mkdirs(outputPath.getParent());
234 FSDataOutputStream out = outputFs.create(outputPath, true);
235 try {
236 copyData(context, inputPath, in, outputPath, out, inputStat.getLen());
237 } finally {
238 out.close();
239 }
240
241
242 if (!preserveAttributes(outputPath, inputStat)) {
243 LOG.warn("You may have to run manually chown on: " + outputPath);
244 }
245 } finally {
246 in.close();
247 }
248 }
249
250
251
252
253
254
255
256
257
258 private boolean preserveAttributes(final Path path, final FileStatus refStat) {
259 FileStatus stat;
260 try {
261 stat = outputFs.getFileStatus(path);
262 } catch (IOException e) {
263 LOG.warn("Unable to get the status for file=" + path);
264 return false;
265 }
266
267 try {
268 if (filesMode > 0 && stat.getPermission().toShort() != filesMode) {
269 outputFs.setPermission(path, new FsPermission(filesMode));
270 } else if (!stat.getPermission().equals(refStat.getPermission())) {
271 outputFs.setPermission(path, refStat.getPermission());
272 }
273 } catch (IOException e) {
274 LOG.warn("Unable to set the permission for file="+ stat.getPath() +": "+ e.getMessage());
275 return false;
276 }
277
278 String user = stringIsNotEmpty(filesUser) ? filesUser : refStat.getOwner();
279 String group = stringIsNotEmpty(filesGroup) ? filesGroup : refStat.getGroup();
280 if (stringIsNotEmpty(user) || stringIsNotEmpty(group)) {
281 try {
282 if (!(user.equals(stat.getOwner()) && group.equals(stat.getGroup()))) {
283 outputFs.setOwner(path, user, group);
284 }
285 } catch (IOException e) {
286 LOG.warn("Unable to set the owner/group for file="+ stat.getPath() +": "+ e.getMessage());
287 LOG.warn("The user/group may not exist on the destination cluster: user=" +
288 user + " group=" + group);
289 return false;
290 }
291 }
292
293 return true;
294 }
295
296 private boolean stringIsNotEmpty(final String str) {
297 return str != null && str.length() > 0;
298 }
299
300 private void copyData(final Context context,
301 final Path inputPath, final FSDataInputStream in,
302 final Path outputPath, final FSDataOutputStream out,
303 final long inputFileSize)
304 throws IOException {
305 final String statusMessage = "copied %s/" + StringUtils.humanReadableInt(inputFileSize) +
306 " (%.1f%%)";
307
308 try {
309 byte[] buffer = new byte[bufferSize];
310 long totalBytesWritten = 0;
311 int reportBytes = 0;
312 int bytesRead;
313
314 long stime = System.currentTimeMillis();
315 while ((bytesRead = in.read(buffer)) > 0) {
316 out.write(buffer, 0, bytesRead);
317 totalBytesWritten += bytesRead;
318 reportBytes += bytesRead;
319
320 if (reportBytes >= REPORT_SIZE) {
321 context.getCounter(Counter.BYTES_COPIED).increment(reportBytes);
322 context.setStatus(String.format(statusMessage,
323 StringUtils.humanReadableInt(totalBytesWritten),
324 (totalBytesWritten/(float)inputFileSize) * 100.0f) +
325 " from " + inputPath + " to " + outputPath);
326 reportBytes = 0;
327 }
328 }
329 long etime = System.currentTimeMillis();
330
331 context.getCounter(Counter.BYTES_COPIED).increment(reportBytes);
332 context.setStatus(String.format(statusMessage,
333 StringUtils.humanReadableInt(totalBytesWritten),
334 (totalBytesWritten/(float)inputFileSize) * 100.0f) +
335 " from " + inputPath + " to " + outputPath);
336
337
338 if (totalBytesWritten != inputFileSize) {
339 String msg = "number of bytes copied not matching copied=" + totalBytesWritten +
340 " expected=" + inputFileSize + " for file=" + inputPath;
341 throw new IOException(msg);
342 }
343
344 LOG.info("copy completed for input=" + inputPath + " output=" + outputPath);
345 LOG.info("size=" + totalBytesWritten +
346 " (" + StringUtils.humanReadableInt(totalBytesWritten) + ")" +
347 " time=" + StringUtils.formatTimeDiff(etime, stime) +
348 String.format(" %.3fM/sec", (totalBytesWritten / ((etime - stime)/1000.0))/1048576.0));
349 context.getCounter(Counter.FILES_COPIED).increment(1);
350 } catch (IOException e) {
351 LOG.error("Error copying " + inputPath + " to " + outputPath, e);
352 context.getCounter(Counter.COPY_FAILED).increment(1);
353 throw e;
354 }
355 }
356
357
358
359
360
361
362 private FSDataInputStream openSourceFile(Context context, final Path path) throws IOException {
363 try {
364 if (HFileLink.isHFileLink(path) || StoreFile.isReference(path)) {
365 return new HFileLink(inputRoot, inputArchive, path).open(inputFs);
366 } else if (isHLogLinkPath(path)) {
367 String serverName = path.getParent().getName();
368 String logName = path.getName();
369 return new HLogLink(inputRoot, serverName, logName).open(inputFs);
370 }
371 return inputFs.open(path);
372 } catch (IOException e) {
373 context.getCounter(Counter.MISSING_FILES).increment(1);
374 LOG.error("Unable to open source file=" + path, e);
375 throw e;
376 }
377 }
378
379 private FileStatus getSourceFileStatus(Context context, final Path path) throws IOException {
380 try {
381 if (HFileLink.isHFileLink(path) || StoreFile.isReference(path)) {
382 HFileLink link = new HFileLink(inputRoot, inputArchive, path);
383 return link.getFileStatus(inputFs);
384 } else if (isHLogLinkPath(path)) {
385 String serverName = path.getParent().getName();
386 String logName = path.getName();
387 return new HLogLink(inputRoot, serverName, logName).getFileStatus(inputFs);
388 }
389 return inputFs.getFileStatus(path);
390 } catch (FileNotFoundException e) {
391 context.getCounter(Counter.MISSING_FILES).increment(1);
392 LOG.error("Unable to get the status for source file=" + path, e);
393 throw e;
394 } catch (IOException e) {
395 LOG.error("Unable to get the status for source file=" + path, e);
396 throw e;
397 }
398 }
399
400 private FileChecksum getFileChecksum(final FileSystem fs, final Path path) {
401 try {
402 return fs.getFileChecksum(path);
403 } catch (IOException e) {
404 LOG.warn("Unable to get checksum for file=" + path, e);
405 return null;
406 }
407 }
408
409
410
411
412
413 private boolean sameFile(final FileStatus inputStat, final FileStatus outputStat) {
414
415 if (inputStat.getLen() != outputStat.getLen()) return false;
416
417
418 if (!verifyChecksum) return true;
419
420
421 FileChecksum inChecksum = getFileChecksum(inputFs, inputStat.getPath());
422 if (inChecksum == null) return false;
423
424 FileChecksum outChecksum = getFileChecksum(outputFs, outputStat.getPath());
425 if (outChecksum == null) return false;
426
427 return inChecksum.equals(outChecksum);
428 }
429
430
431
432
433
434
435 private static boolean isHLogLinkPath(final Path path) {
436 return path.depth() == 2;
437 }
438 }
439
440
441
442
443
444 private List<Pair<Path, Long>> getSnapshotFiles(final FileSystem fs, final Path snapshotDir)
445 throws IOException {
446 SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
447
448 final List<Pair<Path, Long>> files = new ArrayList<Pair<Path, Long>>();
449 final String table = snapshotDesc.getTable();
450 final Configuration conf = getConf();
451
452
453 SnapshotReferenceUtil.visitReferencedFiles(fs, snapshotDir,
454 new SnapshotReferenceUtil.FileVisitor() {
455 public void storeFile (final String region, final String family, final String hfile)
456 throws IOException {
457 Path path = HFileLink.createPath(table, region, family, hfile);
458 long size = new HFileLink(conf, path).getFileStatus(fs).getLen();
459 files.add(new Pair<Path, Long>(path, size));
460 }
461
462 public void recoveredEdits (final String region, final String logfile)
463 throws IOException {
464
465 }
466
467 public void logFile (final String server, final String logfile)
468 throws IOException {
469 long size = new HLogLink(conf, server, logfile).getFileStatus(fs).getLen();
470 files.add(new Pair<Path, Long>(new Path(server, logfile), size));
471 }
472 });
473
474 return files;
475 }
476
477
478
479
480
481
482
483
484
485 static List<List<Path>> getBalancedSplits(final List<Pair<Path, Long>> files, int ngroups) {
486
487 Collections.sort(files, new Comparator<Pair<Path, Long>>() {
488 public int compare(Pair<Path, Long> a, Pair<Path, Long> b) {
489 long r = a.getSecond() - b.getSecond();
490 return (r < 0) ? -1 : ((r > 0) ? 1 : 0);
491 }
492 });
493
494
495 List<List<Path>> fileGroups = new LinkedList<List<Path>>();
496 long[] sizeGroups = new long[ngroups];
497 int hi = files.size() - 1;
498 int lo = 0;
499
500 List<Path> group;
501 int dir = 1;
502 int g = 0;
503
504 while (hi >= lo) {
505 if (g == fileGroups.size()) {
506 group = new LinkedList<Path>();
507 fileGroups.add(group);
508 } else {
509 group = fileGroups.get(g);
510 }
511
512 Pair<Path, Long> fileInfo = files.get(hi--);
513
514
515 sizeGroups[g] += fileInfo.getSecond();
516 group.add(fileInfo.getFirst());
517
518
519 g += dir;
520 if (g == ngroups) {
521 dir = -1;
522 g = ngroups - 1;
523 } else if (g < 0) {
524 dir = 1;
525 g = 0;
526 }
527 }
528
529 if (LOG.isDebugEnabled()) {
530 for (int i = 0; i < sizeGroups.length; ++i) {
531 LOG.debug("export split=" + i + " size=" + StringUtils.humanReadableInt(sizeGroups[i]));
532 }
533 }
534
535 return fileGroups;
536 }
537
538 private static Path getInputFolderPath(final FileSystem fs, final Configuration conf)
539 throws IOException, InterruptedException {
540 String stagingName = "exportSnapshot-" + EnvironmentEdgeManager.currentTimeMillis();
541 Path stagingDir = new Path(conf.get(CONF_STAGING_ROOT, fs.getWorkingDirectory().toString())
542 , stagingName);
543 fs.mkdirs(stagingDir);
544 return new Path(stagingDir, INPUT_FOLDER_PREFIX +
545 String.valueOf(EnvironmentEdgeManager.currentTimeMillis()));
546 }
547
548
549
550
551
552
553
554 private static Path[] createInputFiles(final Configuration conf,
555 final List<Pair<Path, Long>> snapshotFiles, int mappers)
556 throws IOException, InterruptedException {
557 FileSystem fs = FileSystem.get(conf);
558 Path inputFolderPath = getInputFolderPath(fs, conf);
559 LOG.debug("Input folder location: " + inputFolderPath);
560
561 List<List<Path>> splits = getBalancedSplits(snapshotFiles, mappers);
562 Path[] inputFiles = new Path[splits.size()];
563
564 Text key = new Text();
565 for (int i = 0; i < inputFiles.length; i++) {
566 List<Path> files = splits.get(i);
567 inputFiles[i] = new Path(inputFolderPath, String.format("export-%d.seq", i));
568 SequenceFile.Writer writer = SequenceFile.createWriter(fs, conf, inputFiles[i],
569 Text.class, NullWritable.class);
570 LOG.debug("Input split: " + i);
571 try {
572 for (Path file: files) {
573 LOG.debug(file.toString());
574 key.set(file.toString());
575 writer.append(key, NullWritable.get());
576 }
577 } finally {
578 writer.close();
579 }
580 }
581
582 return inputFiles;
583 }
584
585
586
587
588 private void runCopyJob(final Path inputRoot, final Path outputRoot,
589 final List<Pair<Path, Long>> snapshotFiles, final boolean verifyChecksum,
590 final String filesUser, final String filesGroup, final int filesMode,
591 final int mappers) throws IOException, InterruptedException, ClassNotFoundException {
592 Configuration conf = getConf();
593 if (filesGroup != null) conf.set(CONF_FILES_GROUP, filesGroup);
594 if (filesUser != null) conf.set(CONF_FILES_USER, filesUser);
595 conf.setInt(CONF_FILES_MODE, filesMode);
596 conf.setBoolean(CONF_CHECKSUM_VERIFY, verifyChecksum);
597 conf.set(CONF_OUTPUT_ROOT, outputRoot.toString());
598 conf.set(CONF_INPUT_ROOT, inputRoot.toString());
599 conf.setInt("mapreduce.job.maps", mappers);
600
601
602 conf.setBoolean("mapreduce.map.speculative", false);
603 conf.setBoolean("mapreduce.reduce.speculative", false);
604 conf.setBoolean("mapred.map.tasks.speculative.execution", false);
605 conf.setBoolean("mapred.reduce.tasks.speculative.execution", false);
606
607 Job job = new Job(conf);
608 job.setJobName("ExportSnapshot");
609 job.setJarByClass(ExportSnapshot.class);
610 job.setMapperClass(ExportMapper.class);
611 job.setInputFormatClass(SequenceFileInputFormat.class);
612 job.setOutputFormatClass(NullOutputFormat.class);
613 job.setNumReduceTasks(0);
614 for (Path path: createInputFiles(conf, snapshotFiles, mappers)) {
615 LOG.debug("Add Input Path=" + path);
616 SequenceFileInputFormat.addInputPath(job, path);
617 }
618
619
620 if (!job.waitForCompletion(true)) {
621
622
623 throw new ExportSnapshotException("Copy Files Map-Reduce Job failed");
624 }
625 }
626
627
628
629
630
631 @Override
632 public int run(String[] args) throws IOException {
633 boolean verifyChecksum = true;
634 String snapshotName = null;
635 boolean overwrite = false;
636 String filesGroup = null;
637 String filesUser = null;
638 Path outputRoot = null;
639 int filesMode = 0;
640 int mappers = 0;
641
642
643 for (int i = 0; i < args.length; i++) {
644 String cmd = args[i];
645 try {
646 if (cmd.equals("-snapshot")) {
647 snapshotName = args[++i];
648 } else if (cmd.equals("-copy-to")) {
649 outputRoot = new Path(args[++i]);
650 } else if (cmd.equals("-no-checksum-verify")) {
651 verifyChecksum = false;
652 } else if (cmd.equals("-mappers")) {
653 mappers = Integer.parseInt(args[++i]);
654 } else if (cmd.equals("-chuser")) {
655 filesUser = args[++i];
656 } else if (cmd.equals("-chgroup")) {
657 filesGroup = args[++i];
658 } else if (cmd.equals("-chmod")) {
659 filesMode = Integer.parseInt(args[++i], 8);
660 } else if (cmd.equals("-overwrite")) {
661 overwrite = true;
662 } else if (cmd.equals("-h") || cmd.equals("--help")) {
663 printUsageAndExit();
664 } else {
665 System.err.println("UNEXPECTED: " + cmd);
666 printUsageAndExit();
667 }
668 } catch (Exception e) {
669 printUsageAndExit();
670 }
671 }
672
673
674 if (snapshotName == null) {
675 System.err.println("Snapshot name not provided.");
676 printUsageAndExit();
677 }
678
679 if (outputRoot == null) {
680 System.err.println("Destination file-system not provided.");
681 printUsageAndExit();
682 }
683
684 Configuration conf = getConf();
685 Path inputRoot = FSUtils.getRootDir(conf);
686 FileSystem inputFs = FileSystem.get(conf);
687 FileSystem outputFs = FileSystem.get(outputRoot.toUri(), conf);
688
689 Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, inputRoot);
690 Path snapshotTmpDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshotName, outputRoot);
691 Path outputSnapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, outputRoot);
692
693
694 if (outputFs.exists(outputSnapshotDir)) {
695 if (overwrite) {
696 if (!outputFs.delete(outputSnapshotDir, true)) {
697 System.err.println("Unable to remove existing snapshot directory: " + outputSnapshotDir);
698 return 1;
699 }
700 } else {
701 System.err.println("The snapshot '" + snapshotName +
702 "' already exists in the destination: " + outputSnapshotDir);
703 return 1;
704 }
705 }
706
707
708 if (outputFs.exists(snapshotTmpDir)) {
709 if (overwrite) {
710 if (!outputFs.delete(snapshotTmpDir, true)) {
711 System.err.println("Unable to remove existing snapshot tmp directory: " + snapshotTmpDir);
712 return 1;
713 }
714 } else {
715 System.err.println("A snapshot with the same name '" + snapshotName + "' may be in-progress");
716 System.err.println("Please check " + snapshotTmpDir + ". If the snapshot has completed, ");
717 System.err.println("consider removing " + snapshotTmpDir + " before retrying export");
718 return 1;
719 }
720 }
721
722
723 final List<Pair<Path, Long>> files = getSnapshotFiles(inputFs, snapshotDir);
724 if (mappers == 0 && files.size() > 0) {
725 mappers = 1 + (files.size() / conf.getInt(CONF_MAP_GROUP, 10));
726 mappers = Math.min(mappers, files.size());
727 }
728
729
730
731
732 try {
733 FileUtil.copy(inputFs, snapshotDir, outputFs, snapshotTmpDir, false, false, conf);
734 } catch (IOException e) {
735 throw new ExportSnapshotException("Failed to copy the snapshot directory: from=" +
736 snapshotDir + " to=" + snapshotTmpDir);
737 }
738
739
740
741
742 try {
743 if (files.size() == 0) {
744 LOG.warn("There are 0 store file to be copied. There may be no data in the table.");
745 } else {
746 runCopyJob(inputRoot, outputRoot, files, verifyChecksum,
747 filesUser, filesGroup, filesMode, mappers);
748 }
749
750
751 if (!outputFs.rename(snapshotTmpDir, outputSnapshotDir)) {
752 throw new ExportSnapshotException("Unable to rename snapshot directory from=" +
753 snapshotTmpDir + " to=" + outputSnapshotDir);
754 }
755 return 0;
756 } catch (Exception e) {
757 LOG.error("Snapshot export failed", e);
758 outputFs.delete(snapshotTmpDir, true);
759 outputFs.delete(outputSnapshotDir, true);
760 return 1;
761 }
762 }
763
764
765 private void printUsageAndExit() {
766 System.err.printf("Usage: bin/hbase %s [options]%n", getClass().getName());
767 System.err.println(" where [options] are:");
768 System.err.println(" -h|-help Show this help and exit.");
769 System.err.println(" -snapshot NAME Snapshot to restore.");
770 System.err.println(" -copy-to NAME Remote destination hdfs://");
771 System.err.println(" -no-checksum-verify Do not verify checksum.");
772 System.err.println(" -overwrite Rewrite the snapshot manifest if already exists");
773 System.err.println(" -chuser USERNAME Change the owner of the files to the specified one.");
774 System.err.println(" -chgroup GROUP Change the group of the files to the specified one.");
775 System.err.println(" -chmod MODE Change the permission of the files to the specified one.");
776 System.err.println(" -mappers Number of mappers to use during the copy (mapreduce.job.maps).");
777 System.err.println();
778 System.err.println("Examples:");
779 System.err.println(" hbase " + getClass().getName() + " \\");
780 System.err.println(" -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase \\");
781 System.err.println(" -chuser MyUser -chgroup MyGroup -chmod 700 -mappers 16");
782 System.exit(1);
783 }
784
785
786
787
788
789
790
791
792 static int innerMain(final Configuration conf, final String [] args) throws Exception {
793 return ToolRunner.run(conf, new ExportSnapshot(), args);
794 }
795
796 public static void main(String[] args) throws Exception {
797 System.exit(innerMain(HBaseConfiguration.create(), args));
798 }
799 }