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