HBASE-27671 Client should not be able to restore/clone a snapshot after it has TTL expired it's TTL has expired (#5118)
This commit is contained in:
parent
5b50d70e40
commit
7fa8f4d309
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* 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.snapshot;
|
||||
|
||||
import org.apache.hadoop.hbase.client.SnapshotDescription;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Thrown when a snapshot could not be restored/cloned because the ttl for snapshot has already
|
||||
* expired
|
||||
*/
|
||||
@SuppressWarnings("serial")
|
||||
@InterfaceAudience.Public
|
||||
public class SnapshotTTLExpiredException extends HBaseSnapshotException {
|
||||
/**
|
||||
* Failure when the ttl for snapshot has already expired.
|
||||
* @param message the full description of the failure
|
||||
*/
|
||||
public SnapshotTTLExpiredException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
/**
|
||||
* Failure when the ttl for snapshot has already expired.
|
||||
* @param snapshotDescription snapshot that was attempted
|
||||
*/
|
||||
public SnapshotTTLExpiredException(SnapshotDescription snapshotDescription) {
|
||||
super("TTL for snapshot '" + snapshotDescription.getName() + "' has already expired.",
|
||||
snapshotDescription);
|
||||
}
|
||||
}
|
|
@ -19,11 +19,11 @@ package org.apache.hadoop.hbase.master.cleaner;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.ScheduledChore;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -72,22 +72,14 @@ public class SnapshotCleanerChore extends ScheduledChore {
|
|||
for (SnapshotProtos.SnapshotDescription snapshotDescription : completedSnapshotsList) {
|
||||
long snapshotCreatedTime = snapshotDescription.getCreationTime();
|
||||
long snapshotTtl = snapshotDescription.getTtl();
|
||||
/*
|
||||
* Backward compatibility after the patch deployment on HMaster Any snapshot with ttl 0 is
|
||||
* to be considered as snapshot to keep FOREVER Default ttl value specified by
|
||||
* {@HConstants.DEFAULT_SNAPSHOT_TTL}
|
||||
*/
|
||||
long currentTime = EnvironmentEdgeManager.currentTime();
|
||||
if (
|
||||
snapshotCreatedTime > 0 && snapshotTtl > 0
|
||||
&& snapshotTtl < TimeUnit.MILLISECONDS.toSeconds(Long.MAX_VALUE)
|
||||
SnapshotDescriptionUtils.isExpiredSnapshot(snapshotTtl, snapshotCreatedTime, currentTime)
|
||||
) {
|
||||
long currentTime = EnvironmentEdgeManager.currentTime();
|
||||
if ((snapshotCreatedTime + TimeUnit.SECONDS.toMillis(snapshotTtl)) < currentTime) {
|
||||
LOG.info("Event: {} Name: {}, CreatedTime: {}, TTL: {}, currentTime: {}",
|
||||
DELETE_SNAPSHOT_EVENT, snapshotDescription.getName(), snapshotCreatedTime,
|
||||
snapshotTtl, currentTime);
|
||||
deleteExpiredSnapshot(snapshotDescription);
|
||||
}
|
||||
LOG.info("Event: {} Name: {}, CreatedTime: {}, TTL: {}, currentTime: {}",
|
||||
DELETE_SNAPSHOT_EVENT, snapshotDescription.getName(), snapshotCreatedTime, snapshotTtl,
|
||||
currentTime);
|
||||
deleteExpiredSnapshot(snapshotDescription);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -49,7 +49,9 @@ import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
|
|||
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
|
||||
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.FSTableDescriptors;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -331,6 +333,14 @@ public class CloneSnapshotProcedure extends AbstractStateMachineTableProcedure<C
|
|||
if (env.getMasterServices().getTableDescriptors().exists(tableName)) {
|
||||
throw new TableExistsException(tableName);
|
||||
}
|
||||
|
||||
// check whether ttl has expired for this snapshot
|
||||
if (
|
||||
SnapshotDescriptionUtils.isExpiredSnapshot(snapshot.getTtl(), snapshot.getCreationTime(),
|
||||
EnvironmentEdgeManager.currentTime())
|
||||
) {
|
||||
throw new SnapshotTTLExpiredException(ProtobufUtil.createSnapshotDesc(snapshot));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -49,6 +49,8 @@ import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
|||
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
|
||||
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.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -327,6 +329,14 @@ public class RestoreSnapshotProcedure
|
|||
throw new TableNotFoundException(tableName);
|
||||
}
|
||||
|
||||
// check whether ttl has expired for this snapshot
|
||||
if (
|
||||
SnapshotDescriptionUtils.isExpiredSnapshot(snapshot.getTtl(), snapshot.getCreationTime(),
|
||||
EnvironmentEdgeManager.currentTime())
|
||||
) {
|
||||
throw new SnapshotTTLExpiredException(ProtobufUtil.createSnapshotDesc(snapshot));
|
||||
}
|
||||
|
||||
// Check whether table is disabled.
|
||||
env.getMasterServices().checkTableModifiable(tableName);
|
||||
|
||||
|
|
|
@ -455,4 +455,18 @@ public final class SnapshotDescriptionUtils {
|
|||
return snapshot.toBuilder()
|
||||
.setUsersAndPermissions(ShadedAccessControlUtil.toUserTablePermissions(perms)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Method to check whether TTL has expired for specified snapshot creation time and snapshot ttl.
|
||||
* NOTE: For backward compatibility (after the patch deployment on HMaster), any snapshot with ttl
|
||||
* 0 is to be considered as snapshot to keep FOREVER. Default ttl value specified by
|
||||
* {@link HConstants#DEFAULT_SNAPSHOT_TTL}
|
||||
* @return true if ttl has expired, or, false, otherwise
|
||||
*/
|
||||
public static boolean isExpiredSnapshot(long snapshotTtl, long snapshotCreatedTime,
|
||||
long currentTime) {
|
||||
return snapshotCreatedTime > 0 && snapshotTtl > HConstants.DEFAULT_SNAPSHOT_TTL
|
||||
&& snapshotTtl < TimeUnit.MILLISECONDS.toSeconds(Long.MAX_VALUE)
|
||||
&& (snapshotCreatedTime + TimeUnit.SECONDS.toMillis(snapshotTtl)) < currentTime;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,240 @@
|
|||
/*
|
||||
* 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.client;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotTTLExpiredException;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
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;
|
||||
|
||||
/**
|
||||
* Test restore/clone snapshots with TTL from the client
|
||||
*/
|
||||
@Category({ LargeTests.class, ClientTests.class })
|
||||
public class TestSnapshotWithTTLFromClient {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestSnapshotWithTTLFromClient.class);
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotWithTTLFromClient.class);
|
||||
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
private static final int NUM_RS = 2;
|
||||
private static final String STRING_TABLE_NAME = "test";
|
||||
private static final byte[] TEST_FAM = Bytes.toBytes("fam");
|
||||
private static final TableName TABLE_NAME = TableName.valueOf(STRING_TABLE_NAME);
|
||||
private static final TableName CLONED_TABLE_NAME = TableName.valueOf("clonedTable");
|
||||
private static final String TTL_KEY = "TTL";
|
||||
private static final int CHORE_INTERVAL_SECS = 30;
|
||||
|
||||
/**
|
||||
* Setup the config for the cluster
|
||||
* @throws Exception on failure
|
||||
*/
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
setupConf(UTIL.getConfiguration());
|
||||
UTIL.startMiniCluster(NUM_RS);
|
||||
}
|
||||
|
||||
protected static void setupConf(Configuration conf) {
|
||||
// Enable snapshot
|
||||
conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
|
||||
|
||||
// Set this to high value so that cleaner chore is not triggered
|
||||
conf.setInt("hbase.master.cleaner.snapshot.interval", CHORE_INTERVAL_SECS * 60 * 1000);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
createTable();
|
||||
}
|
||||
|
||||
protected void createTable() throws Exception {
|
||||
UTIL.createTable(TABLE_NAME, new byte[][] { TEST_FAM });
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
UTIL.deleteTableIfAny(TABLE_NAME);
|
||||
UTIL.deleteTableIfAny(CLONED_TABLE_NAME);
|
||||
SnapshotTestingUtils.deleteAllSnapshots(UTIL.getAdmin());
|
||||
SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanupTest() throws Exception {
|
||||
try {
|
||||
UTIL.shutdownMiniCluster();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("failure shutting down cluster", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRestoreSnapshotWithTTLSuccess() throws Exception {
|
||||
String snapshotName = "nonExpiredTTLRestoreSnapshotTest";
|
||||
|
||||
// table should exist
|
||||
assertTrue(UTIL.getAdmin().tableExists(TABLE_NAME));
|
||||
|
||||
// create snapshot fo given table with specified ttl
|
||||
createSnapshotWithTTL(TABLE_NAME, snapshotName, CHORE_INTERVAL_SECS * 2);
|
||||
Admin admin = UTIL.getAdmin();
|
||||
|
||||
// Disable and drop table
|
||||
admin.disableTable(TABLE_NAME);
|
||||
admin.deleteTable(TABLE_NAME);
|
||||
assertFalse(UTIL.getAdmin().tableExists(TABLE_NAME));
|
||||
|
||||
// restore snapshot
|
||||
admin.restoreSnapshot(snapshotName);
|
||||
|
||||
// table should be created
|
||||
assertTrue(UTIL.getAdmin().tableExists(TABLE_NAME));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRestoreSnapshotFailsDueToTTLExpired() throws Exception {
|
||||
String snapshotName = "expiredTTLRestoreSnapshotTest";
|
||||
|
||||
// table should exist
|
||||
assertTrue(UTIL.getAdmin().tableExists(TABLE_NAME));
|
||||
|
||||
// create snapshot fo given table with specified ttl
|
||||
createSnapshotWithTTL(TABLE_NAME, snapshotName, 1);
|
||||
Admin admin = UTIL.getAdmin();
|
||||
|
||||
// Disable and drop table
|
||||
admin.disableTable(TABLE_NAME);
|
||||
admin.deleteTable(TABLE_NAME);
|
||||
assertFalse(UTIL.getAdmin().tableExists(TABLE_NAME));
|
||||
|
||||
// Sleep so that TTL may expire
|
||||
Threads.sleep(2000);
|
||||
|
||||
// restore snapshot which has expired
|
||||
try {
|
||||
admin.restoreSnapshot(snapshotName);
|
||||
fail("Restore snapshot succeeded even though TTL has expired.");
|
||||
} catch (SnapshotTTLExpiredException e) {
|
||||
LOG.info("Correctly failed to restore a TTL expired snapshot table:" + e.getMessage());
|
||||
}
|
||||
|
||||
// table should not be created
|
||||
assertFalse(UTIL.getAdmin().tableExists(TABLE_NAME));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCloneSnapshotWithTTLSuccess() throws Exception {
|
||||
String snapshotName = "nonExpiredTTLCloneSnapshotTest";
|
||||
|
||||
// table should exist
|
||||
assertTrue(UTIL.getAdmin().tableExists(TABLE_NAME));
|
||||
|
||||
// create snapshot fo given table with specified ttl
|
||||
createSnapshotWithTTL(TABLE_NAME, snapshotName, CHORE_INTERVAL_SECS * 2);
|
||||
Admin admin = UTIL.getAdmin();
|
||||
|
||||
// restore snapshot
|
||||
admin.cloneSnapshot(snapshotName, CLONED_TABLE_NAME);
|
||||
|
||||
// table should be created
|
||||
assertTrue(UTIL.getAdmin().tableExists(CLONED_TABLE_NAME));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCloneSnapshotFailsDueToTTLExpired() throws Exception {
|
||||
String snapshotName = "expiredTTLCloneSnapshotTest";
|
||||
|
||||
// table should exist
|
||||
assertTrue(UTIL.getAdmin().tableExists(TABLE_NAME));
|
||||
|
||||
// create snapshot fo given table with specified ttl
|
||||
createSnapshotWithTTL(TABLE_NAME, snapshotName, 1);
|
||||
Admin admin = UTIL.getAdmin();
|
||||
|
||||
assertTrue(UTIL.getAdmin().tableExists(TABLE_NAME));
|
||||
|
||||
// Sleep so that TTL may expire
|
||||
Threads.sleep(2000);
|
||||
|
||||
// clone snapshot which has expired
|
||||
try {
|
||||
admin.cloneSnapshot(snapshotName, CLONED_TABLE_NAME);
|
||||
fail("Clone snapshot succeeded even though TTL has expired.");
|
||||
} catch (SnapshotTTLExpiredException e) {
|
||||
LOG.info("Correctly failed to clone a TTL expired snapshot table:" + e.getMessage());
|
||||
}
|
||||
|
||||
// table should not be created
|
||||
assertFalse(UTIL.getAdmin().tableExists(CLONED_TABLE_NAME));
|
||||
}
|
||||
|
||||
private void createSnapshotWithTTL(TableName tableName, final String snapshotName,
|
||||
final int snapshotTTL) throws IOException {
|
||||
Admin admin = UTIL.getAdmin();
|
||||
|
||||
// make sure we don't fail on listing snapshots
|
||||
SnapshotTestingUtils.assertNoSnapshots(admin);
|
||||
|
||||
// put some stuff in the table
|
||||
Table table = UTIL.getConnection().getTable(tableName);
|
||||
UTIL.loadTable(table, TEST_FAM);
|
||||
|
||||
Map<String, Object> props = new HashMap<>();
|
||||
props.put(TTL_KEY, snapshotTTL);
|
||||
|
||||
// take a snapshot of the table
|
||||
SnapshotTestingUtils.snapshot(UTIL.getAdmin(), snapshotName, tableName, SnapshotType.FLUSH, 3,
|
||||
props);
|
||||
LOG.debug("Snapshot completed.");
|
||||
|
||||
// make sure we have the snapshot with expectd TTL
|
||||
List<SnapshotDescription> snapshots =
|
||||
SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, snapshotName, tableName);
|
||||
assertEquals(1, snapshots.size());
|
||||
assertEquals(snapshotTTL, snapshots.get(0).getTtl());
|
||||
}
|
||||
}
|
|
@ -297,11 +297,21 @@ public final class SnapshotTestingUtils {
|
|||
*/
|
||||
public static void snapshot(Admin admin, final String snapshotName, final TableName tableName,
|
||||
final SnapshotType type, final int numTries) throws IOException {
|
||||
snapshot(admin, snapshotName, tableName, type, numTries, null);
|
||||
}
|
||||
|
||||
/*
|
||||
* Take snapshot having snapshot properties with maximum of numTries attempts, ignoring
|
||||
* CorruptedSnapshotException except for the last CorruptedSnapshotException
|
||||
*/
|
||||
public static void snapshot(Admin admin, final String snapshotName, final TableName tableName,
|
||||
final SnapshotType type, final int numTries, Map<String, Object> snapshotProps)
|
||||
throws IOException {
|
||||
int tries = 0;
|
||||
CorruptedSnapshotException lastEx = null;
|
||||
while (tries++ < numTries) {
|
||||
try {
|
||||
admin.snapshot(snapshotName, tableName, type);
|
||||
admin.snapshot(snapshotName, tableName, type, snapshotProps);
|
||||
return;
|
||||
} catch (CorruptedSnapshotException cse) {
|
||||
LOG.warn("Got CorruptedSnapshotException", cse);
|
||||
|
|
Loading…
Reference in New Issue