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