001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.snapshot; 019 020import java.io.BufferedInputStream; 021import java.io.DataInput; 022import java.io.DataOutput; 023import java.io.FileNotFoundException; 024import java.io.IOException; 025import java.io.InputStream; 026import java.util.ArrayList; 027import java.util.Collection; 028import java.util.Collections; 029import java.util.Comparator; 030import java.util.HashMap; 031import java.util.HashSet; 032import java.util.LinkedList; 033import java.util.List; 034import java.util.Map; 035import java.util.Set; 036import java.util.concurrent.ExecutionException; 037import java.util.concurrent.ExecutorService; 038import java.util.concurrent.Executors; 039import java.util.concurrent.Future; 040import java.util.function.BiConsumer; 041import java.util.stream.Collectors; 042import org.apache.hadoop.conf.Configuration; 043import org.apache.hadoop.fs.FSDataInputStream; 044import org.apache.hadoop.fs.FSDataOutputStream; 045import org.apache.hadoop.fs.FileChecksum; 046import org.apache.hadoop.fs.FileStatus; 047import org.apache.hadoop.fs.FileSystem; 048import org.apache.hadoop.fs.Path; 049import org.apache.hadoop.fs.permission.FsPermission; 050import org.apache.hadoop.hbase.HBaseConfiguration; 051import org.apache.hadoop.hbase.HConstants; 052import org.apache.hadoop.hbase.TableName; 053import org.apache.hadoop.hbase.client.RegionInfo; 054import org.apache.hadoop.hbase.io.FileLink; 055import org.apache.hadoop.hbase.io.HFileLink; 056import org.apache.hadoop.hbase.io.WALLink; 057import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream; 058import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; 059import org.apache.hadoop.hbase.mob.MobUtils; 060import org.apache.hadoop.hbase.regionserver.StoreFileInfo; 061import org.apache.hadoop.hbase.util.AbstractHBaseTool; 062import org.apache.hadoop.hbase.util.CommonFSUtils; 063import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 064import org.apache.hadoop.hbase.util.FSUtils; 065import org.apache.hadoop.hbase.util.HFileArchiveUtil; 066import org.apache.hadoop.hbase.util.Pair; 067import org.apache.hadoop.hbase.util.Strings; 068import org.apache.hadoop.io.BytesWritable; 069import org.apache.hadoop.io.NullWritable; 070import org.apache.hadoop.io.Writable; 071import org.apache.hadoop.mapreduce.InputFormat; 072import org.apache.hadoop.mapreduce.InputSplit; 073import org.apache.hadoop.mapreduce.Job; 074import org.apache.hadoop.mapreduce.JobContext; 075import org.apache.hadoop.mapreduce.Mapper; 076import org.apache.hadoop.mapreduce.RecordReader; 077import org.apache.hadoop.mapreduce.TaskAttemptContext; 078import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; 079import org.apache.hadoop.mapreduce.security.TokenCache; 080import org.apache.hadoop.util.ReflectionUtils; 081import org.apache.hadoop.util.StringUtils; 082import org.apache.hadoop.util.Tool; 083import org.apache.yetus.audience.InterfaceAudience; 084import org.slf4j.Logger; 085import org.slf4j.LoggerFactory; 086 087import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; 088import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; 089import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; 090import org.apache.hbase.thirdparty.org.apache.commons.cli.Option; 091 092import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 093import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; 094import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo; 095import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; 096 097/** 098 * Export the specified snapshot to a given FileSystem. The .snapshot/name folder is copied to the 099 * destination cluster and then all the hfiles/wals are copied using a Map-Reduce Job in the 100 * .archive/ location. When everything is done, the second cluster can restore the snapshot. 101 */ 102@InterfaceAudience.Public 103public class ExportSnapshot extends AbstractHBaseTool implements Tool { 104 public static final String NAME = "exportsnapshot"; 105 /** Configuration prefix for overrides for the source filesystem */ 106 public static final String CONF_SOURCE_PREFIX = NAME + ".from."; 107 /** Configuration prefix for overrides for the destination filesystem */ 108 public static final String CONF_DEST_PREFIX = NAME + ".to."; 109 110 private static final Logger LOG = LoggerFactory.getLogger(ExportSnapshot.class); 111 112 private static final String MR_NUM_MAPS = "mapreduce.job.maps"; 113 private static final String CONF_NUM_SPLITS = "snapshot.export.format.splits"; 114 private static final String CONF_SNAPSHOT_NAME = "snapshot.export.format.snapshot.name"; 115 private static final String CONF_SNAPSHOT_DIR = "snapshot.export.format.snapshot.dir"; 116 private static final String CONF_FILES_USER = "snapshot.export.files.attributes.user"; 117 private static final String CONF_FILES_GROUP = "snapshot.export.files.attributes.group"; 118 private static final String CONF_FILES_MODE = "snapshot.export.files.attributes.mode"; 119 private static final String CONF_CHECKSUM_VERIFY = "snapshot.export.checksum.verify"; 120 private static final String CONF_OUTPUT_ROOT = "snapshot.export.output.root"; 121 private static final String CONF_INPUT_ROOT = "snapshot.export.input.root"; 122 private static final String CONF_BUFFER_SIZE = "snapshot.export.buffer.size"; 123 private static final String CONF_MAP_GROUP = "snapshot.export.default.map.group"; 124 private static final String CONF_BANDWIDTH_MB = "snapshot.export.map.bandwidth.mb"; 125 private static final String CONF_MR_JOB_NAME = "mapreduce.job.name"; 126 private static final String CONF_INPUT_FILE_GROUPER_CLASS = 127 "snapshot.export.input.file.grouper.class"; 128 private static final String CONF_INPUT_FILE_LOCATION_RESOLVER_CLASS = 129 "snapshot.export.input.file.location.resolver.class"; 130 protected static final String CONF_SKIP_TMP = "snapshot.export.skip.tmp"; 131 private static final String CONF_COPY_MANIFEST_THREADS = 132 "snapshot.export.copy.references.threads"; 133 private static final int DEFAULT_COPY_MANIFEST_THREADS = 134 Runtime.getRuntime().availableProcessors(); 135 private static final String CONF_STORAGE_POLICY = "snapshot.export.storage.policy.family"; 136 137 static class Testing { 138 static final String CONF_TEST_FAILURE = "test.snapshot.export.failure"; 139 static final String CONF_TEST_FAILURE_COUNT = "test.snapshot.export.failure.count"; 140 int failuresCountToInject = 0; 141 int injectedFailureCount = 0; 142 } 143 144 // Command line options and defaults. 145 static final class Options { 146 static final Option SNAPSHOT = new Option(null, "snapshot", true, "Snapshot to restore."); 147 static final Option TARGET_NAME = 148 new Option(null, "target", true, "Target name for the snapshot."); 149 static final Option COPY_TO = 150 new Option(null, "copy-to", true, "Remote " + "destination hdfs://"); 151 static final Option COPY_FROM = 152 new Option(null, "copy-from", true, "Input folder hdfs:// (default hbase.rootdir)"); 153 static final Option NO_CHECKSUM_VERIFY = new Option(null, "no-checksum-verify", false, 154 "Do not verify checksum, use name+length only."); 155 static final Option NO_TARGET_VERIFY = new Option(null, "no-target-verify", false, 156 "Do not verify the exported snapshot's expiration status and integrity."); 157 static final Option NO_SOURCE_VERIFY = new Option(null, "no-source-verify", false, 158 "Do not verify the source snapshot's expiration status and integrity."); 159 static final Option OVERWRITE = 160 new Option(null, "overwrite", false, "Rewrite the snapshot manifest if already exists."); 161 static final Option CHUSER = 162 new Option(null, "chuser", true, "Change the owner of the files to the specified one."); 163 static final Option CHGROUP = 164 new Option(null, "chgroup", true, "Change the group of the files to the specified one."); 165 static final Option CHMOD = 166 new Option(null, "chmod", true, "Change the permission of the files to the specified one."); 167 static final Option MAPPERS = new Option(null, "mappers", true, 168 "Number of mappers to use during the copy (mapreduce.job.maps). " 169 + "If you provide a --custom-file-grouper, " 170 + "then --mappers is interpreted as the number of mappers per group."); 171 static final Option BANDWIDTH = 172 new Option(null, "bandwidth", true, "Limit bandwidth to this value in MB/second."); 173 static final Option RESET_TTL = 174 new Option(null, "reset-ttl", false, "Do not copy TTL for the snapshot"); 175 static final Option STORAGE_POLICY = new Option(null, "storage-policy", true, 176 "Storage policy for export snapshot output directory, with format like: f=HOT&g=ALL_SDD"); 177 static final Option CUSTOM_FILE_GROUPER = new Option(null, "custom-file-grouper", true, 178 "Fully qualified class name of an implementation of ExportSnapshot.CustomFileGrouper. " 179 + "See JavaDoc on that class for more information."); 180 static final Option FILE_LOCATION_RESOLVER = new Option(null, "file-location-resolver", true, 181 "Fully qualified class name of an implementation of ExportSnapshot.FileLocationResolver. " 182 + "See JavaDoc on that class for more information."); 183 } 184 185 // Export Map-Reduce Counters, to keep track of the progress 186 public enum Counter { 187 MISSING_FILES, 188 FILES_COPIED, 189 FILES_SKIPPED, 190 COPY_FAILED, 191 BYTES_EXPECTED, 192 BYTES_SKIPPED, 193 BYTES_COPIED 194 } 195 196 /** 197 * Indicates the checksum comparison result. 198 */ 199 public enum ChecksumComparison { 200 TRUE, // checksum comparison is compatible and true. 201 FALSE, // checksum comparison is compatible and false. 202 INCOMPATIBLE, // checksum comparison is not compatible. 203 } 204 205 /** 206 * If desired, you may implement a CustomFileGrouper in order to influence how ExportSnapshot 207 * chooses which input files go into the MapReduce job's {@link InputSplit}s. Your implementation 208 * must return a data structure that contains each input file exactly once. Files that appear in 209 * separate entries in the top-level returned Collection are guaranteed to not be placed in the 210 * same InputSplit. This can be used to segregate your input files by the rack or host on which 211 * they are available, which, used in conjunction with {@link FileLocationResolver}, can improve 212 * the performance of your ExportSnapshot runs. To use this, pass the --custom-file-grouper 213 * argument with the fully qualified class name of an implementation of CustomFileGrouper that's 214 * on the classpath. If this argument is not used, no particular grouping logic will be applied. 215 */ 216 @InterfaceAudience.Public 217 public interface CustomFileGrouper { 218 Collection<Collection<Pair<SnapshotFileInfo, Long>>> 219 getGroupedInputFiles(final Collection<Pair<SnapshotFileInfo, Long>> snapshotFiles); 220 } 221 222 private static class NoopCustomFileGrouper implements CustomFileGrouper { 223 @Override 224 public Collection<Collection<Pair<SnapshotFileInfo, Long>>> 225 getGroupedInputFiles(final Collection<Pair<SnapshotFileInfo, Long>> snapshotFiles) { 226 return ImmutableList.of(snapshotFiles); 227 } 228 } 229 230 /** 231 * If desired, you may implement a FileLocationResolver in order to influence the _location_ 232 * metadata attached to each {@link InputSplit} that ExportSnapshot will submit to YARN. The 233 * method {@link #getLocationsForInputFiles(Collection)} method is called once for each InputSplit 234 * being constructed. Whatever is returned will ultimately be reported by that split's 235 * {@link InputSplit#getLocations()} method. This can be used to encourage YARN to schedule the 236 * ExportSnapshot's mappers on rack-local or host-local NodeManagers. To use this, pass the 237 * --file-location-resolver argument with the fully qualified class name of an implementation of 238 * FileLocationResolver that's on the classpath. If this argument is not used, no locations will 239 * be attached to the InputSplits. 240 */ 241 @InterfaceAudience.Public 242 public interface FileLocationResolver { 243 Set<String> getLocationsForInputFiles(final Collection<Pair<SnapshotFileInfo, Long>> files); 244 } 245 246 static class NoopFileLocationResolver implements FileLocationResolver { 247 @Override 248 public Set<String> getLocationsForInputFiles(Collection<Pair<SnapshotFileInfo, Long>> files) { 249 return ImmutableSet.of(); 250 } 251 } 252 253 private static class ExportMapper 254 extends Mapper<BytesWritable, NullWritable, NullWritable, NullWritable> { 255 private static final Logger LOG = LoggerFactory.getLogger(ExportMapper.class); 256 final static int REPORT_SIZE = 1 * 1024 * 1024; 257 final static int BUFFER_SIZE = 64 * 1024; 258 259 private boolean verifyChecksum; 260 private String filesGroup; 261 private String filesUser; 262 private short filesMode; 263 private int bufferSize; 264 265 private FileSystem outputFs; 266 private Path outputArchive; 267 private Path outputRoot; 268 269 private FileSystem inputFs; 270 private Path inputArchive; 271 private Path inputRoot; 272 273 private static Testing testing = new Testing(); 274 275 @Override 276 public void setup(Context context) throws IOException { 277 Configuration conf = context.getConfiguration(); 278 279 Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX); 280 Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX); 281 282 verifyChecksum = conf.getBoolean(CONF_CHECKSUM_VERIFY, true); 283 284 filesGroup = conf.get(CONF_FILES_GROUP); 285 filesUser = conf.get(CONF_FILES_USER); 286 filesMode = (short) conf.getInt(CONF_FILES_MODE, 0); 287 outputRoot = new Path(conf.get(CONF_OUTPUT_ROOT)); 288 inputRoot = new Path(conf.get(CONF_INPUT_ROOT)); 289 290 inputArchive = new Path(inputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY); 291 outputArchive = new Path(outputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY); 292 293 try { 294 inputFs = FileSystem.get(inputRoot.toUri(), srcConf); 295 } catch (IOException e) { 296 throw new IOException("Could not get the input FileSystem with root=" + inputRoot, e); 297 } 298 299 try { 300 outputFs = FileSystem.get(outputRoot.toUri(), destConf); 301 } catch (IOException e) { 302 throw new IOException("Could not get the output FileSystem with root=" + outputRoot, e); 303 } 304 305 // Use the default block size of the outputFs if bigger 306 int defaultBlockSize = Math.max((int) outputFs.getDefaultBlockSize(outputRoot), BUFFER_SIZE); 307 bufferSize = conf.getInt(CONF_BUFFER_SIZE, defaultBlockSize); 308 LOG.info("Using bufferSize=" + Strings.humanReadableInt(bufferSize)); 309 310 for (Counter c : Counter.values()) { 311 context.getCounter(c).increment(0); 312 } 313 if (context.getConfiguration().getBoolean(Testing.CONF_TEST_FAILURE, false)) { 314 testing.failuresCountToInject = conf.getInt(Testing.CONF_TEST_FAILURE_COUNT, 0); 315 // Get number of times we have already injected failure based on attempt number of this 316 // task. 317 testing.injectedFailureCount = context.getTaskAttemptID().getId(); 318 } 319 } 320 321 @Override 322 public void map(BytesWritable key, NullWritable value, Context context) 323 throws InterruptedException, IOException { 324 SnapshotFileInfo inputInfo = SnapshotFileInfo.parseFrom(key.copyBytes()); 325 Path outputPath = getOutputPath(inputInfo); 326 327 copyFile(context, inputInfo, outputPath); 328 } 329 330 /** 331 * Returns the location where the inputPath will be copied. 332 */ 333 private Path getOutputPath(final SnapshotFileInfo inputInfo) throws IOException { 334 Path path = null; 335 switch (inputInfo.getType()) { 336 case HFILE: 337 Path inputPath = new Path(inputInfo.getHfile()); 338 String family = inputPath.getParent().getName(); 339 TableName table = HFileLink.getReferencedTableName(inputPath.getName()); 340 String region = HFileLink.getReferencedRegionName(inputPath.getName()); 341 String hfile = HFileLink.getReferencedHFileName(inputPath.getName()); 342 path = new Path(CommonFSUtils.getTableDir(new Path("./"), table), 343 new Path(region, new Path(family, hfile))); 344 break; 345 case WAL: 346 LOG.warn("snapshot does not keeps WALs: " + inputInfo); 347 break; 348 default: 349 throw new IOException("Invalid File Type: " + inputInfo.getType().toString()); 350 } 351 return new Path(outputArchive, path); 352 } 353 354 @SuppressWarnings("checkstyle:linelength") 355 /** 356 * Used by TestExportSnapshot to test for retries when failures happen. Failure is injected in 357 * {@link #copyFile(Mapper.Context, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo, Path)}. 358 */ 359 private void injectTestFailure(final Context context, final SnapshotFileInfo inputInfo) 360 throws IOException { 361 if (!context.getConfiguration().getBoolean(Testing.CONF_TEST_FAILURE, false)) return; 362 if (testing.injectedFailureCount >= testing.failuresCountToInject) return; 363 testing.injectedFailureCount++; 364 context.getCounter(Counter.COPY_FAILED).increment(1); 365 LOG.debug("Injecting failure. Count: " + testing.injectedFailureCount); 366 throw new IOException(String.format("TEST FAILURE (%d of max %d): Unable to copy input=%s", 367 testing.injectedFailureCount, testing.failuresCountToInject, inputInfo)); 368 } 369 370 private void copyFile(final Context context, final SnapshotFileInfo inputInfo, 371 final Path outputPath) throws IOException { 372 // Get the file information 373 FileStatus inputStat = getSourceFileStatus(context, inputInfo); 374 375 // Verify if the output file exists and is the same that we want to copy 376 if (outputFs.exists(outputPath)) { 377 FileStatus outputStat = outputFs.getFileStatus(outputPath); 378 if (outputStat != null && sameFile(inputStat, outputStat)) { 379 LOG.info("Skip copy " + inputStat.getPath() + " to " + outputPath + ", same file."); 380 context.getCounter(Counter.FILES_SKIPPED).increment(1); 381 context.getCounter(Counter.BYTES_SKIPPED).increment(inputStat.getLen()); 382 return; 383 } 384 } 385 386 InputStream in = openSourceFile(context, inputInfo); 387 int bandwidthMB = context.getConfiguration().getInt(CONF_BANDWIDTH_MB, 100); 388 if (Integer.MAX_VALUE != bandwidthMB) { 389 in = new ThrottledInputStream(new BufferedInputStream(in), bandwidthMB * 1024 * 1024L); 390 } 391 392 Path inputPath = inputStat.getPath(); 393 try { 394 context.getCounter(Counter.BYTES_EXPECTED).increment(inputStat.getLen()); 395 396 // Ensure that the output folder is there and copy the file 397 createOutputPath(outputPath.getParent()); 398 String family = new Path(inputInfo.getHfile()).getParent().getName(); 399 String familyStoragePolicy = generateFamilyStoragePolicyKey(family); 400 if (stringIsNotEmpty(context.getConfiguration().get(familyStoragePolicy))) { 401 String key = context.getConfiguration().get(familyStoragePolicy); 402 LOG.info("Setting storage policy {} for {}", key, outputPath.getParent()); 403 outputFs.setStoragePolicy(outputPath.getParent(), key); 404 } 405 FSDataOutputStream out = outputFs.create(outputPath, true); 406 407 long stime = EnvironmentEdgeManager.currentTime(); 408 long totalBytesWritten = 409 copyData(context, inputPath, in, outputPath, out, inputStat.getLen()); 410 411 // Verify the file length and checksum 412 verifyCopyResult(inputStat, outputFs.getFileStatus(outputPath)); 413 414 long etime = EnvironmentEdgeManager.currentTime(); 415 LOG.info("copy completed for input=" + inputPath + " output=" + outputPath); 416 LOG.info("size=" + totalBytesWritten + " (" + Strings.humanReadableInt(totalBytesWritten) 417 + ")" + " time=" + StringUtils.formatTimeDiff(etime, stime) + String.format(" %.3fM/sec", 418 (totalBytesWritten / ((etime - stime) / 1000.0)) / 1048576.0)); 419 context.getCounter(Counter.FILES_COPIED).increment(1); 420 421 // Try to Preserve attributes 422 if (!preserveAttributes(outputPath, inputStat)) { 423 LOG.warn("You may have to run manually chown on: " + outputPath); 424 } 425 } catch (IOException e) { 426 LOG.error("Error copying " + inputPath + " to " + outputPath, e); 427 context.getCounter(Counter.COPY_FAILED).increment(1); 428 throw e; 429 } finally { 430 injectTestFailure(context, inputInfo); 431 } 432 } 433 434 /** 435 * Create the output folder and optionally set ownership. 436 */ 437 private void createOutputPath(final Path path) throws IOException { 438 if (filesUser == null && filesGroup == null) { 439 outputFs.mkdirs(path); 440 } else { 441 Path parent = path.getParent(); 442 if (!outputFs.exists(parent) && !parent.isRoot()) { 443 createOutputPath(parent); 444 } 445 outputFs.mkdirs(path); 446 if (filesUser != null || filesGroup != null) { 447 // override the owner when non-null user/group is specified 448 outputFs.setOwner(path, filesUser, filesGroup); 449 } 450 if (filesMode > 0) { 451 outputFs.setPermission(path, new FsPermission(filesMode)); 452 } 453 } 454 } 455 456 /** 457 * Try to Preserve the files attribute selected by the user copying them from the source file 458 * This is only required when you are exporting as a different user than "hbase" or on a system 459 * that doesn't have the "hbase" user. This is not considered a blocking failure since the user 460 * can force a chmod with the user that knows is available on the system. 461 */ 462 private boolean preserveAttributes(final Path path, final FileStatus refStat) { 463 FileStatus stat; 464 try { 465 stat = outputFs.getFileStatus(path); 466 } catch (IOException e) { 467 LOG.warn("Unable to get the status for file=" + path); 468 return false; 469 } 470 471 try { 472 if (filesMode > 0 && stat.getPermission().toShort() != filesMode) { 473 outputFs.setPermission(path, new FsPermission(filesMode)); 474 } else if (refStat != null && !stat.getPermission().equals(refStat.getPermission())) { 475 outputFs.setPermission(path, refStat.getPermission()); 476 } 477 } catch (IOException e) { 478 LOG.warn("Unable to set the permission for file=" + stat.getPath() + ": " + e.getMessage()); 479 return false; 480 } 481 482 boolean hasRefStat = (refStat != null); 483 String user = stringIsNotEmpty(filesUser) || !hasRefStat ? filesUser : refStat.getOwner(); 484 String group = stringIsNotEmpty(filesGroup) || !hasRefStat ? filesGroup : refStat.getGroup(); 485 if (stringIsNotEmpty(user) || stringIsNotEmpty(group)) { 486 try { 487 if (!(user.equals(stat.getOwner()) && group.equals(stat.getGroup()))) { 488 outputFs.setOwner(path, user, group); 489 } 490 } catch (IOException e) { 491 LOG.warn( 492 "Unable to set the owner/group for file=" + stat.getPath() + ": " + e.getMessage()); 493 LOG.warn("The user/group may not exist on the destination cluster: user=" + user 494 + " group=" + group); 495 return false; 496 } 497 } 498 499 return true; 500 } 501 502 private boolean stringIsNotEmpty(final String str) { 503 return str != null && !str.isEmpty(); 504 } 505 506 private long copyData(final Context context, final Path inputPath, final InputStream in, 507 final Path outputPath, final FSDataOutputStream out, final long inputFileSize) 508 throws IOException { 509 final String statusMessage = 510 "copied %s/" + Strings.humanReadableInt(inputFileSize) + " (%.1f%%)"; 511 512 try { 513 byte[] buffer = new byte[bufferSize]; 514 long totalBytesWritten = 0; 515 int reportBytes = 0; 516 int bytesRead; 517 518 while ((bytesRead = in.read(buffer)) > 0) { 519 out.write(buffer, 0, bytesRead); 520 totalBytesWritten += bytesRead; 521 reportBytes += bytesRead; 522 523 if (reportBytes >= REPORT_SIZE) { 524 context.getCounter(Counter.BYTES_COPIED).increment(reportBytes); 525 context 526 .setStatus(String.format(statusMessage, Strings.humanReadableInt(totalBytesWritten), 527 (totalBytesWritten / (float) inputFileSize) * 100.0f) + " from " + inputPath 528 + " to " + outputPath); 529 reportBytes = 0; 530 } 531 } 532 533 context.getCounter(Counter.BYTES_COPIED).increment(reportBytes); 534 context.setStatus(String.format(statusMessage, Strings.humanReadableInt(totalBytesWritten), 535 (totalBytesWritten / (float) inputFileSize) * 100.0f) + " from " + inputPath + " to " 536 + outputPath); 537 538 return totalBytesWritten; 539 } finally { 540 out.close(); 541 in.close(); 542 } 543 } 544 545 /** 546 * Try to open the "source" file. Throws an IOException if the communication with the inputFs 547 * fail or if the file is not found. 548 */ 549 private FSDataInputStream openSourceFile(Context context, final SnapshotFileInfo fileInfo) 550 throws IOException { 551 try { 552 Configuration conf = context.getConfiguration(); 553 FileLink link = null; 554 switch (fileInfo.getType()) { 555 case HFILE: 556 Path inputPath = new Path(fileInfo.getHfile()); 557 link = getFileLink(inputPath, conf); 558 break; 559 case WAL: 560 String serverName = fileInfo.getWalServer(); 561 String logName = fileInfo.getWalName(); 562 link = new WALLink(inputRoot, serverName, logName); 563 break; 564 default: 565 throw new IOException("Invalid File Type: " + fileInfo.getType().toString()); 566 } 567 return link.open(inputFs); 568 } catch (IOException e) { 569 context.getCounter(Counter.MISSING_FILES).increment(1); 570 LOG.error("Unable to open source file=" + fileInfo.toString(), e); 571 throw e; 572 } 573 } 574 575 private FileStatus getSourceFileStatus(Context context, final SnapshotFileInfo fileInfo) 576 throws IOException { 577 try { 578 Configuration conf = context.getConfiguration(); 579 FileLink link = null; 580 switch (fileInfo.getType()) { 581 case HFILE: 582 Path inputPath = new Path(fileInfo.getHfile()); 583 link = getFileLink(inputPath, conf); 584 break; 585 case WAL: 586 link = new WALLink(inputRoot, fileInfo.getWalServer(), fileInfo.getWalName()); 587 break; 588 default: 589 throw new IOException("Invalid File Type: " + fileInfo.getType().toString()); 590 } 591 return link.getFileStatus(inputFs); 592 } catch (FileNotFoundException e) { 593 context.getCounter(Counter.MISSING_FILES).increment(1); 594 LOG.error("Unable to get the status for source file=" + fileInfo.toString(), e); 595 throw e; 596 } catch (IOException e) { 597 LOG.error("Unable to get the status for source file=" + fileInfo.toString(), e); 598 throw e; 599 } 600 } 601 602 private FileLink getFileLink(Path path, Configuration conf) throws IOException { 603 String regionName = HFileLink.getReferencedRegionName(path.getName()); 604 TableName tableName = HFileLink.getReferencedTableName(path.getName()); 605 if (MobUtils.getMobRegionInfo(tableName).getEncodedName().equals(regionName)) { 606 return HFileLink.buildFromHFileLinkPattern(MobUtils.getQualifiedMobRootDir(conf), 607 HFileArchiveUtil.getArchivePath(conf), path); 608 } 609 return HFileLink.buildFromHFileLinkPattern(inputRoot, inputArchive, path); 610 } 611 612 private FileChecksum getFileChecksum(final FileSystem fs, final Path path) { 613 try { 614 return fs.getFileChecksum(path); 615 } catch (IOException e) { 616 LOG.warn("Unable to get checksum for file=" + path, e); 617 return null; 618 } 619 } 620 621 /** 622 * Utility to compare the file length and checksums for the paths specified. 623 */ 624 private void verifyCopyResult(final FileStatus inputStat, final FileStatus outputStat) 625 throws IOException { 626 long inputLen = inputStat.getLen(); 627 long outputLen = outputStat.getLen(); 628 Path inputPath = inputStat.getPath(); 629 Path outputPath = outputStat.getPath(); 630 631 if (inputLen != outputLen) { 632 throw new IOException("Mismatch in length of input:" + inputPath + " (" + inputLen 633 + ") and output:" + outputPath + " (" + outputLen + ")"); 634 } 635 636 // If length==0, we will skip checksum 637 if (inputLen != 0 && verifyChecksum) { 638 FileChecksum inChecksum = getFileChecksum(inputFs, inputStat.getPath()); 639 FileChecksum outChecksum = getFileChecksum(outputFs, outputStat.getPath()); 640 641 ChecksumComparison checksumComparison = verifyChecksum(inChecksum, outChecksum); 642 if (!checksumComparison.equals(ChecksumComparison.TRUE)) { 643 StringBuilder errMessage = new StringBuilder("Checksum mismatch between ") 644 .append(inputPath).append(" and ").append(outputPath).append("."); 645 646 boolean addSkipHint = false; 647 String inputScheme = inputFs.getScheme(); 648 String outputScheme = outputFs.getScheme(); 649 if (!inputScheme.equals(outputScheme)) { 650 errMessage.append(" Input and output filesystems are of different types.\n") 651 .append("Their checksum algorithms may be incompatible."); 652 addSkipHint = true; 653 } else if (inputStat.getBlockSize() != outputStat.getBlockSize()) { 654 errMessage.append(" Input and output differ in block-size."); 655 addSkipHint = true; 656 } else if ( 657 inChecksum != null && outChecksum != null 658 && !inChecksum.getAlgorithmName().equals(outChecksum.getAlgorithmName()) 659 ) { 660 errMessage.append(" Input and output checksum algorithms are of different types."); 661 addSkipHint = true; 662 } 663 if (addSkipHint) { 664 errMessage 665 .append(" You can choose file-level checksum validation via " 666 + "-Ddfs.checksum.combine.mode=COMPOSITE_CRC when block-sizes" 667 + " or filesystems are different.\n") 668 .append(" Or you can skip checksum-checks altogether with -no-checksum-verify,") 669 .append( 670 " for the table backup scenario, you should use -i option to skip checksum-checks.\n") 671 .append(" (NOTE: By skipping checksums, one runs the risk of " 672 + "masking data-corruption during file-transfer.)\n"); 673 } 674 throw new IOException(errMessage.toString()); 675 } 676 } 677 } 678 679 /** 680 * Utility to compare checksums 681 */ 682 private ChecksumComparison verifyChecksum(final FileChecksum inChecksum, 683 final FileChecksum outChecksum) { 684 // If the input or output checksum is null, or the algorithms of input and output are not 685 // equal, that means there is no comparison 686 // and return not compatible. else if matched, return compatible with the matched result. 687 if ( 688 inChecksum == null || outChecksum == null 689 || !inChecksum.getAlgorithmName().equals(outChecksum.getAlgorithmName()) 690 ) { 691 return ChecksumComparison.INCOMPATIBLE; 692 } else if (inChecksum.equals(outChecksum)) { 693 return ChecksumComparison.TRUE; 694 } 695 return ChecksumComparison.FALSE; 696 } 697 698 /** 699 * Check if the two files are equal by looking at the file length, and at the checksum (if user 700 * has specified the verifyChecksum flag). 701 */ 702 private boolean sameFile(final FileStatus inputStat, final FileStatus outputStat) { 703 // Not matching length 704 if (inputStat.getLen() != outputStat.getLen()) return false; 705 706 // Mark files as equals, since user asked for no checksum verification 707 if (!verifyChecksum) return true; 708 709 // If checksums are not available, files are not the same. 710 FileChecksum inChecksum = getFileChecksum(inputFs, inputStat.getPath()); 711 if (inChecksum == null) return false; 712 713 FileChecksum outChecksum = getFileChecksum(outputFs, outputStat.getPath()); 714 if (outChecksum == null) return false; 715 716 return inChecksum.equals(outChecksum); 717 } 718 } 719 720 // ========================================================================== 721 // Input Format 722 // ========================================================================== 723 724 /** 725 * Extract the list of files (HFiles/WALs) to copy using Map-Reduce. 726 * @return list of files referenced by the snapshot (pair of path and size) 727 */ 728 private static List<Pair<SnapshotFileInfo, Long>> getSnapshotFiles(final Configuration conf, 729 final FileSystem fs, final Path snapshotDir) throws IOException { 730 SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); 731 732 final List<Pair<SnapshotFileInfo, Long>> files = new ArrayList<>(); 733 final TableName table = TableName.valueOf(snapshotDesc.getTable()); 734 735 // Get snapshot files 736 LOG.info("Loading Snapshot '" + snapshotDesc.getName() + "' hfile list"); 737 Set<String> addedFiles = new HashSet<>(); 738 SnapshotReferenceUtil.visitReferencedFiles(conf, fs, snapshotDir, snapshotDesc, 739 new SnapshotReferenceUtil.SnapshotVisitor() { 740 @Override 741 public void storeFile(final RegionInfo regionInfo, final String family, 742 final SnapshotRegionManifest.StoreFile storeFile) throws IOException { 743 Pair<SnapshotFileInfo, Long> snapshotFileAndSize = null; 744 if (!storeFile.hasReference()) { 745 String region = regionInfo.getEncodedName(); 746 String hfile = storeFile.getName(); 747 snapshotFileAndSize = getSnapshotFileAndSize(fs, conf, table, region, family, hfile, 748 storeFile.hasFileSize() ? storeFile.getFileSize() : -1); 749 } else { 750 Pair<String, String> referredToRegionAndFile = 751 StoreFileInfo.getReferredToRegionAndFile(storeFile.getName()); 752 String referencedRegion = referredToRegionAndFile.getFirst(); 753 String referencedHFile = referredToRegionAndFile.getSecond(); 754 snapshotFileAndSize = getSnapshotFileAndSize(fs, conf, table, referencedRegion, family, 755 referencedHFile, storeFile.hasFileSize() ? storeFile.getFileSize() : -1); 756 } 757 String fileToExport = snapshotFileAndSize.getFirst().getHfile(); 758 if (!addedFiles.contains(fileToExport)) { 759 files.add(snapshotFileAndSize); 760 addedFiles.add(fileToExport); 761 } else { 762 LOG.debug("Skip the existing file: {}.", fileToExport); 763 } 764 } 765 }); 766 767 return files; 768 } 769 770 private static Pair<SnapshotFileInfo, Long> getSnapshotFileAndSize(FileSystem fs, 771 Configuration conf, TableName table, String region, String family, String hfile, long size) 772 throws IOException { 773 Path path = HFileLink.createPath(table, region, family, hfile); 774 SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder().setType(SnapshotFileInfo.Type.HFILE) 775 .setHfile(path.toString()).build(); 776 if (size == -1) { 777 size = HFileLink.buildFromHFileLinkPattern(conf, path).getFileStatus(fs).getLen(); 778 } 779 return new Pair<>(fileInfo, size); 780 } 781 782 /** 783 * Given a list of file paths and sizes, create around ngroups in as balanced a way as possible. 784 * The groups created will have similar amounts of bytes. 785 * <p> 786 * The algorithm used is pretty straightforward; the file list is sorted by size, and then each 787 * group fetch the bigger file available, iterating through groups alternating the direction. 788 */ 789 static List<List<Pair<SnapshotFileInfo, Long>>> getBalancedSplits( 790 final Collection<Pair<SnapshotFileInfo, Long>> unsortedFiles, final int ngroups) { 791 List<Pair<SnapshotFileInfo, Long>> files = new ArrayList<>(unsortedFiles); 792 // Sort files by size, from small to big 793 Collections.sort(files, new Comparator<Pair<SnapshotFileInfo, Long>>() { 794 public int compare(Pair<SnapshotFileInfo, Long> a, Pair<SnapshotFileInfo, Long> b) { 795 long r = a.getSecond() - b.getSecond(); 796 return (r < 0) ? -1 : ((r > 0) ? 1 : 0); 797 } 798 }); 799 800 // create balanced groups 801 List<List<Pair<SnapshotFileInfo, Long>>> fileGroups = new LinkedList<>(); 802 int hi = files.size() - 1; 803 int lo = 0; 804 805 List<Pair<SnapshotFileInfo, Long>> group; 806 int dir = 1; 807 int g = 0; 808 809 while (hi >= lo) { 810 if (g == fileGroups.size()) { 811 group = new LinkedList<>(); 812 fileGroups.add(group); 813 } else { 814 group = fileGroups.get(g); 815 } 816 817 Pair<SnapshotFileInfo, Long> fileInfo = files.get(hi--); 818 819 // add the hi one 820 group.add(fileInfo); 821 822 // change direction when at the end or the beginning 823 g += dir; 824 if (g == ngroups) { 825 dir = -1; 826 g = ngroups - 1; 827 } else if (g < 0) { 828 dir = 1; 829 g = 0; 830 } 831 } 832 833 return fileGroups; 834 } 835 836 static class ExportSnapshotInputFormat extends InputFormat<BytesWritable, NullWritable> { 837 @Override 838 public RecordReader<BytesWritable, NullWritable> createRecordReader(InputSplit split, 839 TaskAttemptContext tac) throws IOException, InterruptedException { 840 return new ExportSnapshotRecordReader(((ExportSnapshotInputSplit) split).getSplitKeys()); 841 } 842 843 @Override 844 public List<InputSplit> getSplits(JobContext context) throws IOException, InterruptedException { 845 Configuration conf = context.getConfiguration(); 846 Path snapshotDir = new Path(conf.get(CONF_SNAPSHOT_DIR)); 847 FileSystem fs = FileSystem.get(snapshotDir.toUri(), conf); 848 849 List<Pair<SnapshotFileInfo, Long>> snapshotFiles = getSnapshotFiles(conf, fs, snapshotDir); 850 851 Collection<List<Pair<SnapshotFileInfo, Long>>> balancedGroups = 852 groupFilesForSplits(conf, snapshotFiles); 853 854 Class<? extends FileLocationResolver> fileLocationResolverClass = 855 conf.getClass(CONF_INPUT_FILE_LOCATION_RESOLVER_CLASS, NoopFileLocationResolver.class, 856 FileLocationResolver.class); 857 FileLocationResolver fileLocationResolver = 858 ReflectionUtils.newInstance(fileLocationResolverClass, conf); 859 LOG.info("FileLocationResolver {} will provide location metadata for each InputSplit", 860 fileLocationResolverClass); 861 862 List<InputSplit> splits = new ArrayList<>(balancedGroups.size()); 863 for (Collection<Pair<SnapshotFileInfo, Long>> files : balancedGroups) { 864 splits.add(new ExportSnapshotInputSplit(files, fileLocationResolver)); 865 } 866 return splits; 867 } 868 869 Collection<List<Pair<SnapshotFileInfo, Long>>> groupFilesForSplits(Configuration conf, 870 List<Pair<SnapshotFileInfo, Long>> snapshotFiles) { 871 int mappers = conf.getInt(CONF_NUM_SPLITS, 0); 872 if (mappers == 0 && !snapshotFiles.isEmpty()) { 873 mappers = 1 + (snapshotFiles.size() / conf.getInt(CONF_MAP_GROUP, 10)); 874 mappers = Math.min(mappers, snapshotFiles.size()); 875 conf.setInt(CONF_NUM_SPLITS, mappers); 876 conf.setInt(MR_NUM_MAPS, mappers); 877 } 878 879 Class<? extends CustomFileGrouper> inputFileGrouperClass = conf.getClass( 880 CONF_INPUT_FILE_GROUPER_CLASS, NoopCustomFileGrouper.class, CustomFileGrouper.class); 881 CustomFileGrouper customFileGrouper = 882 ReflectionUtils.newInstance(inputFileGrouperClass, conf); 883 Collection<Collection<Pair<SnapshotFileInfo, Long>>> groups = 884 customFileGrouper.getGroupedInputFiles(snapshotFiles); 885 886 LOG.info("CustomFileGrouper {} split input files into {} groups", inputFileGrouperClass, 887 groups.size()); 888 int mappersPerGroup = groups.isEmpty() ? 1 : Math.max(mappers / groups.size(), 1); 889 LOG.info( 890 "Splitting each group into {} InputSplits, " 891 + "to achieve closest possible amount of mappers to target of {}", 892 mappersPerGroup, mappers); 893 894 // Within each group, create splits of equal size. Groups are not mixed together. 895 return groups.stream().map(g -> getBalancedSplits(g, mappersPerGroup)) 896 .flatMap(Collection::stream).collect(Collectors.toList()); 897 } 898 899 static class ExportSnapshotInputSplit extends InputSplit implements Writable { 900 901 private List<Pair<BytesWritable, Long>> files; 902 private String[] locations; 903 private long length; 904 905 public ExportSnapshotInputSplit() { 906 this.files = null; 907 this.locations = null; 908 } 909 910 public ExportSnapshotInputSplit(final Collection<Pair<SnapshotFileInfo, Long>> snapshotFiles, 911 FileLocationResolver fileLocationResolver) { 912 this.files = new ArrayList<>(snapshotFiles.size()); 913 for (Pair<SnapshotFileInfo, Long> fileInfo : snapshotFiles) { 914 this.files.add( 915 new Pair<>(new BytesWritable(fileInfo.getFirst().toByteArray()), fileInfo.getSecond())); 916 this.length += fileInfo.getSecond(); 917 } 918 this.locations = 919 fileLocationResolver.getLocationsForInputFiles(snapshotFiles).toArray(new String[0]); 920 LOG.trace("This ExportSnapshotInputSplit has files {} of collective size {}, " 921 + "with location hints: {}", files, length, locations); 922 } 923 924 private List<Pair<BytesWritable, Long>> getSplitKeys() { 925 return files; 926 } 927 928 @Override 929 public long getLength() throws IOException, InterruptedException { 930 return length; 931 } 932 933 @Override 934 public String[] getLocations() throws IOException, InterruptedException { 935 return locations; 936 } 937 938 @Override 939 public void readFields(DataInput in) throws IOException { 940 int count = in.readInt(); 941 files = new ArrayList<>(count); 942 length = 0; 943 for (int i = 0; i < count; ++i) { 944 BytesWritable fileInfo = new BytesWritable(); 945 fileInfo.readFields(in); 946 long size = in.readLong(); 947 files.add(new Pair<>(fileInfo, size)); 948 length += size; 949 } 950 int locationCount = in.readInt(); 951 List<String> locations = new ArrayList<>(locationCount); 952 for (int i = 0; i < locationCount; ++i) { 953 locations.add(in.readUTF()); 954 } 955 this.locations = locations.toArray(new String[0]); 956 } 957 958 @Override 959 public void write(DataOutput out) throws IOException { 960 out.writeInt(files.size()); 961 for (final Pair<BytesWritable, Long> fileInfo : files) { 962 fileInfo.getFirst().write(out); 963 out.writeLong(fileInfo.getSecond()); 964 } 965 out.writeInt(locations.length); 966 for (String location : locations) { 967 out.writeUTF(location); 968 } 969 } 970 } 971 972 private static class ExportSnapshotRecordReader 973 extends RecordReader<BytesWritable, NullWritable> { 974 private final List<Pair<BytesWritable, Long>> files; 975 private long totalSize = 0; 976 private long procSize = 0; 977 private int index = -1; 978 979 ExportSnapshotRecordReader(final List<Pair<BytesWritable, Long>> files) { 980 this.files = files; 981 for (Pair<BytesWritable, Long> fileInfo : files) { 982 totalSize += fileInfo.getSecond(); 983 } 984 } 985 986 @Override 987 public void close() { 988 } 989 990 @Override 991 public BytesWritable getCurrentKey() { 992 return files.get(index).getFirst(); 993 } 994 995 @Override 996 public NullWritable getCurrentValue() { 997 return NullWritable.get(); 998 } 999 1000 @Override 1001 public float getProgress() { 1002 return (float) procSize / totalSize; 1003 } 1004 1005 @Override 1006 public void initialize(InputSplit split, TaskAttemptContext tac) { 1007 } 1008 1009 @Override 1010 public boolean nextKeyValue() { 1011 if (index >= 0) { 1012 procSize += files.get(index).getSecond(); 1013 } 1014 return (++index < files.size()); 1015 } 1016 } 1017 } 1018 1019 // ========================================================================== 1020 // Tool 1021 // ========================================================================== 1022 1023 /** 1024 * Run Map-Reduce Job to perform the files copy. 1025 */ 1026 private void runCopyJob(final Path inputRoot, final Path outputRoot, final String snapshotName, 1027 final Path snapshotDir, final boolean verifyChecksum, final String filesUser, 1028 final String filesGroup, final int filesMode, final int mappers, final int bandwidthMB, 1029 final String storagePolicy, final String customFileGrouper, final String fileLocationResolver) 1030 throws IOException, InterruptedException, ClassNotFoundException { 1031 Configuration conf = getConf(); 1032 if (filesGroup != null) conf.set(CONF_FILES_GROUP, filesGroup); 1033 if (filesUser != null) conf.set(CONF_FILES_USER, filesUser); 1034 if (mappers > 0) { 1035 conf.setInt(CONF_NUM_SPLITS, mappers); 1036 conf.setInt(MR_NUM_MAPS, mappers); 1037 } 1038 conf.setInt(CONF_FILES_MODE, filesMode); 1039 conf.setBoolean(CONF_CHECKSUM_VERIFY, verifyChecksum); 1040 conf.set(CONF_OUTPUT_ROOT, outputRoot.toString()); 1041 conf.set(CONF_INPUT_ROOT, inputRoot.toString()); 1042 conf.setInt(CONF_BANDWIDTH_MB, bandwidthMB); 1043 conf.set(CONF_SNAPSHOT_NAME, snapshotName); 1044 conf.set(CONF_SNAPSHOT_DIR, snapshotDir.toString()); 1045 if (storagePolicy != null) { 1046 for (Map.Entry<String, String> entry : storagePolicyPerFamily(storagePolicy).entrySet()) { 1047 conf.set(generateFamilyStoragePolicyKey(entry.getKey()), entry.getValue()); 1048 } 1049 } 1050 if (customFileGrouper != null) { 1051 conf.set(CONF_INPUT_FILE_GROUPER_CLASS, customFileGrouper); 1052 } 1053 if (fileLocationResolver != null) { 1054 conf.set(CONF_INPUT_FILE_LOCATION_RESOLVER_CLASS, fileLocationResolver); 1055 } 1056 1057 String jobname = conf.get(CONF_MR_JOB_NAME, "ExportSnapshot-" + snapshotName); 1058 Job job = new Job(conf); 1059 job.setJobName(jobname); 1060 job.setJarByClass(ExportSnapshot.class); 1061 TableMapReduceUtil.addDependencyJars(job); 1062 job.setMapperClass(ExportMapper.class); 1063 job.setInputFormatClass(ExportSnapshotInputFormat.class); 1064 job.setOutputFormatClass(NullOutputFormat.class); 1065 job.setMapSpeculativeExecution(false); 1066 job.setNumReduceTasks(0); 1067 1068 // Acquire the delegation Tokens 1069 Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX); 1070 TokenCache.obtainTokensForNamenodes(job.getCredentials(), new Path[] { inputRoot }, srcConf); 1071 Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX); 1072 TokenCache.obtainTokensForNamenodes(job.getCredentials(), new Path[] { outputRoot }, destConf); 1073 1074 // Run the MR Job 1075 if (!job.waitForCompletion(true)) { 1076 throw new ExportSnapshotException(job.getStatus().getFailureInfo()); 1077 } 1078 } 1079 1080 private void verifySnapshot(final SnapshotDescription snapshotDesc, final Configuration baseConf, 1081 final FileSystem fs, final Path rootDir, final Path snapshotDir) throws IOException { 1082 // Update the conf with the current root dir, since may be a different cluster 1083 Configuration conf = new Configuration(baseConf); 1084 CommonFSUtils.setRootDir(conf, rootDir); 1085 CommonFSUtils.setFsDefault(conf, CommonFSUtils.getRootDir(conf)); 1086 boolean isExpired = SnapshotDescriptionUtils.isExpiredSnapshot(snapshotDesc.getTtl(), 1087 snapshotDesc.getCreationTime(), EnvironmentEdgeManager.currentTime()); 1088 if (isExpired) { 1089 throw new SnapshotTTLExpiredException(ProtobufUtil.createSnapshotDesc(snapshotDesc)); 1090 } 1091 SnapshotReferenceUtil.verifySnapshot(conf, fs, snapshotDir, snapshotDesc); 1092 } 1093 1094 private void setConfigParallel(FileSystem outputFs, List<Path> traversedPath, 1095 BiConsumer<FileSystem, Path> task, Configuration conf) throws IOException { 1096 ExecutorService pool = Executors 1097 .newFixedThreadPool(conf.getInt(CONF_COPY_MANIFEST_THREADS, DEFAULT_COPY_MANIFEST_THREADS)); 1098 List<Future<Void>> futures = new ArrayList<>(); 1099 for (Path dstPath : traversedPath) { 1100 Future<Void> future = (Future<Void>) pool.submit(() -> task.accept(outputFs, dstPath)); 1101 futures.add(future); 1102 } 1103 try { 1104 for (Future<Void> future : futures) { 1105 future.get(); 1106 } 1107 } catch (InterruptedException | ExecutionException e) { 1108 throw new IOException(e); 1109 } finally { 1110 pool.shutdownNow(); 1111 } 1112 } 1113 1114 private void setOwnerParallel(FileSystem outputFs, String filesUser, String filesGroup, 1115 Configuration conf, List<Path> traversedPath) throws IOException { 1116 setConfigParallel(outputFs, traversedPath, (fs, path) -> { 1117 try { 1118 fs.setOwner(path, filesUser, filesGroup); 1119 } catch (IOException e) { 1120 throw new RuntimeException( 1121 "set owner for file " + path + " to " + filesUser + ":" + filesGroup + " failed", e); 1122 } 1123 }, conf); 1124 } 1125 1126 private void setPermissionParallel(final FileSystem outputFs, final short filesMode, 1127 final List<Path> traversedPath, final Configuration conf) throws IOException { 1128 if (filesMode <= 0) { 1129 return; 1130 } 1131 FsPermission perm = new FsPermission(filesMode); 1132 setConfigParallel(outputFs, traversedPath, (fs, path) -> { 1133 try { 1134 fs.setPermission(path, perm); 1135 } catch (IOException e) { 1136 throw new RuntimeException( 1137 "set permission for file " + path + " to " + filesMode + " failed", e); 1138 } 1139 }, conf); 1140 } 1141 1142 private Map<String, String> storagePolicyPerFamily(String storagePolicy) { 1143 Map<String, String> familyStoragePolicy = new HashMap<>(); 1144 for (String familyConf : storagePolicy.split("&")) { 1145 String[] familySplit = familyConf.split("="); 1146 if (familySplit.length != 2) { 1147 continue; 1148 } 1149 // family is key, storage policy is value 1150 familyStoragePolicy.put(familySplit[0], familySplit[1]); 1151 } 1152 return familyStoragePolicy; 1153 } 1154 1155 private static String generateFamilyStoragePolicyKey(String family) { 1156 return CONF_STORAGE_POLICY + "." + family; 1157 } 1158 1159 private boolean verifyTarget = true; 1160 private boolean verifySource = true; 1161 private boolean verifyChecksum = true; 1162 private String snapshotName = null; 1163 private String targetName = null; 1164 private boolean overwrite = false; 1165 private String filesGroup = null; 1166 private String filesUser = null; 1167 private Path outputRoot = null; 1168 private Path inputRoot = null; 1169 private int bandwidthMB = Integer.MAX_VALUE; 1170 private int filesMode = 0; 1171 private int mappers = 0; 1172 private boolean resetTtl = false; 1173 private String storagePolicy = null; 1174 private String customFileGrouper = null; 1175 private String fileLocationResolver = null; 1176 1177 @Override 1178 protected void processOptions(CommandLine cmd) { 1179 snapshotName = cmd.getOptionValue(Options.SNAPSHOT.getLongOpt(), snapshotName); 1180 targetName = cmd.getOptionValue(Options.TARGET_NAME.getLongOpt(), targetName); 1181 if (cmd.hasOption(Options.COPY_TO.getLongOpt())) { 1182 outputRoot = new Path(cmd.getOptionValue(Options.COPY_TO.getLongOpt())); 1183 } 1184 if (cmd.hasOption(Options.COPY_FROM.getLongOpt())) { 1185 inputRoot = new Path(cmd.getOptionValue(Options.COPY_FROM.getLongOpt())); 1186 } 1187 mappers = getOptionAsInt(cmd, Options.MAPPERS.getLongOpt(), mappers); 1188 filesUser = cmd.getOptionValue(Options.CHUSER.getLongOpt(), filesUser); 1189 filesGroup = cmd.getOptionValue(Options.CHGROUP.getLongOpt(), filesGroup); 1190 filesMode = getOptionAsInt(cmd, Options.CHMOD.getLongOpt(), filesMode, 8); 1191 bandwidthMB = getOptionAsInt(cmd, Options.BANDWIDTH.getLongOpt(), bandwidthMB); 1192 overwrite = cmd.hasOption(Options.OVERWRITE.getLongOpt()); 1193 // And verifyChecksum and verifyTarget with values read from old args in processOldArgs(...). 1194 verifyChecksum = !cmd.hasOption(Options.NO_CHECKSUM_VERIFY.getLongOpt()); 1195 verifyTarget = !cmd.hasOption(Options.NO_TARGET_VERIFY.getLongOpt()); 1196 verifySource = !cmd.hasOption(Options.NO_SOURCE_VERIFY.getLongOpt()); 1197 resetTtl = cmd.hasOption(Options.RESET_TTL.getLongOpt()); 1198 if (cmd.hasOption(Options.STORAGE_POLICY.getLongOpt())) { 1199 storagePolicy = cmd.getOptionValue(Options.STORAGE_POLICY.getLongOpt()); 1200 } 1201 if (cmd.hasOption(Options.CUSTOM_FILE_GROUPER.getLongOpt())) { 1202 customFileGrouper = cmd.getOptionValue(Options.CUSTOM_FILE_GROUPER.getLongOpt()); 1203 } 1204 if (cmd.hasOption(Options.FILE_LOCATION_RESOLVER.getLongOpt())) { 1205 fileLocationResolver = cmd.getOptionValue(Options.FILE_LOCATION_RESOLVER.getLongOpt()); 1206 } 1207 } 1208 1209 /** 1210 * Execute the export snapshot by copying the snapshot metadata, hfiles and wals. 1211 * @return 0 on success, and != 0 upon failure. 1212 */ 1213 @Override 1214 public int doWork() throws IOException { 1215 Configuration conf = getConf(); 1216 1217 // Check user options 1218 if (snapshotName == null) { 1219 System.err.println("Snapshot name not provided."); 1220 LOG.error("Use -h or --help for usage instructions."); 1221 return EXIT_FAILURE; 1222 } 1223 1224 if (outputRoot == null) { 1225 System.err 1226 .println("Destination file-system (--" + Options.COPY_TO.getLongOpt() + ") not provided."); 1227 LOG.error("Use -h or --help for usage instructions."); 1228 return EXIT_FAILURE; 1229 } 1230 1231 if (targetName == null) { 1232 targetName = snapshotName; 1233 } 1234 if (inputRoot == null) { 1235 inputRoot = CommonFSUtils.getRootDir(conf); 1236 } else { 1237 CommonFSUtils.setRootDir(conf, inputRoot); 1238 } 1239 1240 Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX); 1241 FileSystem inputFs = FileSystem.get(inputRoot.toUri(), srcConf); 1242 Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX); 1243 FileSystem outputFs = FileSystem.get(outputRoot.toUri(), destConf); 1244 boolean skipTmp = conf.getBoolean(CONF_SKIP_TMP, false) 1245 || conf.get(SnapshotDescriptionUtils.SNAPSHOT_WORKING_DIR) != null; 1246 Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, inputRoot); 1247 Path snapshotTmpDir = 1248 SnapshotDescriptionUtils.getWorkingSnapshotDir(targetName, outputRoot, destConf); 1249 Path outputSnapshotDir = 1250 SnapshotDescriptionUtils.getCompletedSnapshotDir(targetName, outputRoot); 1251 Path initialOutputSnapshotDir = skipTmp ? outputSnapshotDir : snapshotTmpDir; 1252 LOG.debug("inputFs={}, inputRoot={}", inputFs.getUri().toString(), inputRoot); 1253 LOG.debug("outputFs={}, outputRoot={}, skipTmp={}, initialOutputSnapshotDir={}", outputFs, 1254 outputRoot.toString(), skipTmp, initialOutputSnapshotDir); 1255 1256 // throw CorruptedSnapshotException if we can't read the snapshot info. 1257 SnapshotDescription sourceSnapshotDesc = 1258 SnapshotDescriptionUtils.readSnapshotInfo(inputFs, snapshotDir); 1259 1260 // Verify snapshot source before copying files 1261 if (verifySource) { 1262 LOG.info("Verify the source snapshot's expiration status and integrity."); 1263 verifySnapshot(sourceSnapshotDesc, srcConf, inputFs, inputRoot, snapshotDir); 1264 } 1265 1266 // Find the necessary directory which need to change owner and group 1267 Path needSetOwnerDir = SnapshotDescriptionUtils.getSnapshotRootDir(outputRoot); 1268 if (outputFs.exists(needSetOwnerDir)) { 1269 if (skipTmp) { 1270 needSetOwnerDir = outputSnapshotDir; 1271 } else { 1272 needSetOwnerDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(outputRoot, destConf); 1273 if (outputFs.exists(needSetOwnerDir)) { 1274 needSetOwnerDir = snapshotTmpDir; 1275 } 1276 } 1277 } 1278 1279 // Check if the snapshot already exists 1280 if (outputFs.exists(outputSnapshotDir)) { 1281 if (overwrite) { 1282 if (!outputFs.delete(outputSnapshotDir, true)) { 1283 System.err.println("Unable to remove existing snapshot directory: " + outputSnapshotDir); 1284 return EXIT_FAILURE; 1285 } 1286 } else { 1287 System.err.println("The snapshot '" + targetName + "' already exists in the destination: " 1288 + outputSnapshotDir); 1289 return EXIT_FAILURE; 1290 } 1291 } 1292 1293 if (!skipTmp) { 1294 // Check if the snapshot already in-progress 1295 if (outputFs.exists(snapshotTmpDir)) { 1296 if (overwrite) { 1297 if (!outputFs.delete(snapshotTmpDir, true)) { 1298 System.err 1299 .println("Unable to remove existing snapshot tmp directory: " + snapshotTmpDir); 1300 return EXIT_FAILURE; 1301 } 1302 } else { 1303 System.err 1304 .println("A snapshot with the same name '" + targetName + "' may be in-progress"); 1305 System.err 1306 .println("Please check " + snapshotTmpDir + ". If the snapshot has completed, "); 1307 System.err 1308 .println("consider removing " + snapshotTmpDir + " by using the -overwrite option"); 1309 return EXIT_FAILURE; 1310 } 1311 } 1312 } 1313 1314 // Step 1 - Copy fs1:/.snapshot/<snapshot> to fs2:/.snapshot/.tmp/<snapshot> 1315 // The snapshot references must be copied before the hfiles otherwise the cleaner 1316 // will remove them because they are unreferenced. 1317 List<Path> travesedPaths = new ArrayList<>(); 1318 boolean copySucceeded = false; 1319 try { 1320 LOG.info("Copy Snapshot Manifest from " + snapshotDir + " to " + initialOutputSnapshotDir); 1321 travesedPaths = 1322 FSUtils.copyFilesParallel(inputFs, snapshotDir, outputFs, initialOutputSnapshotDir, conf, 1323 conf.getInt(CONF_COPY_MANIFEST_THREADS, DEFAULT_COPY_MANIFEST_THREADS)); 1324 copySucceeded = true; 1325 } catch (IOException e) { 1326 throw new ExportSnapshotException("Failed to copy the snapshot directory: from=" + snapshotDir 1327 + " to=" + initialOutputSnapshotDir, e); 1328 } finally { 1329 if (copySucceeded) { 1330 if (filesUser != null || filesGroup != null) { 1331 LOG.warn( 1332 (filesUser == null ? "" : "Change the owner of " + needSetOwnerDir + " to " + filesUser) 1333 + (filesGroup == null 1334 ? "" 1335 : ", Change the group of " + needSetOwnerDir + " to " + filesGroup)); 1336 setOwnerParallel(outputFs, filesUser, filesGroup, conf, travesedPaths); 1337 } 1338 if (filesMode > 0) { 1339 LOG.warn("Change the permission of " + needSetOwnerDir + " to " + filesMode); 1340 setPermissionParallel(outputFs, (short) filesMode, travesedPaths, conf); 1341 } 1342 } 1343 } 1344 1345 // Write a new .snapshotinfo if the target name is different from the source name or we want to 1346 // reset TTL for target snapshot. 1347 if (!targetName.equals(snapshotName) || resetTtl) { 1348 SnapshotDescription.Builder snapshotDescBuilder = 1349 SnapshotDescriptionUtils.readSnapshotInfo(inputFs, snapshotDir).toBuilder(); 1350 if (!targetName.equals(snapshotName)) { 1351 snapshotDescBuilder.setName(targetName); 1352 } 1353 if (resetTtl) { 1354 snapshotDescBuilder.setTtl(HConstants.DEFAULT_SNAPSHOT_TTL); 1355 } 1356 SnapshotDescriptionUtils.writeSnapshotInfo(snapshotDescBuilder.build(), 1357 initialOutputSnapshotDir, outputFs); 1358 if (filesUser != null || filesGroup != null) { 1359 outputFs.setOwner( 1360 new Path(initialOutputSnapshotDir, SnapshotDescriptionUtils.SNAPSHOTINFO_FILE), filesUser, 1361 filesGroup); 1362 } 1363 if (filesMode > 0) { 1364 outputFs.setPermission( 1365 new Path(initialOutputSnapshotDir, SnapshotDescriptionUtils.SNAPSHOTINFO_FILE), 1366 new FsPermission((short) filesMode)); 1367 } 1368 } 1369 1370 // Step 2 - Start MR Job to copy files 1371 // The snapshot references must be copied before the files otherwise the files gets removed 1372 // by the HFileArchiver, since they have no references. 1373 try { 1374 runCopyJob(inputRoot, outputRoot, snapshotName, snapshotDir, verifyChecksum, filesUser, 1375 filesGroup, filesMode, mappers, bandwidthMB, storagePolicy, customFileGrouper, 1376 fileLocationResolver); 1377 1378 LOG.info("Finalize the Snapshot Export"); 1379 if (!skipTmp) { 1380 // Step 3 - Rename fs2:/.snapshot/.tmp/<snapshot> fs2:/.snapshot/<snapshot> 1381 if (!outputFs.rename(snapshotTmpDir, outputSnapshotDir)) { 1382 throw new ExportSnapshotException("Unable to rename snapshot directory from=" 1383 + snapshotTmpDir + " to=" + outputSnapshotDir); 1384 } 1385 } 1386 1387 // Step 4 - Verify snapshot integrity 1388 if (verifyTarget) { 1389 LOG.info("Verify the exported snapshot's expiration status and integrity."); 1390 SnapshotDescription targetSnapshotDesc = 1391 SnapshotDescriptionUtils.readSnapshotInfo(outputFs, outputSnapshotDir); 1392 verifySnapshot(targetSnapshotDesc, destConf, outputFs, outputRoot, outputSnapshotDir); 1393 } 1394 1395 LOG.info("Export Completed: " + targetName); 1396 return EXIT_SUCCESS; 1397 } catch (Exception e) { 1398 LOG.error("Snapshot export failed", e); 1399 if (!skipTmp) { 1400 outputFs.delete(snapshotTmpDir, true); 1401 } 1402 outputFs.delete(outputSnapshotDir, true); 1403 return EXIT_FAILURE; 1404 } 1405 } 1406 1407 @Override 1408 protected void printUsage() { 1409 super.printUsage(); 1410 System.out.println("\n" + "Examples:\n" + " hbase snapshot export \\\n" 1411 + " --snapshot MySnapshot --copy-to hdfs://srv2:8082/hbase \\\n" 1412 + " --chuser MyUser --chgroup MyGroup --chmod 700 --mappers 16\n" + "\n" 1413 + " hbase snapshot export \\\n" 1414 + " --snapshot MySnapshot --copy-from hdfs://srv2:8082/hbase \\\n" 1415 + " --copy-to hdfs://srv1:50070/hbase"); 1416 } 1417 1418 @Override 1419 protected void addOptions() { 1420 addRequiredOption(Options.SNAPSHOT); 1421 addOption(Options.COPY_TO); 1422 addOption(Options.COPY_FROM); 1423 addOption(Options.TARGET_NAME); 1424 addOption(Options.NO_CHECKSUM_VERIFY); 1425 addOption(Options.NO_TARGET_VERIFY); 1426 addOption(Options.NO_SOURCE_VERIFY); 1427 addOption(Options.OVERWRITE); 1428 addOption(Options.CHUSER); 1429 addOption(Options.CHGROUP); 1430 addOption(Options.CHMOD); 1431 addOption(Options.MAPPERS); 1432 addOption(Options.BANDWIDTH); 1433 addOption(Options.RESET_TTL); 1434 addOption(Options.CUSTOM_FILE_GROUPER); 1435 addOption(Options.FILE_LOCATION_RESOLVER); 1436 } 1437 1438 public static void main(String[] args) { 1439 new ExportSnapshot().doStaticMain(args); 1440 } 1441}