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.TableName;
45 import org.apache.hadoop.hbase.HBaseConfiguration;
46 import org.apache.hadoop.hbase.HConstants;
47 import org.apache.hadoop.hbase.io.HFileLink;
48 import org.apache.hadoop.hbase.io.HLogLink;
49 import org.apache.hadoop.hbase.mapreduce.JobUtil;
50 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
51 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
52 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
53 import org.apache.hadoop.hbase.security.UserProvider;
54 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
55 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
56 import org.apache.hadoop.hbase.util.FSUtils;
57 import org.apache.hadoop.hbase.util.Pair;
58 import org.apache.hadoop.io.NullWritable;
59 import org.apache.hadoop.io.SequenceFile;
60 import org.apache.hadoop.io.Text;
61 import org.apache.hadoop.mapreduce.Job;
62 import org.apache.hadoop.mapreduce.Mapper;
63 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
64 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
65 import org.apache.hadoop.util.StringUtils;
66 import org.apache.hadoop.util.Tool;
67 import org.apache.hadoop.util.ToolRunner;
68
69
70
71
72
73
74
75
76 @InterfaceAudience.Public
77 @InterfaceStability.Evolving
78 public final class ExportSnapshot extends Configured implements Tool {
79 private static final Log LOG = LogFactory.getLog(ExportSnapshot.class);
80
81 private static final String CONF_FILES_USER = "snapshot.export.files.attributes.user";
82 private static final String CONF_FILES_GROUP = "snapshot.export.files.attributes.group";
83 private static final String CONF_FILES_MODE = "snapshot.export.files.attributes.mode";
84 private static final String CONF_CHECKSUM_VERIFY = "snapshot.export.checksum.verify";
85 private static final String CONF_OUTPUT_ROOT = "snapshot.export.output.root";
86 private static final String CONF_INPUT_ROOT = "snapshot.export.input.root";
87 private static final String CONF_BUFFER_SIZE = "snapshot.export.buffer.size";
88 private static final String CONF_MAP_GROUP = "snapshot.export.default.map.group";
89
90 static final String CONF_TEST_FAILURE = "test.snapshot.export.failure";
91 static final String CONF_TEST_RETRY = "test.snapshot.export.failure.retry";
92
93 private static final String INPUT_FOLDER_PREFIX = "export-files.";
94
95
96 public enum Counter { MISSING_FILES, COPY_FAILED, BYTES_EXPECTED, BYTES_COPIED, FILES_COPIED };
97
98 private static class ExportMapper extends Mapper<Text, NullWritable, NullWritable, NullWritable> {
99 final static int REPORT_SIZE = 1 * 1024 * 1024;
100 final static int BUFFER_SIZE = 64 * 1024;
101
102 private boolean testFailures;
103 private Random random;
104
105 private boolean verifyChecksum;
106 private String filesGroup;
107 private String filesUser;
108 private short filesMode;
109 private int bufferSize;
110
111 private FileSystem outputFs;
112 private Path outputArchive;
113 private Path outputRoot;
114
115 private FileSystem inputFs;
116 private Path inputArchive;
117 private Path inputRoot;
118
119 @Override
120 public void setup(Context context) throws IOException {
121 Configuration conf = context.getConfiguration();
122 verifyChecksum = conf.getBoolean(CONF_CHECKSUM_VERIFY, true);
123
124 filesGroup = conf.get(CONF_FILES_GROUP);
125 filesUser = conf.get(CONF_FILES_USER);
126 filesMode = (short)conf.getInt(CONF_FILES_MODE, 0);
127 outputRoot = new Path(conf.get(CONF_OUTPUT_ROOT));
128 inputRoot = new Path(conf.get(CONF_INPUT_ROOT));
129
130 inputArchive = new Path(inputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
131 outputArchive = new Path(outputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
132
133 testFailures = conf.getBoolean(CONF_TEST_FAILURE, false);
134
135 try {
136 inputFs = FileSystem.get(inputRoot.toUri(), conf);
137 } catch (IOException e) {
138 throw new IOException("Could not get the input FileSystem with root=" + inputRoot, e);
139 }
140
141 try {
142 outputFs = FileSystem.get(outputRoot.toUri(), conf);
143 } catch (IOException e) {
144 throw new IOException("Could not get the output FileSystem with root="+ outputRoot, e);
145 }
146
147
148 int defaultBlockSize = Math.max((int) outputFs.getDefaultBlockSize(), BUFFER_SIZE);
149 bufferSize = conf.getInt(CONF_BUFFER_SIZE, defaultBlockSize);
150 LOG.info("Using bufferSize=" + StringUtils.humanReadableInt(bufferSize));
151 }
152
153 @Override
154 public void map(Text key, NullWritable value, Context context)
155 throws InterruptedException, IOException {
156 Path inputPath = new Path(key.toString());
157 Path outputPath = getOutputPath(inputPath);
158
159 LOG.info("copy file input=" + inputPath + " output=" + outputPath);
160 copyFile(context, inputPath, outputPath);
161 }
162
163
164
165
166
167
168 private Path getOutputPath(final Path inputPath) throws IOException {
169 Path path;
170 if (HFileLink.isHFileLink(inputPath) || StoreFileInfo.isReference(inputPath)) {
171 String family = inputPath.getParent().getName();
172 TableName table =
173 HFileLink.getReferencedTableName(inputPath.getName());
174 String region = HFileLink.getReferencedRegionName(inputPath.getName());
175 String hfile = HFileLink.getReferencedHFileName(inputPath.getName());
176 path = new Path(FSUtils.getTableDir(new Path("./"), table),
177 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) || StoreFileInfo.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) || StoreFileInfo.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 TableName table =
450 TableName.valueOf(snapshotDesc.getTable());
451 final Configuration conf = getConf();
452
453
454 SnapshotReferenceUtil.visitReferencedFiles(fs, snapshotDir,
455 new SnapshotReferenceUtil.FileVisitor() {
456 public void storeFile (final String region, final String family, final String hfile)
457 throws IOException {
458 Path path = HFileLink.createPath(table, region, family, hfile);
459 long size = new HFileLink(conf, path).getFileStatus(fs).getLen();
460 files.add(new Pair<Path, Long>(path, size));
461 }
462
463 public void recoveredEdits (final String region, final String logfile)
464 throws IOException {
465
466 }
467
468 public void logFile (final String server, final String logfile)
469 throws IOException {
470 long size = new HLogLink(conf, server, logfile).getFileStatus(fs).getLen();
471 files.add(new Pair<Path, Long>(new Path(server, logfile), size));
472 }
473 });
474
475 return files;
476 }
477
478
479
480
481
482
483
484
485
486 static List<List<Path>> getBalancedSplits(final List<Pair<Path, Long>> files, int ngroups) {
487
488 Collections.sort(files, new Comparator<Pair<Path, Long>>() {
489 public int compare(Pair<Path, Long> a, Pair<Path, Long> b) {
490 long r = a.getSecond() - b.getSecond();
491 return (r < 0) ? -1 : ((r > 0) ? 1 : 0);
492 }
493 });
494
495
496 List<List<Path>> fileGroups = new LinkedList<List<Path>>();
497 long[] sizeGroups = new long[ngroups];
498 int hi = files.size() - 1;
499 int lo = 0;
500
501 List<Path> group;
502 int dir = 1;
503 int g = 0;
504
505 while (hi >= lo) {
506 if (g == fileGroups.size()) {
507 group = new LinkedList<Path>();
508 fileGroups.add(group);
509 } else {
510 group = fileGroups.get(g);
511 }
512
513 Pair<Path, Long> fileInfo = files.get(hi--);
514
515
516 sizeGroups[g] += fileInfo.getSecond();
517 group.add(fileInfo.getFirst());
518
519
520 g += dir;
521 if (g == ngroups) {
522 dir = -1;
523 g = ngroups - 1;
524 } else if (g < 0) {
525 dir = 1;
526 g = 0;
527 }
528 }
529
530 if (LOG.isDebugEnabled()) {
531 for (int i = 0; i < sizeGroups.length; ++i) {
532 LOG.debug("export split=" + i + " size=" + StringUtils.humanReadableInt(sizeGroups[i]));
533 }
534 }
535
536 return fileGroups;
537 }
538
539 private static Path getInputFolderPath(Configuration conf)
540 throws IOException, InterruptedException {
541 Path stagingDir = JobUtil.getStagingDir(conf);
542 return new Path(stagingDir, INPUT_FOLDER_PREFIX +
543 String.valueOf(EnvironmentEdgeManager.currentTimeMillis()));
544 }
545
546
547
548
549
550
551
552 private static Path[] createInputFiles(final Configuration conf,
553 final List<Pair<Path, Long>> snapshotFiles, int mappers)
554 throws IOException, InterruptedException {
555 Path inputFolderPath = getInputFolderPath(conf);
556 FileSystem fs = inputFolderPath.getFileSystem(conf);
557 LOG.debug("Input folder location: " + inputFolderPath);
558
559 List<List<Path>> splits = getBalancedSplits(snapshotFiles, mappers);
560 Path[] inputFiles = new Path[splits.size()];
561
562 Text key = new Text();
563 for (int i = 0; i < inputFiles.length; i++) {
564 List<Path> files = splits.get(i);
565 inputFiles[i] = new Path(inputFolderPath, String.format("export-%d.seq", i));
566 SequenceFile.Writer writer = SequenceFile.createWriter(fs, conf, inputFiles[i],
567 Text.class, NullWritable.class);
568 LOG.debug("Input split: " + i);
569 try {
570 for (Path file: files) {
571 LOG.debug(file.toString());
572 key.set(file.toString());
573 writer.append(key, NullWritable.get());
574 }
575 } finally {
576 writer.close();
577 }
578 }
579
580 return inputFiles;
581 }
582
583
584
585
586 private void runCopyJob(final FileSystem inputFs, final Path inputRoot,
587 final FileSystem outputFs, final Path outputRoot,
588 final List<Pair<Path, Long>> snapshotFiles, final boolean verifyChecksum,
589 final String filesUser, final String filesGroup, final int filesMode,
590 final int mappers) throws IOException, InterruptedException, ClassNotFoundException {
591 Configuration conf = getConf();
592 if (filesGroup != null) conf.set(CONF_FILES_GROUP, filesGroup);
593 if (filesUser != null) conf.set(CONF_FILES_USER, filesUser);
594 conf.setInt(CONF_FILES_MODE, filesMode);
595 conf.setBoolean(CONF_CHECKSUM_VERIFY, verifyChecksum);
596 conf.set(CONF_OUTPUT_ROOT, outputRoot.toString());
597 conf.set(CONF_INPUT_ROOT, inputRoot.toString());
598 conf.setInt("mapreduce.job.maps", mappers);
599
600 Job job = new Job(conf);
601 job.setJobName("ExportSnapshot");
602 job.setJarByClass(ExportSnapshot.class);
603 TableMapReduceUtil.addDependencyJars(job);
604 job.setMapperClass(ExportMapper.class);
605 job.setInputFormatClass(SequenceFileInputFormat.class);
606 job.setOutputFormatClass(NullOutputFormat.class);
607 job.setMapSpeculativeExecution(false);
608 job.setNumReduceTasks(0);
609 for (Path path: createInputFiles(conf, snapshotFiles, mappers)) {
610 LOG.debug("Add Input Path=" + path);
611 SequenceFileInputFormat.addInputPath(job, path);
612 }
613
614 UserProvider userProvider = UserProvider.instantiate(job.getConfiguration());
615 FsDelegationToken inputFsToken = new FsDelegationToken(userProvider, "irenewer");
616 FsDelegationToken outputFsToken = new FsDelegationToken(userProvider, "orenewer");
617 try {
618
619 inputFsToken.acquireDelegationToken(inputFs);
620 outputFsToken.acquireDelegationToken(outputFs);
621
622
623 if (!job.waitForCompletion(true)) {
624
625
626 throw new ExportSnapshotException("Copy Files Map-Reduce Job failed");
627 }
628 } finally {
629 inputFsToken.releaseDelegationToken();
630 outputFsToken.releaseDelegationToken();
631 }
632 }
633
634
635
636
637
638 @Override
639 public int run(String[] args) throws IOException {
640 boolean verifyChecksum = true;
641 String snapshotName = null;
642 boolean overwrite = false;
643 String filesGroup = null;
644 String filesUser = null;
645 Path outputRoot = null;
646 int filesMode = 0;
647 int mappers = 0;
648
649
650 for (int i = 0; i < args.length; i++) {
651 String cmd = args[i];
652 try {
653 if (cmd.equals("-snapshot")) {
654 snapshotName = args[++i];
655 } else if (cmd.equals("-copy-to")) {
656 outputRoot = new Path(args[++i]);
657 } else if (cmd.equals("-no-checksum-verify")) {
658 verifyChecksum = false;
659 } else if (cmd.equals("-mappers")) {
660 mappers = Integer.parseInt(args[++i]);
661 } else if (cmd.equals("-chuser")) {
662 filesUser = args[++i];
663 } else if (cmd.equals("-chgroup")) {
664 filesGroup = args[++i];
665 } else if (cmd.equals("-chmod")) {
666 filesMode = Integer.parseInt(args[++i], 8);
667 } else if (cmd.equals("-overwrite")) {
668 overwrite = true;
669 } else if (cmd.equals("-h") || cmd.equals("--help")) {
670 printUsageAndExit();
671 } else {
672 System.err.println("UNEXPECTED: " + cmd);
673 printUsageAndExit();
674 }
675 } catch (Exception e) {
676 printUsageAndExit();
677 }
678 }
679
680
681 if (snapshotName == null) {
682 System.err.println("Snapshot name not provided.");
683 printUsageAndExit();
684 }
685
686 if (outputRoot == null) {
687 System.err.println("Destination file-system not provided.");
688 printUsageAndExit();
689 }
690
691 Configuration conf = getConf();
692 Path inputRoot = FSUtils.getRootDir(conf);
693 FileSystem inputFs = FileSystem.get(conf);
694 FileSystem outputFs = FileSystem.get(outputRoot.toUri(), conf);
695
696 Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, inputRoot);
697 Path snapshotTmpDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshotName, outputRoot);
698 Path outputSnapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, outputRoot);
699
700
701 if (outputFs.exists(outputSnapshotDir)) {
702 if (overwrite) {
703 if (!outputFs.delete(outputSnapshotDir, true)) {
704 System.err.println("Unable to remove existing snapshot directory: " + outputSnapshotDir);
705 return 1;
706 }
707 } else {
708 System.err.println("The snapshot '" + snapshotName +
709 "' already exists in the destination: " + outputSnapshotDir);
710 return 1;
711 }
712 }
713
714
715 if (outputFs.exists(snapshotTmpDir)) {
716 if (overwrite) {
717 if (!outputFs.delete(snapshotTmpDir, true)) {
718 System.err.println("Unable to remove existing snapshot tmp directory: " + snapshotTmpDir);
719 return 1;
720 }
721 } else {
722 System.err.println("A snapshot with the same name '"+ snapshotName +"' may be in-progress");
723 System.err.println("Please check " + snapshotTmpDir + ". If the snapshot has completed, ");
724 System.err.println("consider removing " + snapshotTmpDir + " before retrying export");
725 return 1;
726 }
727 }
728
729
730 final List<Pair<Path, Long>> files = getSnapshotFiles(inputFs, snapshotDir);
731 if (mappers == 0 && files.size() > 0) {
732 mappers = 1 + (files.size() / conf.getInt(CONF_MAP_GROUP, 10));
733 mappers = Math.min(mappers, files.size());
734 }
735
736
737
738
739 try {
740 FileUtil.copy(inputFs, snapshotDir, outputFs, snapshotTmpDir, false, false, conf);
741 } catch (IOException e) {
742 throw new ExportSnapshotException("Failed to copy the snapshot directory: from=" +
743 snapshotDir + " to=" + snapshotTmpDir);
744 }
745
746
747
748
749 try {
750 if (files.size() == 0) {
751 LOG.warn("There are 0 store file to be copied. There may be no data in the table.");
752 } else {
753 runCopyJob(inputFs, inputRoot, outputFs, outputRoot, files, verifyChecksum,
754 filesUser, filesGroup, filesMode, mappers);
755 }
756
757
758 if (!outputFs.rename(snapshotTmpDir, outputSnapshotDir)) {
759 throw new ExportSnapshotException("Unable to rename snapshot directory from=" +
760 snapshotTmpDir + " to=" + outputSnapshotDir);
761 }
762 return 0;
763 } catch (Exception e) {
764 LOG.error("Snapshot export failed", e);
765 outputFs.delete(snapshotTmpDir, true);
766 outputFs.delete(outputSnapshotDir, true);
767 return 1;
768 }
769 }
770
771
772 private void printUsageAndExit() {
773 System.err.printf("Usage: bin/hbase %s [options]%n", getClass().getName());
774 System.err.println(" where [options] are:");
775 System.err.println(" -h|-help Show this help and exit.");
776 System.err.println(" -snapshot NAME Snapshot to restore.");
777 System.err.println(" -copy-to NAME Remote destination hdfs://");
778 System.err.println(" -no-checksum-verify Do not verify checksum.");
779 System.err.println(" -overwrite Rewrite the snapshot manifest if already exists");
780 System.err.println(" -chuser USERNAME Change the owner of the files to the specified one.");
781 System.err.println(" -chgroup GROUP Change the group of the files to the specified one.");
782 System.err.println(" -chmod MODE Change the permission of the files to the specified one.");
783 System.err.println(" -mappers Number of mappers to use during the copy (mapreduce.job.maps).");
784 System.err.println();
785 System.err.println("Examples:");
786 System.err.println(" hbase " + getClass().getName() + " \\");
787 System.err.println(" -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase \\");
788 System.err.println(" -chuser MyUser -chgroup MyGroup -chmod 700 -mappers 16");
789 System.exit(1);
790 }
791
792
793
794
795
796
797
798
799 static int innerMain(final Configuration conf, final String [] args) throws Exception {
800 return ToolRunner.run(conf, new ExportSnapshot(), args);
801 }
802
803 public static void main(String[] args) throws Exception {
804 System.exit(innerMain(HBaseConfiguration.create(), args));
805 }
806 }