ndimiduk commented on code in PR #6370:
URL: https://github.com/apache/hbase/pull/6370#discussion_r1848576428


##########
hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java:
##########
@@ -81,6 +101,28 @@ public static Collection<Object[]> data() {
   public TestIncrementalBackup(Boolean b) {
   }
 
+  @After
+  public void ensurePreviousBackupTestsAreCleanedUp() throws Exception {
+    TEST_UTIL.flush(table1);
+    TEST_UTIL.flush(table2);
+    TEST_UTIL.flush(table1_restore);
+
+    TEST_UTIL.truncateTable(table1).close();
+    TEST_UTIL.truncateTable(table2).close();
+    TEST_UTIL.truncateTable(table1_restore).close();
+
+    TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().forEach(rst -> {
+      try {
+        LogRoller walRoller = rst.getRegionServer().getWalRoller();
+        walRoller.requestRollAll();
+        walRoller.waitUntilWalRollFinished();
+      } catch (Exception ignored) {
+      }
+    });
+
+    TestBackupBase.setUp();

Review Comment:
   nit: calling setup method from an `@After` method is surprising.



##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java:
##########
@@ -30,6 +30,9 @@
 
 @InterfaceAudience.Private
 public interface RestoreJob extends Configurable {
+
+  String KEEP_ORIGINAL_SPLITS_OPT = "keep_original_splits";

Review Comment:
   I missed this before, but it's customary to include `hbase.` as a prefix on 
configuration keys that are used by HBase. I suggest 
`hbase.backup.restorejob.keep_original_splits`.



##########
hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java:
##########
@@ -235,6 +277,92 @@ public void TestIncBackupRestore() throws Exception {
     }
   }
 
+  @Test
+  public void TestIncBackupRestoreWithOriginalSplits() throws Exception {
+    byte[] fam1 = Bytes.toBytes("f");
+    byte[] mobFam = Bytes.toBytes("mob");
+
+    List<TableName> tables = Lists.newArrayList(table1);
+    TableDescriptor newTable1Desc =
+      
TableDescriptorBuilder.newBuilder(table1Desc).setColumnFamily(ColumnFamilyDescriptorBuilder
+        
.newBuilder(mobFam).setMobEnabled(true).setMobThreshold(5L).build()).build();
+    TEST_UTIL.getAdmin().modifyTable(newTable1Desc);
+
+    try (Connection conn = ConnectionFactory.createConnection(conf1)) {
+      BackupAdminImpl backupAdmin = new BackupAdminImpl(conn);
+      BackupRequest request = createBackupRequest(BackupType.FULL, tables, 
BACKUP_ROOT_DIR);
+      String fullBackupId = backupAdmin.backupTables(request);
+      assertTrue(checkSucceeded(fullBackupId));
+
+      TableName[] fromTables = new TableName[] { table1 };
+      TableName[] toTables = new TableName[] { table1_restore };
+      backupAdmin.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, 
fullBackupId, false,
+        fromTables, toTables, true, true));
+
+      Table table = conn.getTable(table1_restore);

Review Comment:
   do be careful to manage `Closeables` in this method -- `Table`, `Admin` 
instances should be closed.



##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java:
##########
@@ -43,8 +44,13 @@ private BackupRestoreFactory() {
    * @return backup restore job instance
    */
   public static RestoreJob getRestoreJob(Configuration conf) {
+    Class<? extends RestoreJob> defaultCls =
+      conf.getBoolean(RestoreJob.KEEP_ORIGINAL_SPLITS_OPT, false)

Review Comment:
   Okay you convinced me about offering the old behavior. Should the new 
behavior be the default though?



##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java:
##########
@@ -437,6 +445,29 @@ protected void walToHFiles(List<String> dirPaths, 
List<String> tableList) throws
     }
   }
 
+  private void incrementalCopyBulkloadHFiles(FileSystem tgtFs, TableName tn) 
throws IOException {
+    Path bulkOutDir = getBulkOutputDirForTable(tn);
+    FileSystem fs = FileSystem.get(conf);
+
+    if (fs.exists(bulkOutDir)) {

Review Comment:
   So long as there's not an error case on which we need to fail, I'm fine with 
just a comment. This is covered in unit test?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to