This is an automated email from the ASF dual-hosted git repository.

andor pushed a commit to branch HBASE-28957_rebased
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 27ea7b34f30054b9a1c06f632040202a8c8b2ff2
Author: asolomon <[email protected]>
AuthorDate: Wed Jul 16 00:18:40 2025 +0530

    HBASE-29449 Update backup describe command for continuous backup (#7045)
    
    Signed-off-by: Tak Lon (Stephen) Wu <[email protected]>
    Reviewed by: Kevin Geiszler <[email protected]>
---
 .../org/apache/hadoop/hbase/backup/BackupInfo.java | 10 +++
 .../hadoop/hbase/backup/TestBackupDescribe.java    | 87 ++++++++++++++++++++++
 .../src/main/protobuf/Backup.proto                 |  2 +
 3 files changed, 99 insertions(+)

diff --git 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
index 0997aec19ec..47731b935ef 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
@@ -440,6 +440,8 @@ public class BackupInfo implements Comparable<BackupInfo> {
     builder.setBackupType(BackupProtos.BackupType.valueOf(getType().name()));
     builder.setWorkersNumber(workers);
     builder.setBandwidth(bandwidth);
+    builder.setContinuousBackupEnabled(isContinuousBackupEnabled());
+    builder.setIncrCommittedWalTs(getIncrCommittedWalTs());
     return builder.build();
   }
 
@@ -535,6 +537,8 @@ public class BackupInfo implements Comparable<BackupInfo> {
     context.setType(BackupType.valueOf(proto.getBackupType().name()));
     context.setWorkers(proto.getWorkersNumber());
     context.setBandwidth(proto.getBandwidth());
+    context.setContinuousBackupEnabled(proto.getContinuousBackupEnabled());
+    context.setIncrCommittedWalTs(proto.getIncrCommittedWalTs());
     return context;
   }
 
@@ -578,6 +582,12 @@ public class BackupInfo implements Comparable<BackupInfo> {
       cal.setTimeInMillis(getCompleteTs());
       date = cal.getTime();
       sb.append("End time=" + date).append(",");
+      if (getType() == BackupType.INCREMENTAL) {
+        cal = Calendar.getInstance();
+        cal.setTimeInMillis(getIncrCommittedWalTs());
+        date = cal.getTime();
+        sb.append("Committed WAL time for incremental backup=" + 
date).append(",");
+      }
     }
     sb.append("Progress=" + getProgress() + "%");
     sb.append("}");
diff --git 
a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
 
b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
index 6084dc730ee..54be17f94da 100644
--- 
a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
+++ 
b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
@@ -17,18 +17,27 @@
  */
 package org.apache.hadoop.hbase.backup;
 
+import static 
org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR;
+import static 
org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP;
+import static 
org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_DEFAULT;
+import static 
org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_KEY;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
 import java.util.List;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
 import org.apache.hadoop.hbase.backup.impl.BackupCommands;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -101,6 +110,7 @@ public class TestBackupDescribe extends TestBackupBase {
     String response = baos.toString();
     assertTrue(response.indexOf(backupId) > 0);
     assertTrue(response.indexOf("COMPLETE") > 0);
+    assertTrue(response.contains("IsContinuous=false"));
 
     BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection());
     BackupInfo status = table.readBackupInfo(backupId);
@@ -108,4 +118,81 @@ public class TestBackupDescribe extends TestBackupBase {
     table.close();
     assertTrue(response.indexOf(desc) >= 0);
   }
+
+  @Test
+  public void testBackupDescribeCommandForContinuousBackup() throws Exception {
+    LOG.info("test backup describe on a single table with data: command-line");
+    Path root = TEST_UTIL.getDataTestDirOnTestFS();
+    Path backupWalDir = new Path(root, "testBackupDescribeCommand");
+    FileSystem fs = FileSystem.get(conf1);
+    fs.mkdirs(backupWalDir);
+    conf1.set(CONF_CONTINUOUS_BACKUP_WAL_DIR, backupWalDir.toString());
+    conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, true);
+
+    try (BackupSystemTable table = new 
BackupSystemTable(TEST_UTIL.getConnection())) {
+      // Continuous backup
+      String[] backupArgs = new String[] { "create", BackupType.FULL.name(), 
BACKUP_ROOT_DIR, "-t",
+        table1.getNameAsString(), "-" + OPTION_ENABLE_CONTINUOUS_BACKUP };
+      int ret = ToolRunner.run(conf1, new BackupDriver(), backupArgs);
+      assertEquals("Backup should succeed", 0, ret);
+      List<BackupInfo> backups = table.getBackupHistory();
+      String backupId = backups.get(0).getBackupId();
+      assertTrue(checkSucceeded(backupId));
+      LOG.info("backup complete");
+
+      BackupInfo info = getBackupAdmin().getBackupInfo(backupId);
+      assertTrue(info.getState() == BackupState.COMPLETE);
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      System.setOut(new PrintStream(baos));
+
+      // Run backup describe
+      String[] args = new String[] { "describe", backupId };
+      ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertTrue(ret == 0);
+      String response = baos.toString();
+      assertTrue(response.contains(backupId));
+      assertTrue(response.contains("COMPLETE"));
+      assertTrue(response.contains("IsContinuous=true"));
+      BackupInfo status = table.readBackupInfo(backupId);
+      String desc = status.getShortDescription();
+      assertTrue(response.contains(desc));
+
+      // load table
+      Put p;
+      for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+        p = new Put(Bytes.toBytes("row" + i));
+        p.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+        TEST_UTIL.getConnection().getTable(table1).put(p);
+      }
+      Thread.sleep(5000);
+
+      // Incremental backup
+      backupArgs = new String[] { "create", BackupType.INCREMENTAL.name(), 
BACKUP_ROOT_DIR, "-t",
+        table1.getNameAsString() };
+      ret = ToolRunner.run(conf1, new BackupDriver(), backupArgs);
+      assertEquals("Incremental Backup should succeed", 0, ret);
+      backups = table.getBackupHistory();
+      String incrBackupId = backups.get(0).getBackupId();
+      assertTrue(checkSucceeded(incrBackupId));
+      LOG.info("Incremental backup complete");
+
+      // Run backup describe
+      args = new String[] { "describe", incrBackupId };
+      ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertTrue(ret == 0);
+      response = baos.toString();
+      assertTrue(response.contains(incrBackupId));
+      assertTrue(response.contains("COMPLETE"));
+      assertTrue(response.contains("Committed WAL time for incremental 
backup="));
+      status = table.readBackupInfo(incrBackupId);
+      desc = status.getShortDescription();
+      assertTrue(response.contains(desc));
+    } finally {
+      if (fs.exists(backupWalDir)) {
+        fs.delete(backupWalDir, true);
+      }
+      conf1.unset(CONF_CONTINUOUS_BACKUP_WAL_DIR);
+      conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, 
REPLICATION_MARKER_ENABLED_DEFAULT);
+    }
+  }
 }
diff --git a/hbase-protocol-shaded/src/main/protobuf/Backup.proto 
b/hbase-protocol-shaded/src/main/protobuf/Backup.proto
index 95a29867325..0ad1f5ba619 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Backup.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Backup.proto
@@ -93,6 +93,8 @@ message BackupInfo {
   optional uint32 workers_number = 11;
   optional uint64 bandwidth = 12;
   map<string, RSTimestampMap> table_set_timestamp = 13;
+  optional bool continuous_backup_enabled = 14;
+  optional uint64 incr_committed_wal_ts = 15;
 
   message RSTimestampMap {
     map<string, uint64> rs_timestamp = 1;

Reply via email to