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 java.io.IOException; 021import java.util.ArrayList; 022import java.util.HashMap; 023import java.util.List; 024import java.util.Map; 025import org.apache.hadoop.conf.Configuration; 026import org.apache.hadoop.fs.FileStatus; 027import org.apache.hadoop.fs.FileSystem; 028import org.apache.hadoop.fs.Path; 029import org.apache.hadoop.hbase.TableName; 030import org.apache.hadoop.hbase.backup.BackupInfo; 031import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase; 032import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; 033import org.apache.hadoop.hbase.backup.BackupRequest; 034import org.apache.hadoop.hbase.backup.BackupRestoreConstants; 035import org.apache.hadoop.hbase.backup.BackupType; 036import org.apache.hadoop.hbase.backup.HBackupFileSystem; 037import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; 038import org.apache.hadoop.hbase.client.Admin; 039import org.apache.hadoop.hbase.client.Connection; 040import org.apache.hadoop.hbase.util.CommonFSUtils; 041import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 042import org.apache.yetus.audience.InterfaceAudience; 043import org.slf4j.Logger; 044import org.slf4j.LoggerFactory; 045 046/** 047 * Base class for backup operation. Concrete implementation for full and incremental backup are 048 * delegated to corresponding sub-classes: {@link FullTableBackupClient} and 049 * {@link IncrementalTableBackupClient} 050 */ 051@InterfaceAudience.Private 052public abstract class TableBackupClient { 053 054 public static final String BACKUP_CLIENT_IMPL_CLASS = "backup.client.impl.class"; 055 056 public static final String BACKUP_TEST_MODE_STAGE = "backup.test.mode.stage"; 057 058 private static final Logger LOG = LoggerFactory.getLogger(TableBackupClient.class); 059 060 protected Configuration conf; 061 protected Connection conn; 062 protected String backupId; 063 protected List<TableName> tableList; 064 protected Map<String, Long> newTimestamps = null; 065 066 protected BackupManager backupManager; 067 protected BackupInfo backupInfo; 068 protected FileSystem fs; 069 070 public TableBackupClient() { 071 } 072 073 public TableBackupClient(final Connection conn, final String backupId, BackupRequest request) 074 throws IOException { 075 init(conn, backupId, request); 076 } 077 078 public void init(final Connection conn, final String backupId, BackupRequest request) 079 throws IOException { 080 if (request.getBackupType() == BackupType.FULL) { 081 backupManager = new BackupManager(conn, conn.getConfiguration()); 082 } else { 083 backupManager = new IncrementalBackupManager(conn, conn.getConfiguration()); 084 } 085 this.backupId = backupId; 086 this.tableList = request.getTableList(); 087 this.conn = conn; 088 this.conf = conn.getConfiguration(); 089 this.fs = CommonFSUtils.getCurrentFileSystem(conf); 090 backupInfo = backupManager.createBackupInfo(backupId, request.getBackupType(), tableList, 091 request.getTargetRootDir(), request.getTotalTasks(), request.getBandwidth()); 092 if (tableList == null || tableList.isEmpty()) { 093 this.tableList = new ArrayList<>(backupInfo.getTables()); 094 } 095 // Start new session 096 backupManager.startBackupSession(); 097 } 098 099 /** 100 * Begin the overall backup. 101 * @param backupInfo backup info 102 * @throws IOException exception 103 */ 104 protected void beginBackup(BackupManager backupManager, BackupInfo backupInfo) 105 throws IOException { 106 107 BackupSystemTable.snapshot(conn); 108 backupManager.setBackupInfo(backupInfo); 109 // set the start timestamp of the overall backup 110 long startTs = EnvironmentEdgeManager.currentTime(); 111 backupInfo.setStartTs(startTs); 112 // set overall backup status: ongoing 113 backupInfo.setState(BackupState.RUNNING); 114 backupInfo.setPhase(BackupPhase.REQUEST); 115 LOG.info("Backup " + backupInfo.getBackupId() + " started at " + startTs + "."); 116 117 backupManager.updateBackupInfo(backupInfo); 118 if (LOG.isDebugEnabled()) { 119 LOG.debug("Backup session " + backupInfo.getBackupId() + " has been started."); 120 } 121 } 122 123 protected String getMessage(Exception e) { 124 String msg = e.getMessage(); 125 if (msg == null || msg.equals("")) { 126 msg = e.getClass().getName(); 127 } 128 return msg; 129 } 130 131 /** 132 * Delete HBase snapshot for backup. 133 * @param backupInfo backup info 134 * @throws IOException exception 135 */ 136 protected static void deleteSnapshots(final Connection conn, BackupInfo backupInfo, 137 Configuration conf) throws IOException { 138 LOG.debug("Trying to delete snapshot for full backup."); 139 for (String snapshotName : backupInfo.getSnapshotNames()) { 140 if (snapshotName == null) { 141 continue; 142 } 143 LOG.debug("Trying to delete snapshot: " + snapshotName); 144 145 try (Admin admin = conn.getAdmin()) { 146 admin.deleteSnapshot(snapshotName); 147 } 148 LOG.debug("Deleting the snapshot " + snapshotName + " for backup " + backupInfo.getBackupId() 149 + " succeeded."); 150 } 151 } 152 153 /** 154 * Clean up directories with prefix "exportSnapshot-", which are generated when exporting 155 * snapshots. 156 * @throws IOException exception 157 */ 158 protected static void cleanupExportSnapshotLog(Configuration conf) throws IOException { 159 FileSystem fs = CommonFSUtils.getCurrentFileSystem(conf); 160 Path stagingDir = new Path( 161 conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory().toString())); 162 FileStatus[] files = CommonFSUtils.listStatus(fs, stagingDir); 163 if (files == null) { 164 return; 165 } 166 for (FileStatus file : files) { 167 if (file.getPath().getName().startsWith("exportSnapshot-")) { 168 LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName()); 169 if (CommonFSUtils.delete(fs, file.getPath(), true) == false) { 170 LOG.warn("Can not delete " + file.getPath()); 171 } 172 } 173 } 174 } 175 176 /** 177 * Clean up the uncompleted data at target directory if the ongoing backup has already entered the 178 * copy phase. 179 */ 180 protected static void cleanupTargetDir(BackupInfo backupInfo, Configuration conf) { 181 try { 182 // clean up the uncompleted data at target directory if the ongoing backup has already entered 183 // the copy phase 184 LOG.debug("Trying to cleanup up target dir. Current backup phase: " + backupInfo.getPhase()); 185 if ( 186 backupInfo.getPhase().equals(BackupPhase.SNAPSHOTCOPY) 187 || backupInfo.getPhase().equals(BackupPhase.INCREMENTAL_COPY) 188 || backupInfo.getPhase().equals(BackupPhase.STORE_MANIFEST) 189 ) { 190 FileSystem outputFs = FileSystem.get(new Path(backupInfo.getBackupRootDir()).toUri(), conf); 191 192 // now treat one backup as a transaction, clean up data that has been partially copied at 193 // table level 194 for (TableName table : backupInfo.getTables()) { 195 Path targetDirPath = new Path(HBackupFileSystem 196 .getTableBackupDir(backupInfo.getBackupRootDir(), backupInfo.getBackupId(), table)); 197 if (outputFs.delete(targetDirPath, true)) { 198 LOG.debug( 199 "Cleaning up uncompleted backup data at " + targetDirPath.toString() + " done."); 200 } else { 201 LOG.debug("No data has been copied to " + targetDirPath.toString() + "."); 202 } 203 204 Path tableDir = targetDirPath.getParent(); 205 FileStatus[] backups = CommonFSUtils.listStatus(outputFs, tableDir); 206 if (backups == null || backups.length == 0) { 207 outputFs.delete(tableDir, true); 208 LOG.debug(tableDir.toString() + " is empty, remove it."); 209 } 210 } 211 } 212 213 } catch (IOException e1) { 214 LOG.error("Cleaning up uncompleted backup data of " + backupInfo.getBackupId() + " at " 215 + backupInfo.getBackupRootDir() + " failed due to " + e1.getMessage() + "."); 216 } 217 } 218 219 /** 220 * Fail the overall backup. 221 * @param backupInfo backup info 222 * @param e exception 223 * @throws IOException exception 224 */ 225 protected void failBackup(Connection conn, BackupInfo backupInfo, BackupManager backupManager, 226 Exception e, String msg, BackupType type, Configuration conf) throws IOException { 227 try { 228 LOG.error(msg + getMessage(e), e); 229 // If this is a cancel exception, then we've already cleaned. 230 // set the failure timestamp of the overall backup 231 backupInfo.setCompleteTs(EnvironmentEdgeManager.currentTime()); 232 // set failure message 233 backupInfo.setFailedMsg(e.getMessage()); 234 // set overall backup status: failed 235 backupInfo.setState(BackupState.FAILED); 236 // compose the backup failed data 237 String backupFailedData = "BackupId=" + backupInfo.getBackupId() + ",startts=" 238 + backupInfo.getStartTs() + ",failedts=" + backupInfo.getCompleteTs() + ",failedphase=" 239 + backupInfo.getPhase() + ",failedmessage=" + backupInfo.getFailedMsg(); 240 LOG.error(backupFailedData); 241 cleanupAndRestoreBackupSystem(conn, backupInfo, conf); 242 // If backup session is updated to FAILED state - means we 243 // processed recovery already. 244 backupManager.updateBackupInfo(backupInfo); 245 backupManager.finishBackupSession(); 246 LOG.error("Backup " + backupInfo.getBackupId() + " failed."); 247 } catch (IOException ee) { 248 LOG.error("Please run backup repair tool manually to restore backup system integrity"); 249 throw ee; 250 } 251 } 252 253 public static void cleanupAndRestoreBackupSystem(Connection conn, BackupInfo backupInfo, 254 Configuration conf) throws IOException { 255 BackupType type = backupInfo.getType(); 256 // if full backup, then delete HBase snapshots if there already are snapshots taken 257 // and also clean up export snapshot log files if exist 258 if (type == BackupType.FULL) { 259 deleteSnapshots(conn, backupInfo, conf); 260 cleanupExportSnapshotLog(conf); 261 } 262 BackupSystemTable.restoreFromSnapshot(conn); 263 BackupSystemTable.deleteSnapshot(conn); 264 // clean up the uncompleted data at target directory if the ongoing backup has already entered 265 // the copy phase 266 // For incremental backup, DistCp logs will be cleaned with the targetDir. 267 cleanupTargetDir(backupInfo, conf); 268 } 269 270 /** 271 * Add manifest for the current backup. The manifest is stored within the table backup directory. 272 * @param backupInfo The current backup info 273 * @throws IOException exception 274 */ 275 protected void addManifest(BackupInfo backupInfo, BackupManager backupManager, BackupType type, 276 Configuration conf) throws IOException { 277 // set the overall backup phase : store manifest 278 backupInfo.setPhase(BackupPhase.STORE_MANIFEST); 279 280 BackupManifest manifest; 281 282 // Since we have each table's backup in its own directory structure, 283 // we'll store its manifest with the table directory. 284 for (TableName table : backupInfo.getTables()) { 285 manifest = new BackupManifest(backupInfo, table); 286 ArrayList<BackupImage> ancestors = backupManager.getAncestors(backupInfo, table); 287 for (BackupImage image : ancestors) { 288 manifest.addDependentImage(image); 289 } 290 291 if (type == BackupType.INCREMENTAL) { 292 // We'll store the log timestamps for this table only in its manifest. 293 Map<TableName, Map<String, Long>> tableTimestampMap = new HashMap<>(); 294 tableTimestampMap.put(table, backupInfo.getIncrTimestampMap().get(table)); 295 manifest.setIncrTimestampMap(tableTimestampMap); 296 ArrayList<BackupImage> ancestorss = backupManager.getAncestors(backupInfo); 297 for (BackupImage image : ancestorss) { 298 manifest.addDependentImage(image); 299 } 300 } 301 manifest.store(conf); 302 } 303 304 // For incremental backup, we store a overall manifest in 305 // <backup-root-dir>/WALs/<backup-id> 306 // This is used when created the next incremental backup 307 if (type == BackupType.INCREMENTAL) { 308 manifest = new BackupManifest(backupInfo); 309 // set the table region server start and end timestamps for incremental backup 310 manifest.setIncrTimestampMap(backupInfo.getIncrTimestampMap()); 311 ArrayList<BackupImage> ancestors = backupManager.getAncestors(backupInfo); 312 for (BackupImage image : ancestors) { 313 manifest.addDependentImage(image); 314 } 315 manifest.store(conf); 316 } 317 } 318 319 /** 320 * Get backup request meta data dir as string. 321 * @param backupInfo backup info 322 * @return meta data dir 323 */ 324 protected String obtainBackupMetaDataStr(BackupInfo backupInfo) { 325 StringBuilder sb = new StringBuilder(); 326 sb.append("type=" + backupInfo.getType() + ",tablelist="); 327 for (TableName table : backupInfo.getTables()) { 328 sb.append(table + ";"); 329 } 330 if (sb.lastIndexOf(";") > 0) { 331 sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1); 332 } 333 sb.append(",targetRootDir=" + backupInfo.getBackupRootDir()); 334 335 return sb.toString(); 336 } 337 338 /** 339 * Complete the overall backup. 340 * @param backupInfo backup info 341 * @throws IOException exception 342 */ 343 protected void completeBackup(final Connection conn, BackupInfo backupInfo, 344 BackupManager backupManager, BackupType type, Configuration conf) throws IOException { 345 // set the complete timestamp of the overall backup 346 backupInfo.setCompleteTs(EnvironmentEdgeManager.currentTime()); 347 // set overall backup status: complete 348 backupInfo.setState(BackupState.COMPLETE); 349 backupInfo.setProgress(100); 350 // add and store the manifest for the backup 351 addManifest(backupInfo, backupManager, type, conf); 352 353 // compose the backup complete data 354 String backupCompleteData = 355 obtainBackupMetaDataStr(backupInfo) + ",startts=" + backupInfo.getStartTs() + ",completets=" 356 + backupInfo.getCompleteTs() + ",bytescopied=" + backupInfo.getTotalBytesCopied(); 357 if (LOG.isDebugEnabled()) { 358 LOG.debug("Backup " + backupInfo.getBackupId() + " finished: " + backupCompleteData); 359 } 360 361 // when full backup is done: 362 // - delete HBase snapshot 363 // - clean up directories with prefix "exportSnapshot-", which are generated when exporting 364 // snapshots 365 // incremental backups use distcp, which handles cleaning up its own directories 366 if (type == BackupType.FULL) { 367 deleteSnapshots(conn, backupInfo, conf); 368 cleanupExportSnapshotLog(conf); 369 } 370 BackupSystemTable.deleteSnapshot(conn); 371 backupManager.updateBackupInfo(backupInfo); 372 373 // Finish active session 374 backupManager.finishBackupSession(); 375 376 LOG.info("Backup " + backupInfo.getBackupId() + " completed."); 377 } 378 379 /** 380 * Backup request execution. 381 * @throws IOException if the execution of the backup fails 382 */ 383 public abstract void execute() throws IOException; 384 385 protected Stage getTestStage() { 386 return Stage.valueOf("stage_" + conf.getInt(BACKUP_TEST_MODE_STAGE, 0)); 387 } 388 389 protected void failStageIf(Stage stage) throws IOException { 390 Stage current = getTestStage(); 391 if (current == stage) { 392 throw new IOException("Failed stage " + stage + " in testing"); 393 } 394 } 395 396 public enum Stage { 397 stage_0, 398 stage_1, 399 stage_2, 400 stage_3, 401 stage_4 402 } 403}