Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -440,6 +440,8 @@ public BackupProtos.BackupInfo toProtosBackupInfo() {
builder.setBackupType(BackupProtos.BackupType.valueOf(getType().name()));
builder.setWorkersNumber(workers);
builder.setBandwidth(bandwidth);
builder.setContinuousBackupEnabled(isContinuousBackupEnabled());
builder.setIncrCommittedWalTs(getIncrCommittedWalTs());
return builder.build();
}

Expand Down Expand Up @@ -535,6 +537,8 @@ public static BackupInfo fromProto(BackupProtos.BackupInfo proto) {
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;
}

Expand Down Expand Up @@ -578,6 +582,12 @@ public String getShortDescription() {
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("}");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -101,11 +110,89 @@ public void testBackupDescribeCommand() throws Exception {
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);
String desc = status.getShortDescription();
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);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

for the incr_committed_wal_ts and message of Committed WAL time for incremental backup= , is it possible to add a new assertion ?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense, added now

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);
}
}
}
2 changes: 2 additions & 0 deletions hbase-protocol-shaded/src/main/protobuf/Backup.proto
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down