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