Skip to content

Commit 6e9561e

Browse files
HBASE-29521: Update Restore Command to Handle Bulkloaded Files (#7300)
Signed-off-by: Tak Lon (Stephen) Wu <[email protected]> Signed-off-by: Andor Molnár [email protected] Reviewed by: Kevin Geiszler <[email protected]> Reviewed by: Kota-SH <[email protected]>
1 parent f5d18fc commit 6e9561e

22 files changed

+1714
-111
lines changed

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

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727

2828
import java.net.URI;
2929
import org.apache.hadoop.conf.Configuration;
30+
import org.apache.hadoop.fs.FileSystem;
3031
import org.apache.hadoop.fs.Path;
3132
import org.apache.hadoop.hbase.HBaseConfiguration;
3233
import org.apache.hadoop.hbase.TableName;
@@ -101,9 +102,15 @@ protected int executeRestore(boolean check, TableName[] fromTables, TableName[]
101102
return -5;
102103
}
103104

104-
PointInTimeRestoreRequest pointInTimeRestoreRequest = new PointInTimeRestoreRequest.Builder()
105-
.withBackupRootDir(backupRootDir).withCheck(check).withFromTables(fromTables)
106-
.withToTables(toTables).withOverwrite(isOverwrite).withToDateTime(endTime).build();
105+
// TODO: Currently hardcoding keepOriginalSplits=false and restoreRootDir via tmp dir.
106+
// These should come from user input (same issue exists in normal restore).
107+
// Expose them as configurable options in future.
108+
PointInTimeRestoreRequest pointInTimeRestoreRequest =
109+
new PointInTimeRestoreRequest.Builder().withBackupRootDir(backupRootDir).withCheck(check)
110+
.withFromTables(fromTables).withToTables(toTables).withOverwrite(isOverwrite)
111+
.withToDateTime(endTime).withKeepOriginalSplits(false).withRestoreRootDir(
112+
BackupUtils.getTmpRestoreOutputDir(FileSystem.get(conf), conf).toString())
113+
.build();
107114

108115
client.pointInTimeRestore(pointInTimeRestoreRequest);
109116
} catch (Exception e) {

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

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,25 +27,33 @@
2727
public final class PointInTimeRestoreRequest {
2828

2929
private final String backupRootDir;
30+
private final String restoreRootDir;
3031
private final boolean check;
3132
private final TableName[] fromTables;
3233
private final TableName[] toTables;
3334
private final boolean overwrite;
3435
private final long toDateTime;
36+
private final boolean isKeepOriginalSplits;
3537

3638
private PointInTimeRestoreRequest(Builder builder) {
3739
this.backupRootDir = builder.backupRootDir;
40+
this.restoreRootDir = builder.restoreRootDir;
3841
this.check = builder.check;
3942
this.fromTables = builder.fromTables;
4043
this.toTables = builder.toTables;
4144
this.overwrite = builder.overwrite;
4245
this.toDateTime = builder.toDateTime;
46+
this.isKeepOriginalSplits = builder.isKeepOriginalSplits;
4347
}
4448

4549
public String getBackupRootDir() {
4650
return backupRootDir;
4751
}
4852

53+
public String getRestoreRootDir() {
54+
return restoreRootDir;
55+
}
56+
4957
public boolean isCheck() {
5058
return check;
5159
}
@@ -66,19 +74,30 @@ public long getToDateTime() {
6674
return toDateTime;
6775
}
6876

77+
public boolean isKeepOriginalSplits() {
78+
return isKeepOriginalSplits;
79+
}
80+
6981
public static class Builder {
7082
private String backupRootDir;
83+
private String restoreRootDir;
7184
private boolean check = false;
7285
private TableName[] fromTables;
7386
private TableName[] toTables;
7487
private boolean overwrite = false;
7588
private long toDateTime;
89+
private boolean isKeepOriginalSplits;
7690

7791
public Builder withBackupRootDir(String backupRootDir) {
7892
this.backupRootDir = backupRootDir;
7993
return this;
8094
}
8195

96+
public Builder withRestoreRootDir(String restoreRootDir) {
97+
this.restoreRootDir = restoreRootDir;
98+
return this;
99+
}
100+
82101
public Builder withCheck(boolean check) {
83102
this.check = check;
84103
return this;
@@ -104,6 +123,11 @@ public Builder withToDateTime(long dateTime) {
104123
return this;
105124
}
106125

126+
public Builder withKeepOriginalSplits(boolean isKeepOriginalSplits) {
127+
this.isKeepOriginalSplits = isKeepOriginalSplits;
128+
return this;
129+
}
130+
107131
public PointInTimeRestoreRequest build() {
108132
return new PointInTimeRestoreRequest(this);
109133
}

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

Lines changed: 86 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,8 @@
2020
import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS;
2121
import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR;
2222
import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS;
23-
import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR;
2423
import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS;
24+
import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.WALS_DIR;
2525
import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT;
2626
import static org.apache.hadoop.hbase.mapreduce.WALPlayer.IGNORE_EMPTY_FILES;
2727

@@ -30,6 +30,7 @@
3030
import java.text.SimpleDateFormat;
3131
import java.util.ArrayList;
3232
import java.util.Arrays;
33+
import java.util.Collections;
3334
import java.util.Date;
3435
import java.util.List;
3536
import java.util.Map;
@@ -41,9 +42,12 @@
4142
import org.apache.hadoop.fs.Path;
4243
import org.apache.hadoop.hbase.HBaseConfiguration;
4344
import org.apache.hadoop.hbase.TableName;
45+
import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
4446
import org.apache.hadoop.hbase.backup.PointInTimeRestoreRequest;
47+
import org.apache.hadoop.hbase.backup.RestoreJob;
4548
import org.apache.hadoop.hbase.backup.RestoreRequest;
4649
import org.apache.hadoop.hbase.backup.util.BackupUtils;
50+
import org.apache.hadoop.hbase.backup.util.BulkFilesCollector;
4751
import org.apache.hadoop.hbase.client.Connection;
4852
import org.apache.hadoop.hbase.mapreduce.WALInputFormat;
4953
import org.apache.hadoop.hbase.mapreduce.WALPlayer;
@@ -305,6 +309,63 @@ private void restoreTableWithWalReplay(TableName sourceTable, TableName targetTa
305309

306310
backupAdmin.restore(restoreRequest);
307311
replayWal(sourceTable, targetTable, backupMetadata.getStartTs(), endTime);
312+
313+
reBulkloadFiles(sourceTable, targetTable, backupMetadata.getStartTs(), endTime,
314+
request.isKeepOriginalSplits(), request.getRestoreRootDir());
315+
}
316+
317+
/**
318+
* Re-applies/re-bulkloads store files discovered from WALs into the target table.
319+
* <p>
320+
* <b>Note:</b> this method re-uses the same {@link RestoreJob} MapReduce job that we originally
321+
* implemented for performing full and incremental backup restores. The MR job (obtained via
322+
* {@link BackupRestoreFactory#getRestoreJob(Configuration)}) is used here to perform an HFile
323+
* bulk-load of the discovered store files into {@code targetTable}.
324+
* @param sourceTable source table name (used for locating bulk files and logging)
325+
* @param targetTable destination table to bulk-load the HFiles into
326+
* @param startTime start of WAL range (ms)
327+
* @param endTime end of WAL range (ms)
328+
* @param keepOriginalSplits pass-through flag to control whether original region splits are
329+
* preserved
330+
* @param restoreRootDir local/DFS path under which temporary and output dirs are created
331+
* @throws IOException on IO or job failure
332+
*/
333+
private void reBulkloadFiles(TableName sourceTable, TableName targetTable, long startTime,
334+
long endTime, boolean keepOriginalSplits, String restoreRootDir) throws IOException {
335+
336+
Configuration conf = HBaseConfiguration.create(conn.getConfiguration());
337+
conf.setBoolean(RestoreJob.KEEP_ORIGINAL_SPLITS_KEY, keepOriginalSplits);
338+
339+
String walBackupDir = conn.getConfiguration().get(CONF_CONTINUOUS_BACKUP_WAL_DIR);
340+
Path walDirPath = new Path(walBackupDir);
341+
conf.set(RestoreJob.BACKUP_ROOT_PATH_KEY, walDirPath.toString());
342+
343+
RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf);
344+
345+
List<Path> bulkloadFiles =
346+
collectBulkFiles(sourceTable, targetTable, startTime, endTime, new Path(restoreRootDir));
347+
348+
if (bulkloadFiles.isEmpty()) {
349+
LOG.info("No bulk-load files found for {} in time range {}-{}. Skipping bulkload restore.",
350+
sourceTable, startTime, endTime);
351+
return;
352+
}
353+
354+
Path[] pathsArray = bulkloadFiles.toArray(new Path[0]);
355+
356+
try {
357+
// Use the existing RestoreJob MR job (the same MapReduce job used for full/incremental
358+
// restores)
359+
// to perform the HFile bulk-load of the discovered store files into `targetTable`.
360+
restoreService.run(pathsArray, new TableName[] { sourceTable }, new Path(restoreRootDir),
361+
new TableName[] { targetTable }, false);
362+
LOG.info("Re-bulkload completed for {}", targetTable);
363+
} catch (Exception e) {
364+
String errorMessage =
365+
String.format("Re-bulkload failed for %s: %s", targetTable, e.getMessage());
366+
LOG.error(errorMessage, e);
367+
throw new IOException(errorMessage, e);
368+
}
308369
}
309370

310371
/**
@@ -329,6 +390,29 @@ private void replayWal(TableName sourceTable, TableName targetTable, long startT
329390
executeWalReplay(validDirs, sourceTable, targetTable, startTime, endTime);
330391
}
331392

393+
private List<Path> collectBulkFiles(TableName sourceTable, TableName targetTable, long startTime,
394+
long endTime, Path restoreRootDir) throws IOException {
395+
396+
String walBackupDir = conn.getConfiguration().get(CONF_CONTINUOUS_BACKUP_WAL_DIR);
397+
Path walDirPath = new Path(walBackupDir);
398+
LOG.info(
399+
"Starting WAL bulk-file collection for source: {}, target: {}, time range: {} - {}, WAL backup dir: {}, restore root: {}",
400+
sourceTable, targetTable, startTime, endTime, walDirPath, restoreRootDir);
401+
402+
List<String> validDirs =
403+
getValidWalDirs(conn.getConfiguration(), walDirPath, startTime, endTime);
404+
if (validDirs.isEmpty()) {
405+
LOG.warn("No valid WAL directories found for range {} - {}. Skipping bulk-file collection.",
406+
startTime, endTime);
407+
return Collections.emptyList();
408+
}
409+
410+
String walDirsCsv = String.join(",", validDirs);
411+
412+
return BulkFilesCollector.collectFromWalDirs(HBaseConfiguration.create(conn.getConfiguration()),
413+
walDirsCsv, restoreRootDir, sourceTable, targetTable, startTime, endTime);
414+
}
415+
332416
/**
333417
* Fetches valid WAL directories based on the given time range.
334418
*/
@@ -356,7 +440,7 @@ private List<String> getValidWalDirs(Configuration conf, Path walBackupDir, long
356440
validDirs.add(dayDir.getPath().toString());
357441
}
358442
} catch (ParseException e) {
359-
LOG.warn("Skipping invalid directory name: " + dirName, e);
443+
LOG.warn("Skipping invalid directory name: {}", dirName, e);
360444
}
361445
}
362446
return validDirs;

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -81,7 +81,7 @@
8181
import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BackupCommand;
8282
import org.apache.hadoop.hbase.backup.BackupType;
8383
import org.apache.hadoop.hbase.backup.HBackupFileSystem;
84-
import org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager;
84+
import org.apache.hadoop.hbase.backup.util.BackupFileSystemManager;
8585
import org.apache.hadoop.hbase.backup.util.BackupSet;
8686
import org.apache.hadoop.hbase.backup.util.BackupUtils;
8787
import org.apache.hadoop.hbase.client.Admin;

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,9 +19,9 @@
1919

2020
import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR;
2121
import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.JOB_NAME_CONF_KEY;
22-
import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR;
23-
import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR;
2422
import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS;
23+
import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.BULKLOAD_FILES_DIR;
24+
import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.WALS_DIR;
2525
import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT;
2626

2727
import java.io.IOException;

0 commit comments

Comments
 (0)