Skip to content
Open
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
Copy link
Contributor

Choose a reason for hiding this comment

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

Haven't gotten around to processing the changes in this file, but can you sketch why they are needed? Since your original ticket only discusses an issue with incremental backups.

Copy link
Contributor

Choose a reason for hiding this comment

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

Figured it out, it's to ensure the newTimestamps for no longer active region servers are updated.

Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,14 @@

import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.BackupCopyJob;
import org.apache.hadoop.hbase.backup.BackupInfo;
Expand All @@ -38,7 +44,9 @@
import org.apache.hadoop.hbase.backup.util.BackupUtils;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -144,21 +152,21 @@ public void execute() throws IOException {
// logs while we do the backup.
backupManager.writeBackupStartCode(0L);
}
// We roll log here before we do the snapshot. It is possible there is duplicate data
// in the log that is already in the snapshot. But if we do it after the snapshot, we
// could have data loss.
// A better approach is to do the roll log on each RS in the same global procedure as
// the snapshot.
LOG.info("Execute roll log procedure for full backup ...");

// Gather the bulk loads being tracked by the system, which can be deleted (since their data
// will be part of the snapshot being taken). We gather this list before taking the actual
// snapshots for the same reason as the log rolls.
List<BulkLoad> bulkLoadsToDelete = backupManager.readBulkloadRows(tableList);
Map<String, Long> previousLogRollsByHost = backupManager.readRegionServerLastLogRollResult();

// We roll log here before we do the snapshot. It is possible there is duplicate data
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: this comment block can be improved a bit.

  • Won't there always be data duplication between the rolled log and the snapshot? I don't see a reason to mention this.
  • I would mention that we roll the logs to create a boundary for WALs that need to be tracked for upcoming incremental backups.
  • A better approach is -> ...would be (to avoid confusion that you're talking about what is actually happen

// in the log that is already in the snapshot. But if we do it after the snapshot, we
// could have data loss.
// A better approach is to do the roll log on each RS in the same global procedure as
// the snapshot.
LOG.info("Execute roll log procedure for full backup ...");
BackupUtils.logRoll(conn, backupInfo.getBackupRootDir(), conf);

newTimestamps = backupManager.readRegionServerLastLogRollResult();
Map<String, Long> latestLogRollsByHost = backupManager.readRegionServerLastLogRollResult();

// SNAPSHOT_TABLES:
backupInfo.setPhase(BackupPhase.SNAPSHOT);
Expand All @@ -181,6 +189,50 @@ public void execute() throws IOException {
// set overall backup status: complete. Here we make sure to complete the backup.
// After this checkpoint, even if entering cancel process, will let the backup finished
backupInfo.setState(BackupState.COMPLETE);

// Scan oldlogs for dead/decommissioned hosts and add their max WAL timestamps
// to newTimestamps. This ensures subsequent incremental backups won't try to back up
// WALs that are already covered by this full backup's snapshot.
Path walRootDir = CommonFSUtils.getWALRootDir(conf);
Path logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
FileSystem fs = walRootDir.getFileSystem(conf);

List<FileStatus> allLogs = new ArrayList<>();
for (FileStatus hostLogDir : fs.listStatus(logDir)) {
String host = BackupUtils.parseHostNameFromLogFile(hostLogDir.getPath());
if (host == null) {
continue;
}
allLogs.addAll(Arrays.asList(fs.listStatus(hostLogDir.getPath())));
}
allLogs.addAll(Arrays.asList(fs.listStatus(oldLogDir)));
Copy link
Contributor

@DieterDP-ng DieterDP-ng Jan 9, 2026

Choose a reason for hiding this comment

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

There's apparently a config key hbase.separate.oldlogdir.by.regionserver that adds the server address as a subfolder, so this line will not work as expected when that's set to true. (Solution is to also check files that are one folder deeper.)

Question: when exactly (and by what) are WALs moved to the oldWals folder (oldLogDir)?

Copy link
Contributor Author

@hgromer hgromer Jan 12, 2026

Choose a reason for hiding this comment

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

Question: when exactly (and by what) are WALs moved to the oldWals folder (oldLogDir)?

They are moved asynchronously, after all edits to that WAL file have been closed and all edits have been replicated (if a peer is online)

This happens in AbstractFSWAL#cleanOldLogs


newTimestamps = new HashMap<>();

for (FileStatus log : allLogs) {
if (AbstractFSWALProvider.isMetaFile(log.getPath())) {
continue;
}
String host = BackupUtils.parseHostNameFromLogFile(log.getPath());
if (host == null) {
continue;
}
long timestamp = BackupUtils.getCreationTime(log.getPath());
Long previousLogRoll = previousLogRollsByHost.get(host);
Long latestLogRoll = latestLogRollsByHost.get(host);
boolean isInactive = latestLogRoll == null || latestLogRoll.equals(previousLogRoll);

if (isInactive) {
long currentTs = newTimestamps.getOrDefault(host, 0L);
if (timestamp > currentTs) {
newTimestamps.put(host, timestamp);
}
} else {
newTimestamps.put(host, latestLogRoll);
}
}

// The table list in backupInfo is good for both full backup and incremental backup.
// For incremental backup, it contains the incremental backup table set.
backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps);
Expand Down
Copy link
Contributor

Choose a reason for hiding this comment

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

Hi @hgromer, I agree HBASE-29776 is an issue (sorry for not responding sooner there, I was on vacation the past weeks), but I'm not yet convinced this is the right approach to fix it. It feels very complex to reason about, so I wonder if there isn't a simpler approach. Already wanted to give some intermediate feedback while I think a bit more about it.

  • Since newTimestamps never is pruned, the entry in the backup table will keep growing over time.
  • newTimestamps will end up being written in BackupSystemTable#createPutForWriteRegionServerLogTimestamp, but these change no longer match the javadoc of that method.

Copy link
Contributor

Choose a reason for hiding this comment

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

My suggestion is to simplify all changes in this file to (the fix for the excluded log files is also needed):

    LOG.info("Execute roll log procedure for incremental backup ...");
    long rollStartTs = EnvironmentEdgeManager.currentTime();
    BackupUtils.logRoll(conn, backupInfo.getBackupRootDir(), conf);

    Map<String, Long> rollTimestamps = readRegionServerLastLogRollResult();

    Map<String, Long> newTimestamps =
      rollTimestamps.entrySet().stream()
        // Region servers that are offline since the last backup will have old roll timestamps,
        // prune their information here, as it is not relevant to be stored or used for finding
        // the relevant logs.
        .filter(entry -> entry.getValue() > rollStartTs)
        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));

    // This method needs to be adjusted to use "rollStartTs" if an entry is not found in newTimestamps.
    // Or alternatively: getLogFilesForNewBackup(previousTimestampMins,
    //     DefaultedMap.defaultedMap(newTimestamps, rollStartTs), conf, savedStartCode);
    logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);

Then when finding which logs to include, these are the options:

  • server found in both previous and newTimestamps: a region server that is unchanged, include logs older than previous and newer than newTimestamps
  • server found in only previous: a region server that has gone offline, all logs will be older than rollStartTs and should be included
  • server found in only newTimestamps: a new region server, include all logs that are older than the corresponding newTimestamp
  • server found in neither: a region server that was started and went back offline in between the previous and current backup, all logs will be older than rollStartTs and should be included

This approach will keep newTimestamps limited to the relevant entries. We could consider cleaning up the entries for readRegionServerLastLogRollResult as well, but left that out of scope for now.

Similar code suffices in the FullTableBackupClient.

Copy link
Contributor Author

@hgromer hgromer Jan 7, 2026

Choose a reason for hiding this comment

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

No worries, appreciate you taking the time to look here. We've found a slew of bugs in the WAL retention system and I think it's important to get this right, so happy to iterate on feedback.

Since newTimestamps never is pruned, the entry in the backup table will keep growing over time.

Agree with this. It's something we should take a look at.

To your point about WAL retention and boundaries, conceptually, I've been trying to think about it from the perspective of "which WAL files have been backed up". Otherwise you run into issues when a host goes offline.

For example, in the case where

We have a Server A with row X

  1. An incremental backup is taken, A is rolled
  2. A writes more WAL files
  3. Row X is deleted
  4. A major compaction happens
  5. A full backup is taken, WALs are rolled, but we don't update the timestamp for A. Row X is not included in the full backup
  6. An incremental backup is taken, we still have a very old roll time for this host. Row X is backed up again, and shows up in the backup even though we had previously deleted (but the tombstone no longer exists).

So we've resurfaced dead data that shouldn't be included. It's problematic to back up WALs that are very old. So this is the main culprit for the added complexity here

Additionally, I'm weary of comparing timestamps across hosts, which is why I was wary of doing something like generating a boundary timestamp in the backup process, which happens client side and opted to compare WAL timestamps which are generated by the same host.

server found in both previous and newTimestamps: a region server that is unchanged, include logs older than previous and newer than newTimestamps

server found in only previous: a region server that has gone offline, all logs will be older than rollStartTs and should be included

If I understand correctly, run into this issue

server found in only newTimestamps: a new region server, include all logs that are older than the corresponding newTimestamp

server found in neither: a region server that was started and went back offline in between the previous and current backup, all logs will be older than rollStartTs and should be included

Agree here on the first backup this happens, but we never update the host TS and so we'll continue to backup the WAL files and run into the issue mentioned above.

I'd be more than happy to find a simpler solution though, I really don't love how complex this WAL retention boundary logic is; but struggled to do so and also avoid corrupting the data

Copy link
Contributor Author

@hgromer hgromer Jan 7, 2026

Choose a reason for hiding this comment

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

Map<String, Long> newTimestamps =
  rollTimestamps.entrySet().stream()
  // Region servers that are offline since the last backup will have old roll timestamps,
  // prune their information here, as it is not relevant to be stored or used for finding
  // the relevant logs.
  .filter(entry -> entry.getValue() > rollStartTs)
  .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));

I'm not sure I necessarily agree with this logic. B/c the RS not being rolled this go around doesn't mean we've backed up all the files from the RS we need to backup. It just means the host doesn't exist on the cluster at the moment.

  1. Server A is backed up
  2. Server A generate more WAL files
  3. Server A is removed from cluster
  4. New backup occurs, but we don't get a roll time for Server A so we ignore its files

We need to backup the files that were generated between the last backup, and this backup

Copy link
Contributor

Choose a reason for hiding this comment

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

No worries, appreciate you taking the time to look here. We've found a slew of bugs in the WAL retention system and I think it's important to get this right, so happy to iterate on feedback.

Since newTimestamps never is pruned, the entry in the backup table will keep growing over time.

Agree with this. It's something we should take a look at.

To your point about WAL retention and boundaries, conceptually, I've been trying to think about it from the perspective of "which WAL files have been backed up". Otherwise you run into issues when a host goes offline.

For example, in the case where

We have a Server A with row X

1. An incremental backup is taken, A is rolled

2. A writes more WAL files

3. Row X is deleted

4. A major compaction happens

5. A full backup is taken, WALs are rolled, but we don't update the timestamp for A. Row X is _not_ included in the full backup

6. An incremental backup is taken, we still have a very old roll time for this host. Row X is backed up again, and shows up in the backup even though we had previously deleted (but the tombstone no longer exists).

So we've resurfaced dead data that shouldn't be included. It's problematic to back up WALs that are very old. So this is the main culprit for the added complexity here

I agree with your example, and agree that the change to FullTableBackupClient would fix this. It also shrinks the newTimestamps, which nice.

Additionally, I'm weary of comparing timestamps across hosts, which is why I was wary of doing something like generating a boundary timestamp in the backup process, which happens client side and opted to compare WAL timestamps which are generated by the same host.

I see, I originally thought it might be less code to generate a client pre-roll timestamp, but it doesn't really simplify things. For the FullTableBackupClient at least, the current code is simple enough. I would suggest to split off a dedicated calculateNewTimestamps method with some proper javadoc. (Still thinking about the incremental case.)

Copy link
Contributor

Choose a reason for hiding this comment

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

One advantage of not basing newTimestamps on the logs, but on the roll timestamps would be that the same logic could be used for full and incremental backups.

Copy link
Contributor

@DieterDP-ng DieterDP-ng Jan 9, 2026

Choose a reason for hiding this comment

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

I'm not sure I necessarily agree with this logic. B/c the RS not being rolled this go around doesn't mean we've backed up all the files from the RS we need to backup. It just means the host doesn't exist on the cluster at the moment.

1. Server A is backed up

2. Server A generate more WAL files

3. Server A is removed from cluster

4. New backup occurs, but we don't get a roll time for Server A so we ignore its files

We need to backup the files that were generated between the last backup, and this backup

Not sure we're on the same page here. (Did you miss that I mentioned that getLogFilesForNewBackup needs some adjustments?)

For the example you gave (quoted above), with the code block I suggested, the following would happen:

  • when the logs are rolled, rollTimestamps would not have been updated, since the RS A was offline
  • so the rollTimestamps value for A would be < rollStartTs, and A would not be present as a key in newTimestamps
  • in the updated getLogFilesForNewBackup, we'd fall in case server found in only previous: a region server that has gone offline, all logs will be older than rollStartTs and should be included, so all logs of A would be included.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

One advantage of not basing newTimestamps on the logs, but on the roll timestamps would be that the same logic could be used for full and incremental backups.

It's tricky to use rollTimestamps b/c we're comparing timestamps across different machines, and so we can run into clock skew issues. If a WAL file for a host is generated roughly at the time we set rollTimestamps; and then the host goes offline before it can be rolled.

Using WAL timestamps allows us to always compare timestamp values generated by the same machine. We avoid any weird clock skew issues

Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,6 @@ public IncrementalBackupManager(Connection conn, Configuration conf) throws IOEx
*/
public Map<String, Long> getIncrBackupLogFileMap() throws IOException {
List<String> logList;
Map<String, Long> newTimestamps;
Map<String, Long> previousTimestampMins;

String savedStartCode = readBackupStartCode();
Expand All @@ -83,12 +82,48 @@ public Map<String, Long> getIncrBackupLogFileMap() throws IOException {
LOG.info("Execute roll log procedure for incremental backup ...");
BackupUtils.logRoll(conn, backupInfo.getBackupRootDir(), conf);

newTimestamps = readRegionServerLastLogRollResult();
Map<String, Long> newTimestamps = readRegionServerLastLogRollResult();
Copy link
Contributor

Choose a reason for hiding this comment

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

This method does an unnecessary scan, since you override all entries in the code you add below.


Map<String, Long> latestLogRollByHost = readRegionServerLastLogRollResult();
for (Map.Entry<String, Long> entry : latestLogRollByHost.entrySet()) {
String host = entry.getKey();
long latestLogRoll = entry.getValue();
Long earliestTimestampToIncludeInBackup = previousTimestampMins.get(host);

boolean isInactive = earliestTimestampToIncludeInBackup != null
&& earliestTimestampToIncludeInBackup > latestLogRoll;

long latestTimestampToIncludeInBackup;
if (isInactive) {
LOG.debug("Avoided resetting latest timestamp boundary for {} from {} to {}", host,
earliestTimestampToIncludeInBackup, latestLogRoll);
latestTimestampToIncludeInBackup = earliestTimestampToIncludeInBackup;
} else {
latestTimestampToIncludeInBackup = latestLogRoll;
}
newTimestamps.put(host, latestTimestampToIncludeInBackup);
}

logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);
logList = excludeProcV2WALs(logList);
backupInfo.setIncrBackupFileList(logList);

// Update boundaries based on WALs that will be backed up
Copy link
Contributor

Choose a reason for hiding this comment

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

For my understanding, is this code block an optimization, or a necessary fix for a specific case of appearing/disappearing region servers?

Copy link
Contributor

Choose a reason for hiding this comment

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

Figured it out, it is to update the newTimestamps entries for regionservers that have since gone offline, but for which the logs are now backed up.

for (String logFile : logList) {
Path logPath = new Path(logFile);
String logHost = BackupUtils.parseHostFromOldLog(logPath);
if (logHost == null) {
logHost = BackupUtils.parseHostNameFromLogFile(logPath.getParent());
Copy link
Contributor

Choose a reason for hiding this comment

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

This method seems to support parsing old log names as well, is it possible to merge with the parsing 2 lines above? Though I am confused as to why the former uses logPath and the latter logPath.getParent()

}
if (logHost != null) {
long logTs = BackupUtils.getCreationTime(logPath);
Long latestTimestampToIncludeInBackup = newTimestamps.get(logHost);
if (latestTimestampToIncludeInBackup == null || logTs > latestTimestampToIncludeInBackup) {
LOG.debug("Updating backup boundary for inactive host {}: timestamp={}", logHost, logTs);
newTimestamps.put(logHost, logTs);
}
}
}
return newTimestamps;
}

Expand Down Expand Up @@ -228,15 +263,6 @@ private List<String> getLogFilesForNewBackup(Map<String, Long> olderTimestamps,
} else if (currentLogTS > oldTimeStamp) {
resultLogFiles.add(currentLogFile);
}

// It is possible that a host in .oldlogs is an obsolete region server
Copy link
Contributor

Choose a reason for hiding this comment

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

I think removing this block entirely is wrong. I believe the semantics of newestTimestamps is "ensure we have everything backed up to this timestamp". So if currentLogTS > newTimestamp is true, we should indeed skip this file.

So I think this block should be kept, but adjusted to:

      if (newTimestamp != null && currentLogTS > newTimestamp) {
        newestLogs.add(currentLogFile);
      }

I also think a similar issue is present for the .logs in this method.

Copy link
Contributor

Choose a reason for hiding this comment

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

From your comment in HBASE-29776:

newTimestamp represents the last time a backup rolled the WAL on the RS. If the RegionServer isn't running and therefore isn't able to roll the WAL again, then this timestamp will be in the past, and we end up filtering out all WAL files that were updated since then. Why would we filter out oldWALs that have been created since then? That seems wrong as well

Your comment is correct, but I think the better fix is to ensure the newTimestamps are correctly updated (as you do in your other changes). Removing this block would result in too many logs being included in the backup.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

if (newTimestamp != null && currentLogTS > newTimestamp) {
  newestLogs.add(currentLogFile);
}

I don't think so, this would exclude all WAL files between last backup (previousTimestamps) and the current log roll (newTimestamp). Unless I'm misunderstanding

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think the relevant filtering out for very old log files happens here

So this logic is safe to remove imo

Copy link
Contributor

Choose a reason for hiding this comment

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

I don't think so, this would exclude all WAL files between last backup (previousTimestamps) and the current log roll (newTimestamp). Unless I'm misunderstanding

A WAL log from between last backup and current log roll would have a currentLogTS <= newTimestamp, no? If currentLogTS > newTimestamp it means it was created after the log roll that just happened, and should be excluded from the backup (so it should be in newestLogs)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It depends

  1. In isolation, without the other changes to this PR, it means we'll never include the WAL files for a host that was removed for the cluster. Keep in mind, currentLogFile is an oldLog. So backup happens, new wals get created and moved to oldWAL dir, server goes offline, the boundary for that server is never updated, those WAL files will never be included in future backups and we lose data
  2. In the context of this PR, we update the boundary for each host based on the latest WAL that was included in the backup for the host, so this logic actually completely irrelevant b/c newTimestamp will never be null

// so newestTimestamps.get(host) here can be null.
// Even if these logs belong to a obsolete region server, we still need
// to include they to avoid loss of edits for backup.
Long newTimestamp = newestTimestamps.get(host);
if (newTimestamp == null || currentLogTS > newTimestamp) {
newestLogs.add(currentLogFile);
}
}
// remove newest log per host because they are still in use
resultLogFiles.removeAll(newestLogs);
Expand Down
Copy link
Contributor

Choose a reason for hiding this comment

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

We should also have a testcases for:

  • a RS goes offline after a full backup was taken, before an incremental is taken
  • a RS comes online & goes back offline before a full backup
  • a RS comes online & goes back offline after a full backup was taken, before an incremental backup

At the same time, I realize the test code for each of these is quite elaborate. So I'm thinking it might be better to write a unit-type test for IncrementalBackupManager where the logs are abstracted away in an interface. Test code could then easily define which logs are present in a given scenario, and production code uses a file-based implementation of the logs.

If you want, I could give a shot at refactoring the code for this.

Original file line number Diff line number Diff line change
@@ -0,0 +1,129 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.backup;

import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;

import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtil;
import org.apache.hadoop.hbase.SingleProcessHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hbase.thirdparty.com.google.common.collect.Lists;

/**
* Tests that WAL files from offline/inactive RegionServers are handled correctly during backup.
* Specifically verifies that WALs from an offline RS are:
* <ol>
* <li>Backed up once in the first backup after the RS goes offline</li>
* <li>NOT re-backed up in subsequent backups</li>
* </ol>
*/
@Category(LargeTests.class)
public class TestBackupOfflineRS extends TestBackupBase {

@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestBackupOfflineRS.class);

private static final Logger LOG = LoggerFactory.getLogger(TestBackupOfflineRS.class);

@BeforeClass
public static void setUp() throws Exception {
TEST_UTIL = new HBaseTestingUtil();
conf1 = TEST_UTIL.getConfiguration();
conf1.setInt("hbase.regionserver.info.port", -1);
autoRestoreOnFailure = true;
useSecondCluster = false;
setUpHelper();
// Start an additional RS so we have at least 2
TEST_UTIL.getMiniHBaseCluster().startRegionServer();
TEST_UTIL.waitTableAvailable(table1);
}

/**
* Tests that when a full backup is taken while an RS is offline (with WALs in oldlogs), the
* offline host's timestamps are recorded so subsequent incremental backups don't re-include those
* WALs.
*/
@Test
public void testBackupWithOfflineRS() throws Exception {
LOG.info("Starting testFullBackupWithOfflineRS");

SingleProcessHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
List<TableName> tables = Lists.newArrayList(table1);

if (cluster.getNumLiveRegionServers() < 2) {
cluster.startRegionServer();
Thread.sleep(2000);
}

LOG.info("Inserting data to generate WAL entries");
try (Connection conn = ConnectionFactory.createConnection(conf1)) {
insertIntoTable(conn, table1, famName, 2, 100);
}

int rsToStop = 0;
HRegionServer rsBeforeStop = cluster.getRegionServer(rsToStop);
String offlineHost =
rsBeforeStop.getServerName().getHostname() + ":" + rsBeforeStop.getServerName().getPort();
LOG.info("Stopping RS: {}", offlineHost);

cluster.stopRegionServer(rsToStop);
// Wait for WALs to be moved to oldlogs
Thread.sleep(5000);

LOG.info("Taking full backup (with offline RS WALs in oldlogs)");
String fullBackupId = fullTableBackup(tables);
assertTrue("Full backup should succeed", checkSucceeded(fullBackupId));

try (BackupSystemTable sysTable = new BackupSystemTable(TEST_UTIL.getConnection())) {
Map<TableName, Map<String, Long>> timestamps = sysTable.readLogTimestampMap(BACKUP_ROOT_DIR);
Map<String, Long> rsTimestamps = timestamps.get(table1);
LOG.info("RS timestamps after full backup: {}", rsTimestamps);

Long tsAfterFullBackup = rsTimestamps.get(offlineHost);
assertNotNull("Offline host should have timestamp recorded in trslm after full backup",
tsAfterFullBackup);

LOG.info("Taking incremental backup (should NOT include offline RS WALs)");
String incrBackupId = incrementalTableBackup(tables);
assertTrue("Incremental backup should succeed", checkSucceeded(incrBackupId));

timestamps = sysTable.readLogTimestampMap(BACKUP_ROOT_DIR);
rsTimestamps = timestamps.get(table1);
assertFalse("Offline host should not have a boundary ",
rsTimestamps.containsKey(offlineHost));
}
}
}