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