Skip to content

Commit 2ecf6b8

Browse files
hgromerHernan Gelaf-Romer
authored andcommitted
HBASE-27659 Incremental backups should re-use splits from last full backup (apache#6370)
1 parent d500466 commit 2ecf6b8

14 files changed

Lines changed: 713 additions & 64 deletions

File tree

hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyJob;
2222
import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
2323
import org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreJob;
24+
import org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreToOriginalSplitsJob;
2425
import org.apache.hadoop.util.ReflectionUtils;
2526
import org.apache.yetus.audience.InterfaceAudience;
2627

@@ -43,8 +44,13 @@ private BackupRestoreFactory() {
4344
* @return backup restore job instance
4445
*/
4546
public static RestoreJob getRestoreJob(Configuration conf) {
47+
Class<? extends RestoreJob> defaultCls =
48+
conf.getBoolean(RestoreJob.KEEP_ORIGINAL_SPLITS_KEY, RestoreJob.KEEP_ORIGINAL_SPLITS_DEFAULT)
49+
? MapReduceRestoreToOriginalSplitsJob.class
50+
: MapReduceRestoreJob.class;
51+
4652
Class<? extends RestoreJob> cls =
47-
conf.getClass(HBASE_INCR_RESTORE_IMPL_CLASS, MapReduceRestoreJob.class, RestoreJob.class);
53+
conf.getClass(HBASE_INCR_RESTORE_IMPL_CLASS, defaultCls, RestoreJob.class);
4854
RestoreJob service = ReflectionUtils.newInstance(cls, conf);
4955
service.setConf(conf);
5056
return service;

hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,8 @@
2727
import org.slf4j.Logger;
2828
import org.slf4j.LoggerFactory;
2929

30+
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
31+
3032
/**
3133
* View to an on-disk Backup Image FileSytem Provides the set of methods necessary to interact with
3234
* the on-disk Backup Image data.
@@ -115,4 +117,13 @@ public static BackupManifest getManifest(Configuration conf, Path backupRootPath
115117
new BackupManifest(conf, getManifestPath(conf, backupRootPath, backupId));
116118
return manifest;
117119
}
120+
121+
public static Path getRootDirFromBackupPath(Path backupPath, String backupId) {
122+
if (backupPath.getName().equals(BackupManifest.MANIFEST_FILE_NAME)) {
123+
backupPath = backupPath.getParent();
124+
}
125+
Preconditions.checkArgument(backupPath.getName().equals(backupId),
126+
String.format("Backup path %s must end in backupId %s", backupPath, backupId));
127+
return backupPath.getParent();
128+
}
118129
}

hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,10 @@
3030

3131
@InterfaceAudience.Private
3232
public interface RestoreJob extends Configurable {
33+
34+
String KEEP_ORIGINAL_SPLITS_KEY = "hbase.backup.restorejob.keep.original.splits";
35+
boolean KEEP_ORIGINAL_SPLITS_DEFAULT = false;
36+
3337
/**
3438
* Run restore operation
3539
* @param dirPaths path array of WAL log directories

hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -67,6 +67,11 @@ public Builder withOvewrite(boolean overwrite) {
6767
return this;
6868
}
6969

70+
public Builder withKeepOriginalSplits(boolean keepOriginalSplits) {
71+
request.setKeepOriginalSplits(keepOriginalSplits);
72+
return this;
73+
}
74+
7075
public RestoreRequest build() {
7176
return request;
7277
}
@@ -80,6 +85,8 @@ public RestoreRequest build() {
8085
private TableName[] toTables;
8186
private boolean overwrite = false;
8287

88+
private boolean keepOriginalSplits = false;
89+
8390
private RestoreRequest() {
8491
}
8592

@@ -145,4 +152,13 @@ private RestoreRequest setOverwrite(boolean overwrite) {
145152
this.overwrite = overwrite;
146153
return this;
147154
}
155+
156+
public boolean isKeepOriginalSplits() {
157+
return keepOriginalSplits;
158+
}
159+
160+
private RestoreRequest setKeepOriginalSplits(boolean keepOriginalSplits) {
161+
this.keepOriginalSplits = keepOriginalSplits;
162+
return this;
163+
}
148164
}

hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@
4444
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
4545
import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
4646
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
47+
import com.google.errorprone.annotations.RestrictedApi;
4748

4849
/**
4950
* Backup manifest contains all the meta data of a backup image. The manifest info will be bundled
@@ -563,4 +564,18 @@ public BackupInfo toBackupInfo() {
563564
}
564565
return info;
565566
}
567+
568+
/* Visible for testing only */
569+
@RestrictedApi(explanation = "Should only be called internally or in tests", link = "",
570+
allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/impl/BackupManifest.java)")
571+
public static BackupImage hydrateRootDir(BackupImage backupImage, Path backupPath)
572+
throws IOException {
573+
String providedRootDir =
574+
HBackupFileSystem.getRootDirFromBackupPath(backupPath, backupImage.backupId).toString();
575+
backupImage.setRootDir(providedRootDir);
576+
for (BackupImage ancestor : backupImage.getAncestors()) {
577+
ancestor.setRootDir(providedRootDir);
578+
}
579+
return backupImage;
580+
}
566581
}

hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java

Lines changed: 101 additions & 42 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,9 @@
3030
import org.apache.commons.io.FilenameUtils;
3131
import org.apache.commons.lang3.StringUtils;
3232
import org.apache.hadoop.fs.FileSystem;
33+
import org.apache.hadoop.fs.LocatedFileStatus;
3334
import org.apache.hadoop.fs.Path;
35+
import org.apache.hadoop.fs.RemoteIterator;
3436
import org.apache.hadoop.hbase.TableName;
3537
import org.apache.hadoop.hbase.backup.BackupCopyJob;
3638
import org.apache.hadoop.hbase.backup.BackupInfo;
@@ -40,14 +42,17 @@
4042
import org.apache.hadoop.hbase.backup.BackupType;
4143
import org.apache.hadoop.hbase.backup.HBackupFileSystem;
4244
import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyJob;
45+
import org.apache.hadoop.hbase.backup.mapreduce.MapReduceHFileSplitterJob;
4346
import org.apache.hadoop.hbase.backup.util.BackupUtils;
4447
import org.apache.hadoop.hbase.client.Admin;
4548
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
4649
import org.apache.hadoop.hbase.client.Connection;
4750
import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
51+
import org.apache.hadoop.hbase.io.hfile.HFile;
4852
import org.apache.hadoop.hbase.mapreduce.WALPlayer;
4953
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
5054
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
55+
import org.apache.hadoop.hbase.snapshot.SnapshotRegionLocator;
5156
import org.apache.hadoop.hbase.util.CommonFSUtils;
5257
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
5358
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
@@ -166,54 +171,60 @@ protected List<BulkLoad> handleBulkLoad(List<TableName> tablesToBackup) throws I
166171
LOG.debug("copying archive {} to {}", archive, tgt);
167172
archiveFiles.add(archive.toString());
168173
}
174+
mergeSplitBulkloads(activeFiles, archiveFiles, srcTable);
175+
incrementalCopyBulkloadHFiles(tgtFs, srcTable);
169176
}
170-
171-
copyBulkLoadedFiles(activeFiles, archiveFiles);
172177
return bulkLoads;
173178
}
174179

175-
private void copyBulkLoadedFiles(List<String> activeFiles, List<String> archiveFiles)
176-
throws IOException {
177-
try {
178-
// Enable special mode of BackupDistCp
179-
conf.setInt(MapReduceBackupCopyJob.NUMBER_OF_LEVELS_TO_PRESERVE_KEY, 5);
180-
// Copy active files
181-
String tgtDest = backupInfo.getBackupRootDir() + Path.SEPARATOR + backupInfo.getBackupId();
182-
int attempt = 1;
183-
while (activeFiles.size() > 0) {
184-
LOG.info("Copy " + activeFiles.size() + " active bulk loaded files. Attempt =" + attempt++);
185-
String[] toCopy = new String[activeFiles.size()];
186-
activeFiles.toArray(toCopy);
187-
// Active file can be archived during copy operation,
188-
// we need to handle this properly
189-
try {
190-
incrementalCopyHFiles(toCopy, tgtDest);
191-
break;
192-
} catch (IOException e) {
193-
// Check if some files got archived
194-
// Update active and archived lists
195-
// When file is being moved from active to archive
196-
// directory, the number of active files decreases
197-
int numOfActive = activeFiles.size();
198-
updateFileLists(activeFiles, archiveFiles);
199-
if (activeFiles.size() < numOfActive) {
200-
continue;
201-
}
202-
// if not - throw exception
203-
throw e;
180+
private void mergeSplitBulkloads(List<String> activeFiles, List<String> archiveFiles,
181+
TableName tn) throws IOException {
182+
int attempt = 1;
183+
184+
while (!activeFiles.isEmpty()) {
185+
LOG.info("MergeSplit {} active bulk loaded files. Attempt={}", activeFiles.size(), attempt++);
186+
// Active file can be archived during copy operation,
187+
// we need to handle this properly
188+
try {
189+
mergeSplitBulkloads(activeFiles, tn);
190+
break;
191+
} catch (IOException e) {
192+
int numActiveFiles = activeFiles.size();
193+
updateFileLists(activeFiles, archiveFiles);
194+
if (activeFiles.size() < numActiveFiles) {
195+
continue;
204196
}
197+
198+
throw e;
205199
}
206-
// If incremental copy will fail for archived files
207-
// we will have partially loaded files in backup destination (only files from active data
208-
// directory). It is OK, because the backup will marked as FAILED and data will be cleaned up
209-
if (archiveFiles.size() > 0) {
210-
String[] toCopy = new String[archiveFiles.size()];
211-
archiveFiles.toArray(toCopy);
212-
incrementalCopyHFiles(toCopy, tgtDest);
213-
}
214-
} finally {
215-
// Disable special mode of BackupDistCp
216-
conf.unset(MapReduceBackupCopyJob.NUMBER_OF_LEVELS_TO_PRESERVE_KEY);
200+
}
201+
202+
if (!archiveFiles.isEmpty()) {
203+
mergeSplitBulkloads(archiveFiles, tn);
204+
}
205+
}
206+
207+
private void mergeSplitBulkloads(List<String> files, TableName tn) throws IOException {
208+
MapReduceHFileSplitterJob player = new MapReduceHFileSplitterJob();
209+
conf.set(MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY,
210+
getBulkOutputDirForTable(tn).toString());
211+
player.setConf(conf);
212+
213+
String inputDirs = StringUtils.join(files, ",");
214+
String[] args = { inputDirs, tn.getNameWithNamespaceInclAsString() };
215+
216+
int result;
217+
218+
try {
219+
result = player.run(args);
220+
} catch (Exception e) {
221+
LOG.error("Failed to run MapReduceHFileSplitterJob", e);
222+
throw new IOException(e);
223+
}
224+
225+
if (result != 0) {
226+
throw new IOException(
227+
"Failed to run MapReduceHFileSplitterJob with invalid result: " + result);
217228
}
218229
}
219230

@@ -264,6 +275,7 @@ public void execute() throws IOException, ColumnFamilyMismatchException {
264275
try {
265276
// copy out the table and region info files for each table
266277
BackupUtils.copyTableRegionInfo(conn, backupInfo, conf);
278+
setupRegionLocator();
267279
// convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT
268280
convertWALsToHFiles();
269281
incrementalCopyHFiles(new String[] { getBulkOutputDir().toString() },
@@ -407,6 +419,29 @@ protected void walToHFiles(List<String> dirPaths, List<String> tableList) throws
407419
}
408420
}
409421

422+
private void incrementalCopyBulkloadHFiles(FileSystem tgtFs, TableName tn) throws IOException {
423+
Path bulkOutDir = getBulkOutputDirForTable(tn);
424+
FileSystem fs = FileSystem.get(conf);
425+
426+
if (fs.exists(bulkOutDir)) {
427+
conf.setInt(MapReduceBackupCopyJob.NUMBER_OF_LEVELS_TO_PRESERVE_KEY, 2);
428+
Path tgtPath = getTargetDirForTable(tn);
429+
try {
430+
RemoteIterator<LocatedFileStatus> locatedFiles = tgtFs.listFiles(bulkOutDir, true);
431+
List<String> files = new ArrayList<>();
432+
while (locatedFiles.hasNext()) {
433+
LocatedFileStatus file = locatedFiles.next();
434+
if (file.isFile() && HFile.isHFileFormat(tgtFs, file.getPath())) {
435+
files.add(file.getPath().toString());
436+
}
437+
}
438+
incrementalCopyHFiles(files.toArray(files.toArray(new String[0])), tgtPath.toString());
439+
} finally {
440+
conf.unset(MapReduceBackupCopyJob.NUMBER_OF_LEVELS_TO_PRESERVE_KEY);
441+
}
442+
}
443+
}
444+
410445
protected Path getBulkOutputDirForTable(TableName table) {
411446
Path tablePath = getBulkOutputDir();
412447
tablePath = new Path(tablePath, table.getNamespaceAsString());
@@ -422,6 +457,30 @@ protected Path getBulkOutputDir() {
422457
return path;
423458
}
424459

460+
private Path getTargetDirForTable(TableName table) {
461+
Path path = new Path(backupInfo.getBackupRootDir() + Path.SEPARATOR + backupInfo.getBackupId());
462+
path = new Path(path, table.getNamespaceAsString());
463+
path = new Path(path, table.getNameAsString());
464+
return path;
465+
}
466+
467+
private void setupRegionLocator() throws IOException {
468+
Map<TableName, String> fullBackupIds = getFullBackupIds();
469+
try (BackupAdminImpl backupAdmin = new BackupAdminImpl(conn)) {
470+
471+
for (TableName tableName : backupInfo.getTables()) {
472+
String fullBackupId = fullBackupIds.get(tableName);
473+
BackupInfo fullBackupInfo = backupAdmin.getBackupInfo(fullBackupId);
474+
String snapshotName = fullBackupInfo.getSnapshotName(tableName);
475+
Path root = HBackupFileSystem.getTableBackupPath(tableName,
476+
new Path(fullBackupInfo.getBackupRootDir()), fullBackupId);
477+
String manifestDir =
478+
SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, root).toString();
479+
SnapshotRegionLocator.setSnapshotManifestDir(conf, manifestDir, tableName);
480+
}
481+
}
482+
}
483+
425484
private Map<TableName, String> getFullBackupIds() throws IOException {
426485
// Ancestors are stored from newest to oldest, so we can iterate backwards
427486
// in order to populate our backupId map with the most recent full backup

hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java

Lines changed: 9 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -59,6 +59,8 @@ public class RestoreTablesClient {
5959
private Path restoreRootDir;
6060
private boolean isOverwrite;
6161

62+
private boolean isKeepOriginalSplits;
63+
6264
public RestoreTablesClient(Connection conn, RestoreRequest request) throws IOException {
6365
this.backupRootDir = request.getBackupRootDir();
6466
this.backupId = request.getBackupId();
@@ -68,6 +70,7 @@ public RestoreTablesClient(Connection conn, RestoreRequest request) throws IOExc
6870
this.tTableArray = sTableArray;
6971
}
7072
this.isOverwrite = request.isOverwrite();
73+
this.isKeepOriginalSplits = request.isKeepOriginalSplits();
7174
this.conn = conn;
7275
this.conf = conn.getConfiguration();
7376
if (request.getRestoreRootDir() != null) {
@@ -132,7 +135,7 @@ private void checkTargetTables(TableName[] tTableArray, boolean isOverwrite) thr
132135
*/
133136

134137
private void restoreImages(BackupImage[] images, TableName sTable, TableName tTable,
135-
boolean truncateIfExists) throws IOException {
138+
boolean truncateIfExists, boolean isKeepOriginalSplits) throws IOException {
136139
// First image MUST be image of a FULL backup
137140
BackupImage image = images[0];
138141
String rootDir = image.getRootDir();
@@ -148,7 +151,7 @@ private void restoreImages(BackupImage[] images, TableName sTable, TableName tTa
148151
+ tableBackupPath.toString());
149152
conf.set(JOB_NAME_CONF_KEY, "Full_Restore-" + backupId + "-" + tTable);
150153
restoreTool.fullRestoreTable(conn, tableBackupPath, sTable, tTable, truncateIfExists,
151-
lastIncrBackupId);
154+
isKeepOriginalSplits, lastIncrBackupId);
152155
conf.unset(JOB_NAME_CONF_KEY);
153156
} else { // incremental Backup
154157
throw new IOException("Unexpected backup type " + image.getType());
@@ -182,7 +185,7 @@ private void restoreImages(BackupImage[] images, TableName sTable, TableName tTa
182185
dirList.toArray(paths);
183186
conf.set(JOB_NAME_CONF_KEY, "Incremental_Restore-" + backupId + "-" + tTable);
184187
restoreTool.incrementalRestoreTable(conn, tableBackupPath, paths, new TableName[] { sTable },
185-
new TableName[] { tTable }, lastIncrBackupId);
188+
new TableName[] { tTable }, lastIncrBackupId, isKeepOriginalSplits);
186189
LOG.info(sTable + " has been successfully restored to " + tTable);
187190
}
188191

@@ -207,7 +210,7 @@ private List<Path> getFilesRecursively(String fileBackupDir)
207210
* @throws IOException exception
208211
*/
209212
private void restore(BackupManifest manifest, TableName[] sTableArray, TableName[] tTableArray,
210-
boolean isOverwrite) throws IOException {
213+
boolean isOverwrite, boolean isKeepOriginalSplits) throws IOException {
211214
TreeSet<BackupImage> restoreImageSet = new TreeSet<>();
212215

213216
for (int i = 0; i < sTableArray.length; i++) {
@@ -222,7 +225,7 @@ private void restore(BackupManifest manifest, TableName[] sTableArray, TableName
222225
set.addAll(depList);
223226
BackupImage[] arr = new BackupImage[set.size()];
224227
set.toArray(arr);
225-
restoreImages(arr, table, tTableArray[i], isOverwrite);
228+
restoreImages(arr, table, tTableArray[i], isOverwrite, isKeepOriginalSplits);
226229
restoreImageSet.addAll(list);
227230
if (restoreImageSet != null && !restoreImageSet.isEmpty()) {
228231
LOG.info("Restore includes the following image(s):");
@@ -256,6 +259,6 @@ public void execute() throws IOException {
256259
Path rootPath = new Path(backupRootDir);
257260
BackupManifest manifest = HBackupFileSystem.getManifest(conf, rootPath, backupId);
258261

259-
restore(manifest, sTableArray, tTableArray, isOverwrite);
262+
restore(manifest, sTableArray, tTableArray, isOverwrite, isKeepOriginalSplits);
260263
}
261264
}

0 commit comments

Comments
 (0)