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