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
@@ -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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,11 @@

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;
Expand Down Expand Up @@ -72,22 +72,14 @@ protected void chore() {
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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,9 @@
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;
Expand Down Expand Up @@ -331,6 +333,14 @@ private void prepareClone(final MasterProcedureEnv env) throws IOException {
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));
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,8 @@
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;
Expand Down Expand Up @@ -327,6 +329,14 @@ private void prepareRestore(final MasterProcedureEnv env) throws IOException {
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);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -455,4 +455,18 @@ public ListMultimap<String, UserPermission> run() throws Exception {
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;
}
}
Original file line number Diff line number Diff line change
@@ -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());
}
}
Loading