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.backup.impl; 019 020import com.google.errorprone.annotations.RestrictedApi; 021import java.io.IOException; 022import java.util.ArrayList; 023import java.util.Collections; 024import java.util.HashMap; 025import java.util.List; 026import java.util.Map; 027import java.util.Map.Entry; 028import java.util.TreeMap; 029import org.apache.hadoop.conf.Configuration; 030import org.apache.hadoop.fs.FSDataInputStream; 031import org.apache.hadoop.fs.FSDataOutputStream; 032import org.apache.hadoop.fs.FileStatus; 033import org.apache.hadoop.fs.FileSystem; 034import org.apache.hadoop.fs.Path; 035import org.apache.hadoop.hbase.ServerName; 036import org.apache.hadoop.hbase.TableName; 037import org.apache.hadoop.hbase.backup.BackupInfo; 038import org.apache.hadoop.hbase.backup.BackupType; 039import org.apache.hadoop.hbase.backup.HBackupFileSystem; 040import org.apache.hadoop.hbase.backup.util.BackupUtils; 041import org.apache.yetus.audience.InterfaceAudience; 042import org.slf4j.Logger; 043import org.slf4j.LoggerFactory; 044 045import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 046import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos; 047import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; 048 049/** 050 * Backup manifest contains all the meta data of a backup image. The manifest info will be bundled 051 * as manifest file together with data. So that each backup image will contain all the info needed 052 * for restore. BackupManifest is a storage container for BackupImage. It is responsible for 053 * storing/reading backup image data and has some additional utility methods. 054 */ 055@InterfaceAudience.Private 056public class BackupManifest { 057 private static final Logger LOG = LoggerFactory.getLogger(BackupManifest.class); 058 059 // manifest file name 060 public static final String MANIFEST_FILE_NAME = ".backup.manifest"; 061 062 /** 063 * Backup image, the dependency graph is made up by series of backup images BackupImage contains 064 * all the relevant information to restore the backup and is used during restore operation 065 */ 066 public static class BackupImage implements Comparable<BackupImage> { 067 static class Builder { 068 BackupImage image; 069 070 Builder() { 071 image = new BackupImage(); 072 } 073 074 Builder withBackupId(String backupId) { 075 image.setBackupId(backupId); 076 return this; 077 } 078 079 Builder withType(BackupType type) { 080 image.setType(type); 081 return this; 082 } 083 084 Builder withRootDir(String rootDir) { 085 image.setRootDir(rootDir); 086 return this; 087 } 088 089 Builder withTableList(List<TableName> tableList) { 090 image.setTableList(tableList); 091 return this; 092 } 093 094 Builder withStartTime(long startTime) { 095 image.setStartTs(startTime); 096 return this; 097 } 098 099 Builder withCompleteTime(long completeTime) { 100 image.setCompleteTs(completeTime); 101 return this; 102 } 103 104 BackupImage build() { 105 return image; 106 } 107 108 } 109 110 private String backupId; 111 private BackupType type; 112 private String rootDir; 113 private List<TableName> tableList; 114 private long startTs; 115 private long completeTs; 116 private ArrayList<BackupImage> ancestors; 117 private Map<TableName, Map<String, Long>> incrTimeRanges; 118 119 static Builder newBuilder() { 120 return new Builder(); 121 } 122 123 public BackupImage() { 124 super(); 125 } 126 127 private BackupImage(String backupId, BackupType type, String rootDir, List<TableName> tableList, 128 long startTs, long completeTs) { 129 this.backupId = backupId; 130 this.type = type; 131 this.rootDir = rootDir; 132 this.tableList = tableList; 133 this.startTs = startTs; 134 this.completeTs = completeTs; 135 } 136 137 static BackupImage fromProto(BackupProtos.BackupImage im) { 138 String backupId = im.getBackupId(); 139 String rootDir = im.getBackupRootDir(); 140 long startTs = im.getStartTs(); 141 long completeTs = im.getCompleteTs(); 142 List<HBaseProtos.TableName> tableListList = im.getTableListList(); 143 List<TableName> tableList = new ArrayList<>(); 144 for (HBaseProtos.TableName tn : tableListList) { 145 tableList.add(ProtobufUtil.toTableName(tn)); 146 } 147 148 List<BackupProtos.BackupImage> ancestorList = im.getAncestorsList(); 149 150 BackupType type = im.getBackupType() == BackupProtos.BackupType.FULL 151 ? BackupType.FULL 152 : BackupType.INCREMENTAL; 153 154 BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs); 155 for (BackupProtos.BackupImage img : ancestorList) { 156 image.addAncestor(fromProto(img)); 157 } 158 image.setIncrTimeRanges(loadIncrementalTimestampMap(im)); 159 return image; 160 } 161 162 /** 163 * This method deliberately does not include the backup root dir on the produced proto. This is 164 * because we don't want to persist the root dir on the backup itself, so that backups can still 165 * be used after they have moved locations. A restore's operator will always provide the root 166 * dir. 167 */ 168 BackupProtos.BackupImage toProto() { 169 BackupProtos.BackupImage.Builder builder = BackupProtos.BackupImage.newBuilder(); 170 builder.setBackupId(backupId); 171 builder.setCompleteTs(completeTs); 172 builder.setStartTs(startTs); 173 if (type == BackupType.FULL) { 174 builder.setBackupType(BackupProtos.BackupType.FULL); 175 } else { 176 builder.setBackupType(BackupProtos.BackupType.INCREMENTAL); 177 } 178 179 for (TableName name : tableList) { 180 builder.addTableList(ProtobufUtil.toProtoTableName(name)); 181 } 182 183 if (ancestors != null) { 184 for (BackupImage im : ancestors) { 185 builder.addAncestors(im.toProto()); 186 } 187 } 188 189 setIncrementalTimestampMap(builder); 190 return builder.build(); 191 } 192 193 private static Map<TableName, Map<String, Long>> 194 loadIncrementalTimestampMap(BackupProtos.BackupImage proto) { 195 List<BackupProtos.TableServerTimestamp> list = proto.getTstMapList(); 196 197 Map<TableName, Map<String, Long>> incrTimeRanges = new HashMap<>(); 198 199 if (list == null || list.size() == 0) { 200 return incrTimeRanges; 201 } 202 203 for (BackupProtos.TableServerTimestamp tst : list) { 204 TableName tn = ProtobufUtil.toTableName(tst.getTableName()); 205 Map<String, Long> map = incrTimeRanges.get(tn); 206 if (map == null) { 207 map = new HashMap<>(); 208 incrTimeRanges.put(tn, map); 209 } 210 List<BackupProtos.ServerTimestamp> listSt = tst.getServerTimestampList(); 211 for (BackupProtos.ServerTimestamp stm : listSt) { 212 ServerName sn = ProtobufUtil.toServerName(stm.getServerName()); 213 map.put(sn.getHostname() + ":" + sn.getPort(), stm.getTimestamp()); 214 } 215 } 216 return incrTimeRanges; 217 } 218 219 private void setIncrementalTimestampMap(BackupProtos.BackupImage.Builder builder) { 220 if (this.incrTimeRanges == null) { 221 return; 222 } 223 for (Entry<TableName, Map<String, Long>> entry : this.incrTimeRanges.entrySet()) { 224 TableName key = entry.getKey(); 225 Map<String, Long> value = entry.getValue(); 226 BackupProtos.TableServerTimestamp.Builder tstBuilder = 227 BackupProtos.TableServerTimestamp.newBuilder(); 228 tstBuilder.setTableName(ProtobufUtil.toProtoTableName(key)); 229 230 for (Map.Entry<String, Long> entry2 : value.entrySet()) { 231 String s = entry2.getKey(); 232 BackupProtos.ServerTimestamp.Builder stBuilder = 233 BackupProtos.ServerTimestamp.newBuilder(); 234 HBaseProtos.ServerName.Builder snBuilder = HBaseProtos.ServerName.newBuilder(); 235 ServerName sn = ServerName.parseServerName(s); 236 snBuilder.setHostName(sn.getHostname()); 237 snBuilder.setPort(sn.getPort()); 238 stBuilder.setServerName(snBuilder.build()); 239 stBuilder.setTimestamp(entry2.getValue()); 240 tstBuilder.addServerTimestamp(stBuilder.build()); 241 } 242 builder.addTstMap(tstBuilder.build()); 243 } 244 } 245 246 public String getBackupId() { 247 return backupId; 248 } 249 250 private void setBackupId(String backupId) { 251 this.backupId = backupId; 252 } 253 254 public BackupType getType() { 255 return type; 256 } 257 258 private void setType(BackupType type) { 259 this.type = type; 260 } 261 262 public String getRootDir() { 263 return rootDir; 264 } 265 266 private void setRootDir(String rootDir) { 267 this.rootDir = rootDir; 268 } 269 270 public List<TableName> getTableNames() { 271 return tableList; 272 } 273 274 private void setTableList(List<TableName> tableList) { 275 this.tableList = tableList; 276 } 277 278 public long getStartTs() { 279 return startTs; 280 } 281 282 private void setStartTs(long startTs) { 283 this.startTs = startTs; 284 } 285 286 public long getCompleteTs() { 287 return completeTs; 288 } 289 290 private void setCompleteTs(long completeTs) { 291 this.completeTs = completeTs; 292 } 293 294 public ArrayList<BackupImage> getAncestors() { 295 if (this.ancestors == null) { 296 this.ancestors = new ArrayList<>(); 297 } 298 return this.ancestors; 299 } 300 301 public void removeAncestors(List<String> backupIds) { 302 List<BackupImage> toRemove = new ArrayList<>(); 303 for (BackupImage im : this.ancestors) { 304 if (backupIds.contains(im.getBackupId())) { 305 toRemove.add(im); 306 } 307 } 308 this.ancestors.removeAll(toRemove); 309 } 310 311 private void addAncestor(BackupImage backupImage) { 312 this.getAncestors().add(backupImage); 313 } 314 315 public boolean hasAncestor(String token) { 316 for (BackupImage image : this.getAncestors()) { 317 if (image.getBackupId().equals(token)) { 318 return true; 319 } 320 } 321 return false; 322 } 323 324 public boolean hasTable(TableName table) { 325 return tableList.contains(table); 326 } 327 328 @Override 329 public int compareTo(BackupImage other) { 330 String thisBackupId = this.getBackupId(); 331 String otherBackupId = other.getBackupId(); 332 int index1 = thisBackupId.lastIndexOf("_"); 333 int index2 = otherBackupId.lastIndexOf("_"); 334 String name1 = thisBackupId.substring(0, index1); 335 String name2 = otherBackupId.substring(0, index2); 336 if (name1.equals(name2)) { 337 Long thisTS = Long.valueOf(thisBackupId.substring(index1 + 1)); 338 Long otherTS = Long.valueOf(otherBackupId.substring(index2 + 1)); 339 return thisTS.compareTo(otherTS); 340 } else { 341 return name1.compareTo(name2); 342 } 343 } 344 345 @Override 346 public boolean equals(Object obj) { 347 if (obj instanceof BackupImage) { 348 return this.compareTo((BackupImage) obj) == 0; 349 } 350 return false; 351 } 352 353 @Override 354 public int hashCode() { 355 int hash = 33 * this.getBackupId().hashCode() + type.hashCode(); 356 hash = 33 * hash + rootDir.hashCode(); 357 hash = 33 * hash + Long.valueOf(startTs).hashCode(); 358 hash = 33 * hash + Long.valueOf(completeTs).hashCode(); 359 for (TableName table : tableList) { 360 hash = 33 * hash + table.hashCode(); 361 } 362 return hash; 363 } 364 365 public Map<TableName, Map<String, Long>> getIncrTimeRanges() { 366 return incrTimeRanges; 367 } 368 369 private void setIncrTimeRanges(Map<TableName, Map<String, Long>> incrTimeRanges) { 370 this.incrTimeRanges = incrTimeRanges; 371 } 372 } 373 374 // backup image directory 375 private BackupImage backupImage; 376 377 /** 378 * Construct manifest for a ongoing backup. 379 * @param backup The ongoing backup info 380 */ 381 public BackupManifest(BackupInfo backup) { 382 BackupImage.Builder builder = BackupImage.newBuilder(); 383 this.backupImage = builder.withBackupId(backup.getBackupId()).withType(backup.getType()) 384 .withRootDir(backup.getBackupRootDir()).withTableList(backup.getTableNames()) 385 .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build(); 386 } 387 388 /** 389 * Construct a table level manifest for a backup of the named table. 390 * @param backup The ongoing backup session info 391 */ 392 public BackupManifest(BackupInfo backup, TableName table) { 393 List<TableName> tables = new ArrayList<TableName>(); 394 tables.add(table); 395 BackupImage.Builder builder = BackupImage.newBuilder(); 396 this.backupImage = builder.withBackupId(backup.getBackupId()).withType(backup.getType()) 397 .withRootDir(backup.getBackupRootDir()).withTableList(tables) 398 .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build(); 399 } 400 401 /** 402 * Construct manifest from a backup directory. 403 * @param conf configuration 404 * @param backupPath backup path 405 * @throws IOException if constructing the manifest from the backup directory fails 406 */ 407 public BackupManifest(Configuration conf, Path backupPath) throws IOException { 408 this(backupPath.getFileSystem(conf), backupPath); 409 } 410 411 /** 412 * Construct manifest from a backup directory. 413 * @param fs the FileSystem 414 * @param backupPath backup path 415 * @throws BackupException exception 416 */ 417 public BackupManifest(FileSystem fs, Path backupPath) throws BackupException { 418 if (LOG.isDebugEnabled()) { 419 LOG.debug("Loading manifest from: " + backupPath.toString()); 420 } 421 // The input backupDir may not exactly be the backup table dir. 422 // It could be the backup log dir where there is also a manifest file stored. 423 // This variable's purpose is to keep the correct and original location so 424 // that we can store/persist it. 425 try { 426 FileStatus[] subFiles = BackupUtils.listStatus(fs, backupPath, null); 427 if (subFiles == null) { 428 String errorMsg = backupPath.toString() + " does not exist"; 429 LOG.error(errorMsg); 430 throw new IOException(errorMsg); 431 } 432 for (FileStatus subFile : subFiles) { 433 if (subFile.getPath().getName().equals(MANIFEST_FILE_NAME)) { 434 // load and set manifest field from file content 435 long len = subFile.getLen(); 436 byte[] pbBytes = new byte[(int) len]; 437 try (FSDataInputStream in = fs.open(subFile.getPath())) { 438 in.readFully(pbBytes); 439 } catch (IOException e) { 440 throw new BackupException(e.getMessage()); 441 } 442 BackupProtos.BackupImage proto = null; 443 try { 444 proto = BackupProtos.BackupImage.parseFrom(pbBytes); 445 } catch (Exception e) { 446 throw new BackupException(e); 447 } 448 this.backupImage = hydrateRootDir(BackupImage.fromProto(proto), backupPath); 449 LOG.debug("Loaded manifest instance from manifest file: " 450 + BackupUtils.getPath(subFile.getPath())); 451 return; 452 } 453 } 454 String errorMsg = "No manifest file found in: " + backupPath.toString(); 455 throw new IOException(errorMsg); 456 } catch (IOException e) { 457 throw new BackupException(e.getMessage()); 458 } 459 } 460 461 /* Visible for testing only */ 462 @RestrictedApi(explanation = "Should only be called internally or in tests", link = "", 463 allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/impl/BackupManifest.java)") 464 public static BackupImage hydrateRootDir(BackupImage backupImage, Path backupPath) 465 throws IOException { 466 String providedRootDir = 467 HBackupFileSystem.getRootDirFromBackupPath(backupPath, backupImage.backupId).toString(); 468 backupImage.setRootDir(providedRootDir); 469 for (BackupImage ancestor : backupImage.getAncestors()) { 470 ancestor.setRootDir(providedRootDir); 471 } 472 return backupImage; 473 } 474 475 public BackupType getType() { 476 return backupImage.getType(); 477 } 478 479 /** 480 * Get the table set of this image. 481 * @return The table set list 482 */ 483 public List<TableName> getTableList() { 484 return backupImage.getTableNames(); 485 } 486 487 /** 488 * Persist the manifest file. 489 * @throws BackupException if an error occurred while storing the manifest file. 490 */ 491 public void store(Configuration conf) throws BackupException { 492 byte[] data = backupImage.toProto().toByteArray(); 493 // write the file, overwrite if already exist 494 Path manifestFilePath = 495 new Path(HBackupFileSystem.getBackupPath(backupImage.getRootDir(), backupImage.getBackupId()), 496 MANIFEST_FILE_NAME); 497 try (FSDataOutputStream out = 498 manifestFilePath.getFileSystem(conf).create(manifestFilePath, true)) { 499 out.write(data); 500 } catch (IOException e) { 501 throw new BackupException(e.getMessage()); 502 } 503 504 LOG.info("Manifest file stored to " + manifestFilePath); 505 } 506 507 /** 508 * Get this backup image. 509 * @return the backup image. 510 */ 511 public BackupImage getBackupImage() { 512 return backupImage; 513 } 514 515 /** 516 * Add dependent backup image for this backup. 517 * @param image The direct dependent backup image 518 */ 519 public void addDependentImage(BackupImage image) { 520 this.backupImage.addAncestor(image); 521 } 522 523 /** 524 * Set the incremental timestamp map directly. 525 * @param incrTimestampMap timestamp map 526 */ 527 public void setIncrTimestampMap(Map<TableName, Map<String, Long>> incrTimestampMap) { 528 this.backupImage.setIncrTimeRanges(incrTimestampMap); 529 } 530 531 public Map<TableName, Map<String, Long>> getIncrTimestampMap() { 532 return backupImage.getIncrTimeRanges(); 533 } 534 535 /** 536 * Get the image list of this backup for restore in time order. 537 * @param reverse If true, then output in reverse order, otherwise in time order from old to new 538 * @return the backup image list for restore in time order 539 */ 540 public ArrayList<BackupImage> getRestoreDependentList(boolean reverse) { 541 TreeMap<Long, BackupImage> restoreImages = new TreeMap<>(); 542 restoreImages.put(backupImage.startTs, backupImage); 543 for (BackupImage image : backupImage.getAncestors()) { 544 restoreImages.put(Long.valueOf(image.startTs), image); 545 } 546 return new ArrayList<>( 547 reverse ? restoreImages.descendingMap().values() : restoreImages.values()); 548 } 549 550 /** 551 * Get the dependent image list for a specific table of this backup in time order from old to new 552 * if want to restore to this backup image level. 553 * @param table table 554 * @return the backup image list for a table in time order 555 */ 556 public ArrayList<BackupImage> getDependentListByTable(TableName table) { 557 ArrayList<BackupImage> tableImageList = new ArrayList<>(); 558 ArrayList<BackupImage> imageList = getRestoreDependentList(true); 559 for (BackupImage image : imageList) { 560 if (image.hasTable(table)) { 561 tableImageList.add(image); 562 if (image.getType() == BackupType.FULL) { 563 break; 564 } 565 } 566 } 567 Collections.reverse(tableImageList); 568 return tableImageList; 569 } 570 571 public BackupInfo toBackupInfo() { 572 BackupInfo info = new BackupInfo(); 573 info.setType(backupImage.getType()); 574 List<TableName> list = backupImage.getTableNames(); 575 TableName[] tables = new TableName[list.size()]; 576 info.addTables(list.toArray(tables)); 577 info.setBackupId(backupImage.getBackupId()); 578 info.setStartTs(backupImage.getStartTs()); 579 info.setBackupRootDir(backupImage.getRootDir()); 580 if (backupImage.getType() == BackupType.INCREMENTAL) { 581 info.setHLogTargetDir( 582 BackupUtils.getLogBackupDir(backupImage.getRootDir(), backupImage.getBackupId())); 583 } 584 return info; 585 } 586}