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.BufferedInputStream;
22 import java.io.FileNotFoundException;
23 import java.io.DataInput;
24 import java.io.DataOutput;
25 import java.io.IOException;
26 import java.io.InputStream;
27 import java.util.ArrayList;
28 import java.util.Collections;
29 import java.util.Comparator;
30 import java.util.LinkedList;
31 import java.util.List;
32 import java.util.Random;
33
34 import org.apache.commons.logging.Log;
35 import org.apache.commons.logging.LogFactory;
36 import org.apache.hadoop.hbase.classification.InterfaceAudience;
37 import org.apache.hadoop.hbase.classification.InterfaceStability;
38 import org.apache.hadoop.conf.Configuration;
39 import org.apache.hadoop.conf.Configured;
40 import org.apache.hadoop.fs.FSDataInputStream;
41 import org.apache.hadoop.fs.FSDataOutputStream;
42 import org.apache.hadoop.fs.FileChecksum;
43 import org.apache.hadoop.fs.FileStatus;
44 import org.apache.hadoop.fs.FileSystem;
45 import org.apache.hadoop.fs.FileUtil;
46 import org.apache.hadoop.fs.Path;
47 import org.apache.hadoop.fs.permission.FsPermission;
48 import org.apache.hadoop.hbase.TableName;
49 import org.apache.hadoop.hbase.HBaseConfiguration;
50 import org.apache.hadoop.hbase.HConstants;
51 import org.apache.hadoop.hbase.HRegionInfo;
52 import org.apache.hadoop.hbase.io.FileLink;
53 import org.apache.hadoop.hbase.io.HFileLink;
54 import org.apache.hadoop.hbase.io.WALLink;
55 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
56 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
57 import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
58 import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
59 import org.apache.hadoop.hbase.util.FSUtils;
60 import org.apache.hadoop.hbase.util.Pair;
61 import org.apache.hadoop.io.BytesWritable;
62 import org.apache.hadoop.io.IOUtils;
63 import org.apache.hadoop.io.NullWritable;
64 import org.apache.hadoop.io.Writable;
65 import org.apache.hadoop.mapreduce.Job;
66 import org.apache.hadoop.mapreduce.JobContext;
67 import org.apache.hadoop.mapreduce.Mapper;
68 import org.apache.hadoop.mapreduce.InputFormat;
69 import org.apache.hadoop.mapreduce.InputSplit;
70 import org.apache.hadoop.mapreduce.RecordReader;
71 import org.apache.hadoop.mapreduce.TaskAttemptContext;
72 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
73 import org.apache.hadoop.mapreduce.security.TokenCache;
74 import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream;
75 import org.apache.hadoop.util.StringUtils;
76 import org.apache.hadoop.util.Tool;
77 import org.apache.hadoop.util.ToolRunner;
78
79
80
81
82
83
84
85
86 @InterfaceAudience.Public
87 @InterfaceStability.Evolving
88 public class ExportSnapshot extends Configured implements Tool {
89 private static final Log LOG = LogFactory.getLog(ExportSnapshot.class);
90
91 private static final String MR_NUM_MAPS = "mapreduce.job.maps";
92 private static final String CONF_NUM_SPLITS = "snapshot.export.format.splits";
93 private static final String CONF_SNAPSHOT_NAME = "snapshot.export.format.snapshot.name";
94 private static final String CONF_SNAPSHOT_DIR = "snapshot.export.format.snapshot.dir";
95 private static final String CONF_FILES_USER = "snapshot.export.files.attributes.user";
96 private static final String CONF_FILES_GROUP = "snapshot.export.files.attributes.group";
97 private static final String CONF_FILES_MODE = "snapshot.export.files.attributes.mode";
98 private static final String CONF_CHECKSUM_VERIFY = "snapshot.export.checksum.verify";
99 private static final String CONF_OUTPUT_ROOT = "snapshot.export.output.root";
100 private static final String CONF_INPUT_ROOT = "snapshot.export.input.root";
101 private static final String CONF_BUFFER_SIZE = "snapshot.export.buffer.size";
102 private static final String CONF_MAP_GROUP = "snapshot.export.default.map.group";
103 private static final String CONF_BANDWIDTH_MB = "snapshot.export.map.bandwidth.mb";
104 protected static final String CONF_SKIP_TMP = "snapshot.export.skip.tmp";
105
106 static final String CONF_TEST_FAILURE = "test.snapshot.export.failure";
107 static final String CONF_TEST_RETRY = "test.snapshot.export.failure.retry";
108
109 private static final String INPUT_FOLDER_PREFIX = "export-files.";
110
111
112 public enum Counter {
113 MISSING_FILES, FILES_COPIED, FILES_SKIPPED, COPY_FAILED,
114 BYTES_EXPECTED, BYTES_SKIPPED, BYTES_COPIED
115 }
116
117 private static class ExportMapper extends Mapper<BytesWritable, NullWritable,
118 NullWritable, NullWritable> {
119 final static int REPORT_SIZE = 1 * 1024 * 1024;
120 final static int BUFFER_SIZE = 64 * 1024;
121
122 private boolean testFailures;
123 private Random random;
124
125 private boolean verifyChecksum;
126 private String filesGroup;
127 private String filesUser;
128 private short filesMode;
129 private int bufferSize;
130
131 private FileSystem outputFs;
132 private Path outputArchive;
133 private Path outputRoot;
134
135 private FileSystem inputFs;
136 private Path inputArchive;
137 private Path inputRoot;
138
139 @Override
140 public void setup(Context context) throws IOException {
141 Configuration conf = context.getConfiguration();
142 verifyChecksum = conf.getBoolean(CONF_CHECKSUM_VERIFY, true);
143
144 filesGroup = conf.get(CONF_FILES_GROUP);
145 filesUser = conf.get(CONF_FILES_USER);
146 filesMode = (short)conf.getInt(CONF_FILES_MODE, 0);
147 outputRoot = new Path(conf.get(CONF_OUTPUT_ROOT));
148 inputRoot = new Path(conf.get(CONF_INPUT_ROOT));
149
150 inputArchive = new Path(inputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
151 outputArchive = new Path(outputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
152
153 testFailures = conf.getBoolean(CONF_TEST_FAILURE, false);
154
155 try {
156 conf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true);
157 inputFs = FileSystem.get(inputRoot.toUri(), conf);
158 } catch (IOException e) {
159 throw new IOException("Could not get the input FileSystem with root=" + inputRoot, e);
160 }
161
162 try {
163 conf.setBoolean("fs." + outputRoot.toUri().getScheme() + ".impl.disable.cache", true);
164 outputFs = FileSystem.get(outputRoot.toUri(), conf);
165 } catch (IOException e) {
166 throw new IOException("Could not get the output FileSystem with root="+ outputRoot, e);
167 }
168
169
170 int defaultBlockSize = Math.max((int) outputFs.getDefaultBlockSize(outputRoot), BUFFER_SIZE);
171 bufferSize = conf.getInt(CONF_BUFFER_SIZE, defaultBlockSize);
172 LOG.info("Using bufferSize=" + StringUtils.humanReadableInt(bufferSize));
173
174 for (Counter c : Counter.values()) {
175 context.getCounter(c).increment(0);
176 }
177 }
178
179 @Override
180 protected void cleanup(Context context) {
181 IOUtils.closeStream(inputFs);
182 IOUtils.closeStream(outputFs);
183 }
184
185 @Override
186 public void map(BytesWritable key, NullWritable value, Context context)
187 throws InterruptedException, IOException {
188 SnapshotFileInfo inputInfo = SnapshotFileInfo.parseFrom(key.copyBytes());
189 Path outputPath = getOutputPath(inputInfo);
190
191 copyFile(context, inputInfo, outputPath);
192 }
193
194
195
196
197 private Path getOutputPath(final SnapshotFileInfo inputInfo) throws IOException {
198 Path path = null;
199 switch (inputInfo.getType()) {
200 case HFILE:
201 Path inputPath = new Path(inputInfo.getHfile());
202 String family = inputPath.getParent().getName();
203 TableName table =HFileLink.getReferencedTableName(inputPath.getName());
204 String region = HFileLink.getReferencedRegionName(inputPath.getName());
205 String hfile = HFileLink.getReferencedHFileName(inputPath.getName());
206 path = new Path(FSUtils.getTableDir(new Path("./"), table),
207 new Path(region, new Path(family, hfile)));
208 break;
209 case WAL:
210 Path oldLogsDir = new Path(outputRoot, HConstants.HREGION_OLDLOGDIR_NAME);
211 path = new Path(oldLogsDir, inputInfo.getWalName());
212 break;
213 default:
214 throw new IOException("Invalid File Type: " + inputInfo.getType().toString());
215 }
216 return new Path(outputArchive, path);
217 }
218
219
220
221
222 private void injectTestFailure(final Context context, final SnapshotFileInfo inputInfo)
223 throws IOException {
224 if (testFailures) {
225 if (context.getConfiguration().getBoolean(CONF_TEST_RETRY, false)) {
226 if (random == null) {
227 random = new Random();
228 }
229
230
231
232
233 if (random.nextFloat() < 0.03) {
234 throw new IOException("TEST RETRY FAILURE: Unable to copy input=" + inputInfo
235 + " time=" + System.currentTimeMillis());
236 }
237 } else {
238 context.getCounter(Counter.COPY_FAILED).increment(1);
239 throw new IOException("TEST FAILURE: Unable to copy input=" + inputInfo);
240 }
241 }
242 }
243
244 private void copyFile(final Context context, final SnapshotFileInfo inputInfo,
245 final Path outputPath) throws IOException {
246 injectTestFailure(context, inputInfo);
247
248
249 FileStatus inputStat = getSourceFileStatus(context, inputInfo);
250
251
252 if (outputFs.exists(outputPath)) {
253 FileStatus outputStat = outputFs.getFileStatus(outputPath);
254 if (outputStat != null && sameFile(inputStat, outputStat)) {
255 LOG.info("Skip copy " + inputStat.getPath() + " to " + outputPath + ", same file.");
256 context.getCounter(Counter.FILES_SKIPPED).increment(1);
257 context.getCounter(Counter.BYTES_SKIPPED).increment(inputStat.getLen());
258 return;
259 }
260 }
261
262 InputStream in = openSourceFile(context, inputInfo);
263 int bandwidthMB = context.getConfiguration().getInt(CONF_BANDWIDTH_MB, 100);
264 if (Integer.MAX_VALUE != bandwidthMB) {
265 in = new ThrottledInputStream(new BufferedInputStream(in), bandwidthMB * 1024 * 1024);
266 }
267
268 try {
269 context.getCounter(Counter.BYTES_EXPECTED).increment(inputStat.getLen());
270
271
272 outputFs.mkdirs(outputPath.getParent());
273 FSDataOutputStream out = outputFs.create(outputPath, true);
274 try {
275 copyData(context, inputStat.getPath(), in, outputPath, out, inputStat.getLen());
276 } finally {
277 out.close();
278 }
279
280
281 if (!preserveAttributes(outputPath, inputStat)) {
282 LOG.warn("You may have to run manually chown on: " + outputPath);
283 }
284 } finally {
285 in.close();
286 }
287 }
288
289
290
291
292
293
294
295
296
297 private boolean preserveAttributes(final Path path, final FileStatus refStat) {
298 FileStatus stat;
299 try {
300 stat = outputFs.getFileStatus(path);
301 } catch (IOException e) {
302 LOG.warn("Unable to get the status for file=" + path);
303 return false;
304 }
305
306 try {
307 if (filesMode > 0 && stat.getPermission().toShort() != filesMode) {
308 outputFs.setPermission(path, new FsPermission(filesMode));
309 } else if (refStat != null && !stat.getPermission().equals(refStat.getPermission())) {
310 outputFs.setPermission(path, refStat.getPermission());
311 }
312 } catch (IOException e) {
313 LOG.warn("Unable to set the permission for file="+ stat.getPath() +": "+ e.getMessage());
314 return false;
315 }
316
317 boolean hasRefStat = (refStat != null);
318 String user = stringIsNotEmpty(filesUser) || !hasRefStat ? filesUser : refStat.getOwner();
319 String group = stringIsNotEmpty(filesGroup) || !hasRefStat ? filesGroup : refStat.getGroup();
320 if (stringIsNotEmpty(user) || stringIsNotEmpty(group)) {
321 try {
322 if (!(user.equals(stat.getOwner()) && group.equals(stat.getGroup()))) {
323 outputFs.setOwner(path, user, group);
324 }
325 } catch (IOException e) {
326 LOG.warn("Unable to set the owner/group for file="+ stat.getPath() +": "+ e.getMessage());
327 LOG.warn("The user/group may not exist on the destination cluster: user=" +
328 user + " group=" + group);
329 return false;
330 }
331 }
332
333 return true;
334 }
335
336 private boolean stringIsNotEmpty(final String str) {
337 return str != null && str.length() > 0;
338 }
339
340 private void copyData(final Context context,
341 final Path inputPath, final InputStream in,
342 final Path outputPath, final FSDataOutputStream out,
343 final long inputFileSize)
344 throws IOException {
345 final String statusMessage = "copied %s/" + StringUtils.humanReadableInt(inputFileSize) +
346 " (%.1f%%)";
347
348 try {
349 byte[] buffer = new byte[bufferSize];
350 long totalBytesWritten = 0;
351 int reportBytes = 0;
352 int bytesRead;
353
354 long stime = System.currentTimeMillis();
355 while ((bytesRead = in.read(buffer)) > 0) {
356 out.write(buffer, 0, bytesRead);
357 totalBytesWritten += bytesRead;
358 reportBytes += bytesRead;
359
360 if (reportBytes >= REPORT_SIZE) {
361 context.getCounter(Counter.BYTES_COPIED).increment(reportBytes);
362 context.setStatus(String.format(statusMessage,
363 StringUtils.humanReadableInt(totalBytesWritten),
364 (totalBytesWritten/(float)inputFileSize) * 100.0f) +
365 " from " + inputPath + " to " + outputPath);
366 reportBytes = 0;
367 }
368 }
369 long etime = System.currentTimeMillis();
370
371 context.getCounter(Counter.BYTES_COPIED).increment(reportBytes);
372 context.setStatus(String.format(statusMessage,
373 StringUtils.humanReadableInt(totalBytesWritten),
374 (totalBytesWritten/(float)inputFileSize) * 100.0f) +
375 " from " + inputPath + " to " + outputPath);
376
377
378 if (totalBytesWritten != inputFileSize) {
379 String msg = "number of bytes copied not matching copied=" + totalBytesWritten +
380 " expected=" + inputFileSize + " for file=" + inputPath;
381 throw new IOException(msg);
382 }
383
384 LOG.info("copy completed for input=" + inputPath + " output=" + outputPath);
385 LOG.info("size=" + totalBytesWritten +
386 " (" + StringUtils.humanReadableInt(totalBytesWritten) + ")" +
387 " time=" + StringUtils.formatTimeDiff(etime, stime) +
388 String.format(" %.3fM/sec", (totalBytesWritten / ((etime - stime)/1000.0))/1048576.0));
389 context.getCounter(Counter.FILES_COPIED).increment(1);
390 } catch (IOException e) {
391 LOG.error("Error copying " + inputPath + " to " + outputPath, e);
392 context.getCounter(Counter.COPY_FAILED).increment(1);
393 throw e;
394 }
395 }
396
397
398
399
400
401
402 private FSDataInputStream openSourceFile(Context context, final SnapshotFileInfo fileInfo)
403 throws IOException {
404 try {
405 FileLink link = null;
406 switch (fileInfo.getType()) {
407 case HFILE:
408 Path inputPath = new Path(fileInfo.getHfile());
409 link = new HFileLink(inputRoot, inputArchive, inputPath);
410 break;
411 case WAL:
412 String serverName = fileInfo.getWalServer();
413 String logName = fileInfo.getWalName();
414 link = new WALLink(inputRoot, serverName, logName);
415 break;
416 default:
417 throw new IOException("Invalid File Type: " + fileInfo.getType().toString());
418 }
419 return link.open(inputFs);
420 } catch (IOException e) {
421 context.getCounter(Counter.MISSING_FILES).increment(1);
422 LOG.error("Unable to open source file=" + fileInfo.toString(), e);
423 throw e;
424 }
425 }
426
427 private FileStatus getSourceFileStatus(Context context, final SnapshotFileInfo fileInfo)
428 throws IOException {
429 try {
430 FileLink link = null;
431 switch (fileInfo.getType()) {
432 case HFILE:
433 Path inputPath = new Path(fileInfo.getHfile());
434 link = new HFileLink(inputRoot, inputArchive, inputPath);
435 break;
436 case WAL:
437 link = new WALLink(inputRoot, fileInfo.getWalServer(), fileInfo.getWalName());
438 break;
439 default:
440 throw new IOException("Invalid File Type: " + fileInfo.getType().toString());
441 }
442 return link.getFileStatus(inputFs);
443 } catch (FileNotFoundException e) {
444 context.getCounter(Counter.MISSING_FILES).increment(1);
445 LOG.error("Unable to get the status for source file=" + fileInfo.toString(), e);
446 throw e;
447 } catch (IOException e) {
448 LOG.error("Unable to get the status for source file=" + fileInfo.toString(), e);
449 throw e;
450 }
451 }
452
453 private FileChecksum getFileChecksum(final FileSystem fs, final Path path) {
454 try {
455 return fs.getFileChecksum(path);
456 } catch (IOException e) {
457 LOG.warn("Unable to get checksum for file=" + path, e);
458 return null;
459 }
460 }
461
462
463
464
465
466 private boolean sameFile(final FileStatus inputStat, final FileStatus outputStat) {
467
468 if (inputStat.getLen() != outputStat.getLen()) return false;
469
470
471 if (!verifyChecksum) return true;
472
473
474 FileChecksum inChecksum = getFileChecksum(inputFs, inputStat.getPath());
475 if (inChecksum == null) return false;
476
477 FileChecksum outChecksum = getFileChecksum(outputFs, outputStat.getPath());
478 if (outChecksum == null) return false;
479
480 return inChecksum.equals(outChecksum);
481 }
482 }
483
484
485
486
487
488
489
490
491
492 private static List<Pair<SnapshotFileInfo, Long>> getSnapshotFiles(final Configuration conf,
493 final FileSystem fs, final Path snapshotDir) throws IOException {
494 SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
495
496 final List<Pair<SnapshotFileInfo, Long>> files = new ArrayList<Pair<SnapshotFileInfo, Long>>();
497 final TableName table = TableName.valueOf(snapshotDesc.getTable());
498
499
500 LOG.info("Loading Snapshot '" + snapshotDesc.getName() + "' hfile list");
501 SnapshotReferenceUtil.visitReferencedFiles(conf, fs, snapshotDir, snapshotDesc,
502 new SnapshotReferenceUtil.SnapshotVisitor() {
503 @Override
504 public void storeFile(final HRegionInfo regionInfo, final String family,
505 final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
506 if (storeFile.hasReference()) {
507
508 } else {
509 String region = regionInfo.getEncodedName();
510 String hfile = storeFile.getName();
511 Path path = HFileLink.createPath(table, region, family, hfile);
512
513 SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder()
514 .setType(SnapshotFileInfo.Type.HFILE)
515 .setHfile(path.toString())
516 .build();
517
518 long size;
519 if (storeFile.hasFileSize()) {
520 size = storeFile.getFileSize();
521 } else {
522 size = new HFileLink(conf, path).getFileStatus(fs).getLen();
523 }
524 files.add(new Pair<SnapshotFileInfo, Long>(fileInfo, size));
525 }
526 }
527
528 @Override
529 public void logFile (final String server, final String logfile)
530 throws IOException {
531 SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder()
532 .setType(SnapshotFileInfo.Type.WAL)
533 .setWalServer(server)
534 .setWalName(logfile)
535 .build();
536
537 long size = new WALLink(conf, server, logfile).getFileStatus(fs).getLen();
538 files.add(new Pair<SnapshotFileInfo, Long>(fileInfo, size));
539 }
540 });
541
542 return files;
543 }
544
545
546
547
548
549
550
551
552
553 static List<List<Pair<SnapshotFileInfo, Long>>> getBalancedSplits(
554 final List<Pair<SnapshotFileInfo, Long>> files, final int ngroups) {
555
556 Collections.sort(files, new Comparator<Pair<SnapshotFileInfo, Long>>() {
557 public int compare(Pair<SnapshotFileInfo, Long> a, Pair<SnapshotFileInfo, Long> b) {
558 long r = a.getSecond() - b.getSecond();
559 return (r < 0) ? -1 : ((r > 0) ? 1 : 0);
560 }
561 });
562
563
564 List<List<Pair<SnapshotFileInfo, Long>>> fileGroups =
565 new LinkedList<List<Pair<SnapshotFileInfo, Long>>>();
566 long[] sizeGroups = new long[ngroups];
567 int hi = files.size() - 1;
568 int lo = 0;
569
570 List<Pair<SnapshotFileInfo, Long>> group;
571 int dir = 1;
572 int g = 0;
573
574 while (hi >= lo) {
575 if (g == fileGroups.size()) {
576 group = new LinkedList<Pair<SnapshotFileInfo, Long>>();
577 fileGroups.add(group);
578 } else {
579 group = fileGroups.get(g);
580 }
581
582 Pair<SnapshotFileInfo, Long> fileInfo = files.get(hi--);
583
584
585 sizeGroups[g] += fileInfo.getSecond();
586 group.add(fileInfo);
587
588
589 g += dir;
590 if (g == ngroups) {
591 dir = -1;
592 g = ngroups - 1;
593 } else if (g < 0) {
594 dir = 1;
595 g = 0;
596 }
597 }
598
599 if (LOG.isDebugEnabled()) {
600 for (int i = 0; i < sizeGroups.length; ++i) {
601 LOG.debug("export split=" + i + " size=" + StringUtils.humanReadableInt(sizeGroups[i]));
602 }
603 }
604
605 return fileGroups;
606 }
607
608 private static class ExportSnapshotInputFormat extends InputFormat<BytesWritable, NullWritable> {
609 @Override
610 public RecordReader<BytesWritable, NullWritable> createRecordReader(InputSplit split,
611 TaskAttemptContext tac) throws IOException, InterruptedException {
612 return new ExportSnapshotRecordReader(((ExportSnapshotInputSplit)split).getSplitKeys());
613 }
614
615 @Override
616 public List<InputSplit> getSplits(JobContext context) throws IOException, InterruptedException {
617 Configuration conf = context.getConfiguration();
618 String snapshotName = conf.get(CONF_SNAPSHOT_NAME);
619 Path snapshotDir = new Path(conf.get(CONF_SNAPSHOT_DIR));
620 FileSystem fs = FileSystem.get(snapshotDir.toUri(), conf);
621
622 List<Pair<SnapshotFileInfo, Long>> snapshotFiles = getSnapshotFiles(conf, fs, snapshotDir);
623 int mappers = conf.getInt(CONF_NUM_SPLITS, 0);
624 if (mappers == 0 && snapshotFiles.size() > 0) {
625 mappers = 1 + (snapshotFiles.size() / conf.getInt(CONF_MAP_GROUP, 10));
626 mappers = Math.min(mappers, snapshotFiles.size());
627 conf.setInt(CONF_NUM_SPLITS, mappers);
628 conf.setInt(MR_NUM_MAPS, mappers);
629 }
630
631 List<List<Pair<SnapshotFileInfo, Long>>> groups = getBalancedSplits(snapshotFiles, mappers);
632 List<InputSplit> splits = new ArrayList(groups.size());
633 for (List<Pair<SnapshotFileInfo, Long>> files: groups) {
634 splits.add(new ExportSnapshotInputSplit(files));
635 }
636 return splits;
637 }
638
639 private static class ExportSnapshotInputSplit extends InputSplit implements Writable {
640 private List<Pair<BytesWritable, Long>> files;
641 private long length;
642
643 public ExportSnapshotInputSplit() {
644 this.files = null;
645 }
646
647 public ExportSnapshotInputSplit(final List<Pair<SnapshotFileInfo, Long>> snapshotFiles) {
648 this.files = new ArrayList(snapshotFiles.size());
649 for (Pair<SnapshotFileInfo, Long> fileInfo: snapshotFiles) {
650 this.files.add(new Pair<BytesWritable, Long>(
651 new BytesWritable(fileInfo.getFirst().toByteArray()), fileInfo.getSecond()));
652 this.length += fileInfo.getSecond();
653 }
654 }
655
656 private List<Pair<BytesWritable, Long>> getSplitKeys() {
657 return files;
658 }
659
660 @Override
661 public long getLength() throws IOException, InterruptedException {
662 return length;
663 }
664
665 @Override
666 public String[] getLocations() throws IOException, InterruptedException {
667 return new String[] {};
668 }
669
670 @Override
671 public void readFields(DataInput in) throws IOException {
672 int count = in.readInt();
673 files = new ArrayList<Pair<BytesWritable, Long>>(count);
674 length = 0;
675 for (int i = 0; i < count; ++i) {
676 BytesWritable fileInfo = new BytesWritable();
677 fileInfo.readFields(in);
678 long size = in.readLong();
679 files.add(new Pair<BytesWritable, Long>(fileInfo, size));
680 length += size;
681 }
682 }
683
684 @Override
685 public void write(DataOutput out) throws IOException {
686 out.writeInt(files.size());
687 for (final Pair<BytesWritable, Long> fileInfo: files) {
688 fileInfo.getFirst().write(out);
689 out.writeLong(fileInfo.getSecond());
690 }
691 }
692 }
693
694 private static class ExportSnapshotRecordReader
695 extends RecordReader<BytesWritable, NullWritable> {
696 private final List<Pair<BytesWritable, Long>> files;
697 private long totalSize = 0;
698 private long procSize = 0;
699 private int index = -1;
700
701 ExportSnapshotRecordReader(final List<Pair<BytesWritable, Long>> files) {
702 this.files = files;
703 for (Pair<BytesWritable, Long> fileInfo: files) {
704 totalSize += fileInfo.getSecond();
705 }
706 }
707
708 @Override
709 public void close() { }
710
711 @Override
712 public BytesWritable getCurrentKey() { return files.get(index).getFirst(); }
713
714 @Override
715 public NullWritable getCurrentValue() { return NullWritable.get(); }
716
717 @Override
718 public float getProgress() { return (float)procSize / totalSize; }
719
720 @Override
721 public void initialize(InputSplit split, TaskAttemptContext tac) { }
722
723 @Override
724 public boolean nextKeyValue() {
725 if (index >= 0) {
726 procSize += files.get(index).getSecond();
727 }
728 return(++index < files.size());
729 }
730 }
731 }
732
733
734
735
736
737
738
739
740 private void runCopyJob(final Path inputRoot, final Path outputRoot,
741 final String snapshotName, final Path snapshotDir, final boolean verifyChecksum,
742 final String filesUser, final String filesGroup, final int filesMode,
743 final int mappers, final int bandwidthMB)
744 throws IOException, InterruptedException, ClassNotFoundException {
745 Configuration conf = getConf();
746 if (filesGroup != null) conf.set(CONF_FILES_GROUP, filesGroup);
747 if (filesUser != null) conf.set(CONF_FILES_USER, filesUser);
748 if (mappers > 0) {
749 conf.setInt(CONF_NUM_SPLITS, mappers);
750 conf.setInt(MR_NUM_MAPS, mappers);
751 }
752 conf.setInt(CONF_FILES_MODE, filesMode);
753 conf.setBoolean(CONF_CHECKSUM_VERIFY, verifyChecksum);
754 conf.set(CONF_OUTPUT_ROOT, outputRoot.toString());
755 conf.set(CONF_INPUT_ROOT, inputRoot.toString());
756 conf.setInt(CONF_BANDWIDTH_MB, bandwidthMB);
757 conf.set(CONF_SNAPSHOT_NAME, snapshotName);
758 conf.set(CONF_SNAPSHOT_DIR, snapshotDir.toString());
759
760 Job job = new Job(conf);
761 job.setJobName("ExportSnapshot-" + snapshotName);
762 job.setJarByClass(ExportSnapshot.class);
763 TableMapReduceUtil.addDependencyJars(job);
764 job.setMapperClass(ExportMapper.class);
765 job.setInputFormatClass(ExportSnapshotInputFormat.class);
766 job.setOutputFormatClass(NullOutputFormat.class);
767 job.setMapSpeculativeExecution(false);
768 job.setNumReduceTasks(0);
769
770
771 TokenCache.obtainTokensForNamenodes(job.getCredentials(),
772 new Path[] { inputRoot, outputRoot }, conf);
773
774
775 if (!job.waitForCompletion(true)) {
776
777
778 throw new ExportSnapshotException("Copy Files Map-Reduce Job failed");
779 }
780 }
781
782 private void verifySnapshot(final Configuration baseConf,
783 final FileSystem fs, final Path rootDir, final Path snapshotDir) throws IOException {
784
785 Configuration conf = new Configuration(baseConf);
786 FSUtils.setRootDir(conf, rootDir);
787 FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
788 SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
789 SnapshotReferenceUtil.verifySnapshot(conf, fs, snapshotDir, snapshotDesc);
790 }
791
792
793
794
795
796 @Override
797 public int run(String[] args) throws IOException {
798 boolean verifyTarget = true;
799 boolean verifyChecksum = true;
800 String snapshotName = null;
801 String targetName = null;
802 boolean overwrite = false;
803 String filesGroup = null;
804 String filesUser = null;
805 Path outputRoot = null;
806 int bandwidthMB = Integer.MAX_VALUE;
807 int filesMode = 0;
808 int mappers = 0;
809
810 Configuration conf = getConf();
811 Path inputRoot = FSUtils.getRootDir(conf);
812
813
814 for (int i = 0; i < args.length; i++) {
815 String cmd = args[i];
816 if (cmd.equals("-snapshot")) {
817 snapshotName = args[++i];
818 } else if (cmd.equals("-target")) {
819 targetName = args[++i];
820 } else if (cmd.equals("-copy-to")) {
821 outputRoot = new Path(args[++i]);
822 } else if (cmd.equals("-copy-from")) {
823 inputRoot = new Path(args[++i]);
824 FSUtils.setRootDir(conf, inputRoot);
825 } else if (cmd.equals("-no-checksum-verify")) {
826 verifyChecksum = false;
827 } else if (cmd.equals("-no-target-verify")) {
828 verifyTarget = false;
829 } else if (cmd.equals("-mappers")) {
830 mappers = Integer.parseInt(args[++i]);
831 } else if (cmd.equals("-chuser")) {
832 filesUser = args[++i];
833 } else if (cmd.equals("-chgroup")) {
834 filesGroup = args[++i];
835 } else if (cmd.equals("-bandwidth")) {
836 bandwidthMB = Integer.parseInt(args[++i]);
837 } else if (cmd.equals("-chmod")) {
838 filesMode = Integer.parseInt(args[++i], 8);
839 } else if (cmd.equals("-overwrite")) {
840 overwrite = true;
841 } else if (cmd.equals("-h") || cmd.equals("--help")) {
842 printUsageAndExit();
843 } else {
844 System.err.println("UNEXPECTED: " + cmd);
845 printUsageAndExit();
846 }
847 }
848
849
850 if (snapshotName == null) {
851 System.err.println("Snapshot name not provided.");
852 printUsageAndExit();
853 }
854
855 if (outputRoot == null) {
856 System.err.println("Destination file-system not provided.");
857 printUsageAndExit();
858 }
859
860 if (targetName == null) {
861 targetName = snapshotName;
862 }
863
864 conf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true);
865 FileSystem inputFs = FileSystem.get(inputRoot.toUri(), conf);
866 LOG.debug("inputFs=" + inputFs.getUri().toString() + " inputRoot=" + inputRoot);
867 conf.setBoolean("fs." + outputRoot.toUri().getScheme() + ".impl.disable.cache", true);
868 FileSystem outputFs = FileSystem.get(outputRoot.toUri(), conf);
869 LOG.debug("outputFs=" + outputFs.getUri().toString() + " outputRoot=" + outputRoot.toString());
870
871 boolean skipTmp = conf.getBoolean(CONF_SKIP_TMP, false);
872
873 Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, inputRoot);
874 Path snapshotTmpDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(targetName, outputRoot);
875 Path outputSnapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(targetName, outputRoot);
876 Path initialOutputSnapshotDir = skipTmp ? outputSnapshotDir : snapshotTmpDir;
877
878
879 if (outputFs.exists(outputSnapshotDir)) {
880 if (overwrite) {
881 if (!outputFs.delete(outputSnapshotDir, true)) {
882 System.err.println("Unable to remove existing snapshot directory: " + outputSnapshotDir);
883 return 1;
884 }
885 } else {
886 System.err.println("The snapshot '" + targetName +
887 "' already exists in the destination: " + outputSnapshotDir);
888 return 1;
889 }
890 }
891
892 if (!skipTmp) {
893
894 if (outputFs.exists(snapshotTmpDir)) {
895 if (overwrite) {
896 if (!outputFs.delete(snapshotTmpDir, true)) {
897 System.err.println("Unable to remove existing snapshot tmp directory: "+snapshotTmpDir);
898 return 1;
899 }
900 } else {
901 System.err.println("A snapshot with the same name '"+ targetName +"' may be in-progress");
902 System.err.println("Please check "+snapshotTmpDir+". If the snapshot has completed, ");
903 System.err.println("consider removing "+snapshotTmpDir+" by using the -overwrite option");
904 return 1;
905 }
906 }
907 }
908
909
910
911
912 try {
913 LOG.info("Copy Snapshot Manifest");
914 FileUtil.copy(inputFs, snapshotDir, outputFs, initialOutputSnapshotDir, false, false, conf);
915 } catch (IOException e) {
916 throw new ExportSnapshotException("Failed to copy the snapshot directory: from=" +
917 snapshotDir + " to=" + initialOutputSnapshotDir, e);
918 }
919
920
921 if (!targetName.equals(snapshotName)) {
922 SnapshotDescription snapshotDesc =
923 SnapshotDescriptionUtils.readSnapshotInfo(inputFs, snapshotDir)
924 .toBuilder()
925 .setName(targetName)
926 .build();
927 SnapshotDescriptionUtils.writeSnapshotInfo(snapshotDesc, snapshotTmpDir, outputFs);
928 }
929
930
931
932
933 try {
934 runCopyJob(inputRoot, outputRoot, snapshotName, snapshotDir, verifyChecksum,
935 filesUser, filesGroup, filesMode, mappers, bandwidthMB);
936
937 LOG.info("Finalize the Snapshot Export");
938 if (!skipTmp) {
939
940 if (!outputFs.rename(snapshotTmpDir, outputSnapshotDir)) {
941 throw new ExportSnapshotException("Unable to rename snapshot directory from=" +
942 snapshotTmpDir + " to=" + outputSnapshotDir);
943 }
944 }
945
946
947 if (verifyTarget) {
948 LOG.info("Verify snapshot integrity");
949 verifySnapshot(conf, outputFs, outputRoot, outputSnapshotDir);
950 }
951
952 LOG.info("Export Completed: " + targetName);
953 return 0;
954 } catch (Exception e) {
955 LOG.error("Snapshot export failed", e);
956 if (!skipTmp) {
957 outputFs.delete(snapshotTmpDir, true);
958 }
959 outputFs.delete(outputSnapshotDir, true);
960 return 1;
961 } finally {
962 IOUtils.closeStream(inputFs);
963 IOUtils.closeStream(outputFs);
964 }
965 }
966
967
968 private void printUsageAndExit() {
969 System.err.printf("Usage: bin/hbase %s [options]%n", getClass().getName());
970 System.err.println(" where [options] are:");
971 System.err.println(" -h|-help Show this help and exit.");
972 System.err.println(" -snapshot NAME Snapshot to restore.");
973 System.err.println(" -copy-to NAME Remote destination hdfs://");
974 System.err.println(" -copy-from NAME Input folder hdfs:// (default hbase.rootdir)");
975 System.err.println(" -no-checksum-verify Do not verify checksum, use name+length only.");
976 System.err.println(" -no-target-verify Do not verify the integrity of the \\" +
977 "exported snapshot.");
978 System.err.println(" -overwrite Rewrite the snapshot manifest if already exists");
979 System.err.println(" -chuser USERNAME Change the owner of the files " +
980 "to the specified one.");
981 System.err.println(" -chgroup GROUP Change the group of the files to " +
982 "the specified one.");
983 System.err.println(" -chmod MODE Change the permission of the files " +
984 "to the specified one.");
985 System.err.println(" -mappers Number of mappers to use during the " +
986 "copy (mapreduce.job.maps).");
987 System.err.println(" -bandwidth Limit bandwidth to this value in MB/second.");
988 System.err.println();
989 System.err.println("Examples:");
990 System.err.println(" hbase " + getClass().getName() + " \\");
991 System.err.println(" -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase \\");
992 System.err.println(" -chuser MyUser -chgroup MyGroup -chmod 700 -mappers 16");
993 System.err.println();
994 System.err.println(" hbase " + getClass().getName() + " \\");
995 System.err.println(" -snapshot MySnapshot -copy-from hdfs://srv2:8082/hbase \\");
996 System.err.println(" -copy-to hdfs://srv1:50070/hbase \\");
997 System.exit(1);
998 }
999
1000
1001
1002
1003
1004
1005
1006
1007 static int innerMain(final Configuration conf, final String [] args) throws Exception {
1008 return ToolRunner.run(conf, new ExportSnapshot(), args);
1009 }
1010
1011 public static void main(String[] args) throws Exception {
1012 System.exit(innerMain(HBaseConfiguration.create(), args));
1013 }
1014 }