Skip to content

HBASE-29296 Missing critical snapshot expiration checks #6970

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 6 commits into
base: master
Choose a base branch
from
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
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,9 @@
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.snapshot.SnapshotRegionLocator;
import org.apache.hadoop.hbase.snapshot.SnapshotTTLExpiredException;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.util.Tool;
Expand All @@ -62,6 +64,7 @@

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

import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;

/**
Expand Down Expand Up @@ -539,6 +542,13 @@ private void verifyCfCompatibility(Set<TableName> tables,
SnapshotDescriptionUtils.readSnapshotInfo(fs, manifestDir);
SnapshotManifest manifest =
SnapshotManifest.open(conf, fs, manifestDir, snapshotDescription);
if (
SnapshotDescriptionUtils.isExpiredSnapshot(snapshotDescription.getTtl(),
snapshotDescription.getCreationTime(), EnvironmentEdgeManager.currentTime())
) {
throw new SnapshotTTLExpiredException(
ProtobufUtil.createSnapshotDesc(snapshotDescription));
}

ColumnFamilyDescriptor[] backupCfs = manifest.getTableDescriptor().getColumnFamilies();
if (!areCfsCompatible(currentCfs, backupCfs)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.snapshot.SnapshotTTLExpiredException;
import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
Expand All @@ -54,6 +55,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;

/**
Expand Down Expand Up @@ -265,6 +267,12 @@ TableDescriptor getTableDesc(TableName tableName) throws IOException {
Path tableInfoPath = this.getTableInfoPath(tableName);
SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, tableInfoPath);
SnapshotManifest manifest = SnapshotManifest.open(conf, fs, tableInfoPath, desc);
if (
SnapshotDescriptionUtils.isExpiredSnapshot(desc.getTtl(), desc.getCreationTime(),
EnvironmentEdgeManager.currentTime())
) {
throw new SnapshotTTLExpiredException(ProtobufUtil.createSnapshotDesc(desc));
}
TableDescriptor tableDescriptor = manifest.getTableDescriptor();
if (!tableDescriptor.getTableName().equals(tableName)) {
LOG.error("couldn't find Table Desc for table: " + tableName + " under tableInfoPath: "
Expand Down Expand Up @@ -310,6 +318,12 @@ private void createAndRestoreTable(Connection conn, TableName tableName, TableNa
SnapshotDescription desc =
SnapshotDescriptionUtils.readSnapshotInfo(fileSys, tableSnapshotPath);
SnapshotManifest manifest = SnapshotManifest.open(conf, fileSys, tableSnapshotPath, desc);
if (
SnapshotDescriptionUtils.isExpiredSnapshot(desc.getTtl(), desc.getCreationTime(),
EnvironmentEdgeManager.currentTime())
) {
throw new SnapshotTTLExpiredException(ProtobufUtil.createSnapshotDesc(desc));
}
tableDescriptor = manifest.getTableDescriptor();
} else {
tableDescriptor = getTableDesc(tableName);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,238 @@
/*
* 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.assertNotEquals;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;

import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
import org.apache.hadoop.hbase.backup.util.BackupUtils;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.LogRoller;
import org.apache.hadoop.hbase.snapshot.SnapshotTTLExpiredException;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdge;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;

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

@Category(LargeTests.class)
public class TestBackupRestoreExpiry extends TestBackupBase {

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

@BeforeClass
public static void setUp() throws Exception {
TEST_UTIL = new HBaseTestingUtil();
conf1 = TEST_UTIL.getConfiguration();
conf1.setLong(HConstants.DEFAULT_SNAPSHOT_TTL_CONFIG_KEY, 30);
autoRestoreOnFailure = true;
useSecondCluster = false;
setUpHelper();
}

public void ensurePreviousBackupTestsAreCleanedUp() throws Exception {
TEST_UTIL.flush(table1);
TEST_UTIL.flush(table2);

TEST_UTIL.truncateTable(table1).close();
TEST_UTIL.truncateTable(table2).close();

if (TEST_UTIL.getAdmin().tableExists(table1_restore)) {
TEST_UTIL.flush(table1_restore);
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) {
}
});

try (Table table = TEST_UTIL.getConnection().getTable(table1)) {
loadTable(table);
}

try (Table table = TEST_UTIL.getConnection().getTable(table2)) {
loadTable(table);
}
}

@Test
public void TestSequentially() throws Exception {
try {
TestRestoreOnExpiredFullBackup();
} catch (Exception e) {
throw e;
} finally {
ensurePreviousBackupTestsAreCleanedUp();
}

try {
TestIncrementalBackupOnExpiredFullBackup();
} catch (Exception e) {
throw e;
} finally {
ensurePreviousBackupTestsAreCleanedUp();
}
}

public void TestRestoreOnExpiredFullBackup() throws Exception {
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);

Connection conn = TEST_UTIL.getConnection();
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 };

EnvironmentEdgeManager.injectEdge(new EnvironmentEdge() {
// time + 30s
@Override
public long currentTime() {
return System.currentTimeMillis() + (30 * 1000);
}
});

assertThrows(SnapshotTTLExpiredException.class, () -> {
backupAdmin.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, fullBackupId, false,
fromTables, toTables, true, true));
});

EnvironmentEdgeManager.reset();
backupAdmin.close();
}

public void TestIncrementalBackupOnExpiredFullBackup() throws Exception {
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);

Connection conn = TEST_UTIL.getConnection();
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 };

List<LocatedFileStatus> preRestoreBackupFiles = getBackupFiles();
backupAdmin.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, fullBackupId, false,
fromTables, toTables, true, true));
List<LocatedFileStatus> postRestoreBackupFiles = getBackupFiles();

// Check that the backup files are the same before and after the restore process
Assert.assertEquals(postRestoreBackupFiles, preRestoreBackupFiles);
Assert.assertEquals(TEST_UTIL.countRows(table1_restore), NB_ROWS_IN_BATCH);

int ROWS_TO_ADD = 1_000;
// different IDs so that rows don't overlap
insertIntoTable(conn, table1, famName, 3, ROWS_TO_ADD);
insertIntoTable(conn, table1, mobFam, 4, ROWS_TO_ADD);

try (Admin admin = conn.getAdmin()) {
List<HRegion> currentRegions = TEST_UTIL.getHBaseCluster().getRegions(table1);
for (HRegion region : currentRegions) {
byte[] name = region.getRegionInfo().getEncodedNameAsBytes();
admin.splitRegionAsync(name).get();
}

TEST_UTIL.waitTableAvailable(table1);

// Make sure we've split regions
assertNotEquals(currentRegions, TEST_UTIL.getHBaseCluster().getRegions(table1));

EnvironmentEdgeManager.injectEdge(new EnvironmentEdge() {
// time + 30s
@Override
public long currentTime() {
return System.currentTimeMillis() + (30 * 1000);
}
});

try {
backupAdmin
.backupTables(createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR));
fail("Should not reach here");
} catch (Exception e) {
if (e instanceof SnapshotTTLExpiredException) {
throw (SnapshotTTLExpiredException) e;
}
} finally {
EnvironmentEdgeManager.reset();
backupAdmin.close();
}
}
}

private List<LocatedFileStatus> getBackupFiles() throws IOException {
FileSystem fs = TEST_UTIL.getTestFileSystem();
RemoteIterator<LocatedFileStatus> iter = fs.listFiles(new Path(BACKUP_ROOT_DIR), true);
List<LocatedFileStatus> files = new ArrayList<>();

while (iter.hasNext()) {
files.add(iter.next());
}

return files;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,9 @@
import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.snapshot.SnapshotTTLExpiredException;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.yetus.audience.InterfaceAudience;
Expand Down Expand Up @@ -184,6 +186,12 @@ protected Flow executeFromState(MasterProcedureEnv env, SnapshotState state)
if (isSnapshotCorrupted()) {
throw new CorruptedSnapshotException(snapshot.getName());
}
if (
SnapshotDescriptionUtils.isExpiredSnapshot(snapshot.getTtl(),
snapshot.getCreationTime(), EnvironmentEdgeManager.currentTime())
) {
throw new SnapshotTTLExpiredException(ProtobufUtil.createSnapshotDesc(snapshot));
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks for the contribution!
For case 3:
I think we don't need to perform checks now, because even if the snapshot has expired, as long as we perform proper checks during the restore process, it should be fine. Let's hear what @Apache9 thinks about it.

If we go with your approach, i think we should also add the same validation logic in SnapshotManager.java, since HBase also supports creating snapshots without using snapshot procedure, right?

Copy link
Author

Choose a reason for hiding this comment

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

Thank you for the review!

I agree that it might be fine because we perform checks during the restore process. I just thought that redundant check would make it more consistent. Let's hear what @Apache9 thinks of it.

If we go with your approach, i think we should also add the same validation logic in SnapshotManager.java, since HBase also supports creating snapshots without using snapshot procedure, right?

Oh that's right, I missed that one. Thanks for noticing it, I'll put the check if we agree that preemptive checking is needed

completeSnapshot(env);
setNextState(SnapshotState.SNAPSHOT_POST_OPERATION);
return Flow.HAS_MORE_STATE;
Expand Down
Loading