diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java index 39bc832b0fa..64a03230c13 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.master.RegionPlan; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import java.io.IOException; import java.util.List; @@ -287,4 +288,50 @@ public class BaseMasterObserver implements MasterObserver { HRegionInfo region, ServerName srcServer, ServerName destServer) throws IOException { } + + @Override + public void preSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + } + + @Override + public void postSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + } + + @Override + public void preCloneSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + } + + @Override + public void postCloneSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + } + + @Override + public void preRestoreSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + } + + @Override + public void postRestoreSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + } + + @Override + public void preDeleteSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot) throws IOException { + } + + @Override + public void postDeleteSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot) throws IOException { + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java index 7dbe1a8f4e6..186a2c70ec8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.master.RegionPlan; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; /** * Defines coprocessor hooks for interacting with operations on the @@ -496,4 +497,100 @@ public interface MasterObserver extends Coprocessor { */ void postStartMaster(final ObserverContext ctx) throws IOException; + + /** + * Called before a new snapshot is taken. + * Called as part of snapshot RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param snapshot the SnapshotDescriptor for the snapshot + * @param hTableDescriptor the hTableDescriptor of the table to snapshot + * @throws IOException + */ + void preSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException; + + /** + * Called after the snapshot operation has been requested. + * Called as part of snapshot RPC call. + * @param ctx the environment to interact with the framework and master + * @param snapshot the SnapshotDescriptor for the snapshot + * @param hTableDescriptor the hTableDescriptor of the table to snapshot + * @throws IOException + */ + void postSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException; + + /** + * Called before a snapshot is cloned. + * Called as part of restoreSnapshot RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param snapshot the SnapshotDescriptor for the snapshot + * @param hTableDescriptor the hTableDescriptor of the table to create + * @throws IOException + */ + void preCloneSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException; + + /** + * Called after a snapshot clone operation has been requested. + * Called as part of restoreSnapshot RPC call. + * @param ctx the environment to interact with the framework and master + * @param snapshot the SnapshotDescriptor for the snapshot + * @param hTableDescriptor the hTableDescriptor of the table to create + * @throws IOException + */ + void postCloneSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException; + + /** + * Called before a snapshot is restored. + * Called as part of restoreSnapshot RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param snapshot the SnapshotDescriptor for the snapshot + * @param hTableDescriptor the hTableDescriptor of the table to restore + * @throws IOException + */ + void preRestoreSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException; + + /** + * Called after a snapshot restore operation has been requested. + * Called as part of restoreSnapshot RPC call. + * @param ctx the environment to interact with the framework and master + * @param snapshot the SnapshotDescriptor for the snapshot + * @param hTableDescriptor the hTableDescriptor of the table to restore + * @throws IOException + */ + void postRestoreSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException; + + /** + * Called before a snapshot is deleted. + * Called as part of deleteSnapshot RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param snapshot the SnapshotDescriptor of the snapshot to delete + * @throws IOException + */ + void preDeleteSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot) throws IOException; + + /** + * Called after the delete snapshot operation has been requested. + * Called as part of deleteSnapshot RPC call. + * @param ctx the environment to interact with the framework and master + * @param snapshot the SnapshotDescriptor of the snapshot to delete + * @throws IOException + */ + void postDeleteSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot) throws IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 7c0cbeb3714..c9b5539d9a8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -2036,6 +2036,7 @@ Server { return zooKeeper; } + @Override public MasterCoprocessorHost getCoprocessorHost() { return cpHost; } @@ -2483,6 +2484,14 @@ Server { snapshot = snapshot.toBuilder().setVersion(SnapshotDescriptionUtils.SNAPSHOT_LAYOUT_VERSION) .build(); + try { + if (cpHost != null) { + cpHost.preSnapshot(snapshot, desc); + } + } catch (IOException e) { + throw new ServiceException(e); + } + // if the table is enabled, then have the RS run actually the snapshot work if (this.assignmentManager.getZKTable().isEnabledTable(snapshot.getTable())) { LOG.debug("Table enabled, starting distributed snapshot."); @@ -2506,6 +2515,15 @@ Server { "Table is not entirely open or closed", new TablePartiallyOpenException( snapshot.getTable() + " isn't fully open."), snapshot)); } + + try { + if (cpHost != null) { + cpHost.postSnapshot(snapshot, desc); + } + } catch (IOException e) { + throw new ServiceException(e); + } + // send back the max amount of time the client should wait for the snapshot to complete long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(), SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME); @@ -2566,6 +2584,10 @@ Server { public DeleteSnapshotResponse deleteSnapshot(RpcController controller, DeleteSnapshotRequest request) throws ServiceException { try { + if (cpHost != null) { + cpHost.preDeleteSnapshot(request.getSnapshot()); + } + // check to see if it is completed if (!isSnapshotCompleted(request.getSnapshot())) { throw new SnapshotDoesNotExistException(request.getSnapshot()); @@ -2581,6 +2603,11 @@ Server { if (!this.getMasterFileSystem().getFileSystem().delete(snapshotDir, true)) { throw new ServiceException("Failed to delete snapshot directory: " + snapshotDir); } + + if (cpHost != null) { + cpHost.postDeleteSnapshot(request.getSnapshot()); + } + return DeleteSnapshotResponse.newBuilder().build(); } catch (IOException e) { throw new ServiceException(e); @@ -2701,13 +2728,30 @@ Server { snapshot.getTable() + "' must be disabled in order to perform a restore operation.")); } + if (cpHost != null) { + cpHost.preRestoreSnapshot(snapshot, snapshotTableDesc); + } + snapshotManager.restoreSnapshot(snapshot, snapshotTableDesc); LOG.info("Restore snapshot=" + snapshot.getName() + " as table=" + tableName); + + if (cpHost != null) { + cpHost.postRestoreSnapshot(snapshot, snapshotTableDesc); + } } else { HTableDescriptor htd = RestoreSnapshotHelper.cloneTableSchema(snapshotTableDesc, Bytes.toBytes(tableName)); + + if (cpHost != null) { + cpHost.preCloneSnapshot(snapshot, htd); + } + snapshotManager.cloneSnapshot(snapshot, htd); LOG.info("Clone snapshot=" + snapshot.getName() + " as table=" + tableName); + + if (cpHost != null) { + cpHost.postCloneSnapshot(snapshot, htd); + } } return RestoreSnapshotResponse.newBuilder().build(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java index de93998310b..0ecb2f465d5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java @@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.coprocessor.*; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import java.io.IOException; import java.util.List; @@ -159,6 +160,7 @@ public class MasterCoprocessorHost } } } + public void preDeleteTable(byte[] tableName) throws IOException { ObserverContext ctx = null; for (MasterEnvironment env: coprocessors) { @@ -934,4 +936,146 @@ public class MasterCoprocessorHost } } } + + public void preSnapshot(final SnapshotDescription snapshot, + final HTableDescriptor hTableDescriptor) throws IOException { + ObserverContext ctx = null; + for (MasterEnvironment env: coprocessors) { + if (env.getInstance() instanceof MasterObserver) { + ctx = ObserverContext.createAndPrepare(env, ctx); + try { + ((MasterObserver)env.getInstance()).preSnapshot(ctx, snapshot, hTableDescriptor); + } catch (Throwable e) { + handleCoprocessorThrowable(env, e); + } + if (ctx.shouldComplete()) { + break; + } + } + } + } + + public void postSnapshot(final SnapshotDescription snapshot, + final HTableDescriptor hTableDescriptor) throws IOException { + ObserverContext ctx = null; + for (MasterEnvironment env: coprocessors) { + if (env.getInstance() instanceof MasterObserver) { + ctx = ObserverContext.createAndPrepare(env, ctx); + try { + ((MasterObserver)env.getInstance()).postSnapshot(ctx, snapshot, hTableDescriptor); + } catch (Throwable e) { + handleCoprocessorThrowable(env, e); + } + if (ctx.shouldComplete()) { + break; + } + } + } + } + + public void preCloneSnapshot(final SnapshotDescription snapshot, + final HTableDescriptor hTableDescriptor) throws IOException { + ObserverContext ctx = null; + for (MasterEnvironment env: coprocessors) { + if (env.getInstance() instanceof MasterObserver) { + ctx = ObserverContext.createAndPrepare(env, ctx); + try { + ((MasterObserver)env.getInstance()).preCloneSnapshot(ctx, snapshot, hTableDescriptor); + } catch (Throwable e) { + handleCoprocessorThrowable(env, e); + } + if (ctx.shouldComplete()) { + break; + } + } + } + } + + public void postCloneSnapshot(final SnapshotDescription snapshot, + final HTableDescriptor hTableDescriptor) throws IOException { + ObserverContext ctx = null; + for (MasterEnvironment env: coprocessors) { + if (env.getInstance() instanceof MasterObserver) { + ctx = ObserverContext.createAndPrepare(env, ctx); + try { + ((MasterObserver)env.getInstance()).postCloneSnapshot(ctx, snapshot, hTableDescriptor); + } catch (Throwable e) { + handleCoprocessorThrowable(env, e); + } + if (ctx.shouldComplete()) { + break; + } + } + } + } + + public void preRestoreSnapshot(final SnapshotDescription snapshot, + final HTableDescriptor hTableDescriptor) throws IOException { + ObserverContext ctx = null; + for (MasterEnvironment env: coprocessors) { + if (env.getInstance() instanceof MasterObserver) { + ctx = ObserverContext.createAndPrepare(env, ctx); + try { + ((MasterObserver)env.getInstance()).preRestoreSnapshot(ctx, snapshot, hTableDescriptor); + } catch (Throwable e) { + handleCoprocessorThrowable(env, e); + } + if (ctx.shouldComplete()) { + break; + } + } + } + } + + public void postRestoreSnapshot(final SnapshotDescription snapshot, + final HTableDescriptor hTableDescriptor) throws IOException { + ObserverContext ctx = null; + for (MasterEnvironment env: coprocessors) { + if (env.getInstance() instanceof MasterObserver) { + ctx = ObserverContext.createAndPrepare(env, ctx); + try { + ((MasterObserver)env.getInstance()).postRestoreSnapshot(ctx, snapshot, hTableDescriptor); + } catch (Throwable e) { + handleCoprocessorThrowable(env, e); + } + if (ctx.shouldComplete()) { + break; + } + } + } + } + + public void preDeleteSnapshot(final SnapshotDescription snapshot) throws IOException { + ObserverContext ctx = null; + for (MasterEnvironment env: coprocessors) { + if (env.getInstance() instanceof MasterObserver) { + ctx = ObserverContext.createAndPrepare(env, ctx); + try { + ((MasterObserver)env.getInstance()).preDeleteSnapshot(ctx, snapshot); + } catch (Throwable e) { + handleCoprocessorThrowable(env, e); + } + if (ctx.shouldComplete()) { + break; + } + } + } + } + + public void postDeleteSnapshot(final SnapshotDescription snapshot) throws IOException { + ObserverContext ctx = null; + for (MasterEnvironment env: coprocessors) { + if (env.getInstance() instanceof MasterObserver) { + ctx = ObserverContext.createAndPrepare(env, ctx); + try { + ((MasterObserver)env.getInstance()).postDeleteSnapshot(ctx, snapshot); + } catch (Throwable e) { + handleCoprocessorThrowable(env, e); + } + if (ctx.shouldComplete()) { + break; + } + } + } + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java index 51d9bafff5a..480cf7845b1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java @@ -54,6 +54,11 @@ public interface MasterServices extends Server { */ public ExecutorService getExecutorService(); + /** + * @return Master's instance of {@link MasterCoprocessorHost} + */ + public MasterCoprocessorHost getCoprocessorHost(); + /** * Check table is modifiable; i.e. exists and is offline. * @param tableName Name of table to check. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index f795e755606..1fdbcd96703 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.RegionScanner; @@ -775,6 +776,55 @@ public class AccessController extends BaseRegionObserver AccessControlLists.init(ctx.getEnvironment().getMasterServices()); } + @Override + public void preSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + requirePermission(Permission.Action.ADMIN); + } + + @Override + public void postSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + } + + @Override + public void preCloneSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + requirePermission(Permission.Action.ADMIN); + } + + @Override + public void postCloneSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + } + + @Override + public void preRestoreSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + requirePermission(Permission.Action.ADMIN); + } + + @Override + public void postRestoreSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + } + + @Override + public void preDeleteSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot) throws IOException { + requirePermission(Permission.Action.ADMIN); + } + + @Override + public void postDeleteSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot) throws IOException { + } /* ---- RegionObserver implementation ---- */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java index a39f1f03548..188fd6a8ccc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.master.RegionState; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -62,7 +63,7 @@ import org.junit.experimental.categories.Category; public class TestMasterObserver { private static final Log LOG = LogFactory.getLog(TestMasterObserver.class); - public static CountDownLatch countDown = new CountDownLatch(1); + public static CountDownLatch tableCreationLatch = new CountDownLatch(1); public static class CPMasterObserver implements MasterObserver { @@ -98,6 +99,14 @@ public class TestMasterObserver { private boolean postStartMasterCalled; private boolean startCalled; private boolean stopCalled; + private boolean preSnapshotCalled; + private boolean postSnapshotCalled; + private boolean preCloneSnapshotCalled; + private boolean postCloneSnapshotCalled; + private boolean preRestoreSnapshotCalled; + private boolean postRestoreSnapshotCalled; + private boolean preDeleteSnapshotCalled; + private boolean postDeleteSnapshotCalled; private boolean preCreateTableHandlerCalled; private boolean postCreateTableHandlerCalled; private boolean preDeleteTableHandlerCalled; @@ -115,7 +124,6 @@ public class TestMasterObserver { private boolean preModifyTableHandlerCalled; private boolean postModifyTableHandlerCalled; - public void enableBypass(boolean bypass) { this.bypass = bypass; } @@ -147,6 +155,14 @@ public class TestMasterObserver { postBalanceCalled = false; preBalanceSwitchCalled = false; postBalanceSwitchCalled = false; + preSnapshotCalled = false; + postSnapshotCalled = false; + preCloneSnapshotCalled = false; + postCloneSnapshotCalled = false; + preRestoreSnapshotCalled = false; + postRestoreSnapshotCalled = false; + preDeleteSnapshotCalled = false; + postDeleteSnapshotCalled = false; preCreateTableHandlerCalled = false; postCreateTableHandlerCalled = false; preDeleteTableHandlerCalled = false; @@ -508,6 +524,76 @@ public class TestMasterObserver { public boolean wasStopped() { return stopCalled; } + @Override + public void preSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + preSnapshotCalled = true; + } + + @Override + public void postSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + postSnapshotCalled = true; + } + + public boolean wasSnapshotCalled() { + return preSnapshotCalled && postSnapshotCalled; + } + + @Override + public void preCloneSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + preCloneSnapshotCalled = true; + } + + @Override + public void postCloneSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + postCloneSnapshotCalled = true; + } + + public boolean wasCloneSnapshotCalled() { + return preCloneSnapshotCalled && postCloneSnapshotCalled; + } + + @Override + public void preRestoreSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + preRestoreSnapshotCalled = true; + } + + @Override + public void postRestoreSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) + throws IOException { + postRestoreSnapshotCalled = true; + } + + public boolean wasRestoreSnapshotCalled() { + return preRestoreSnapshotCalled && postRestoreSnapshotCalled; + } + + @Override + public void preDeleteSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot) throws IOException { + preDeleteSnapshotCalled = true; + } + + @Override + public void postDeleteSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot) throws IOException { + postDeleteSnapshotCalled = true; + } + + public boolean wasDeleteSnapshotCalled() { + return preDeleteSnapshotCalled && postDeleteSnapshotCalled; + } + @Override public void preCreateTableHandler( ObserverContext env, @@ -523,7 +609,7 @@ public class TestMasterObserver { ObserverContext ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException { postCreateTableHandlerCalled = true; - countDown.countDown(); + tableCreationLatch.countDown(); } public boolean wasPreCreateTableHandlerCalled(){ @@ -711,7 +797,9 @@ public class TestMasterObserver { } private static HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private static byte[] TEST_SNAPSHOT = Bytes.toBytes("observed_snapshot"); private static byte[] TEST_TABLE = Bytes.toBytes("observed_table"); + private static byte[] TEST_CLONE = Bytes.toBytes("observed_clone"); private static byte[] TEST_FAMILY = Bytes.toBytes("fam1"); private static byte[] TEST_FAMILY2 = Bytes.toBytes("fam2"); private static byte[] TEST_FAMILY3 = Bytes.toBytes("fam3"); @@ -721,6 +809,11 @@ public class TestMasterObserver { Configuration conf = UTIL.getConfiguration(); conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, CPMasterObserver.class.getName()); + conf.set("hbase.master.hfilecleaner.plugins", + "org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner," + + "org.apache.hadoop.hbase.master.snapshot.SnapshotHFileCleaner"); + conf.set("hbase.master.logcleaner.plugins", + "org.apache.hadoop.hbase.master.snapshot.SnapshotLogCleaner"); // We need more than one data server on this test UTIL.startMiniCluster(2); } @@ -765,16 +858,17 @@ public class TestMasterObserver { htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); HBaseAdmin admin = UTIL.getHBaseAdmin(); + tableCreationLatch = new CountDownLatch(1); admin.createTable(htd); // preCreateTable can't bypass default action. assertTrue("Test table should be created", cp.wasCreateTableCalled()); - countDown.await(); + tableCreationLatch.await(); assertTrue("Table pre create handler called.", cp .wasPreCreateTableHandlerCalled()); assertTrue("Table create handler should be called.", cp.wasCreateTableHandlerCalled()); - countDown = new CountDownLatch(1); + tableCreationLatch = new CountDownLatch(1); admin.disableTable(TEST_TABLE); assertTrue(admin.isTableDisabled(TEST_TABLE)); // preDisableTable can't bypass default action. @@ -831,7 +925,7 @@ public class TestMasterObserver { admin.createTable(htd); assertTrue("Test table should be created", cp.wasCreateTableCalled()); - countDown.await(); + tableCreationLatch.await(); assertTrue("Table pre create handler called.", cp .wasPreCreateTableHandlerCalled()); assertTrue("Table create handler should be called.", @@ -908,6 +1002,62 @@ public class TestMasterObserver { cp.wasDeleteTableHandlerCalled()); } + @Test + public void testSnapshotOperations() throws Exception { + MiniHBaseCluster cluster = UTIL.getHBaseCluster(); + HMaster master = cluster.getMaster(); + MasterCoprocessorHost host = master.getCoprocessorHost(); + CPMasterObserver cp = (CPMasterObserver)host.findCoprocessor( + CPMasterObserver.class.getName()); + cp.resetStates(); + + // create a table + HTableDescriptor htd = new HTableDescriptor(TEST_TABLE); + htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); + HBaseAdmin admin = UTIL.getHBaseAdmin(); + + tableCreationLatch = new CountDownLatch(1); + admin.createTable(htd); + tableCreationLatch.await(); + tableCreationLatch = new CountDownLatch(1); + + admin.disableTable(TEST_TABLE); + assertTrue(admin.isTableDisabled(TEST_TABLE)); + + try { + // Test snapshot operation + assertFalse("Coprocessor should not have been called yet", + cp.wasSnapshotCalled()); + admin.snapshot(TEST_SNAPSHOT, TEST_TABLE); + assertTrue("Coprocessor should have been called on snapshot", + cp.wasSnapshotCalled()); + + // Test clone operation + admin.cloneSnapshot(TEST_SNAPSHOT, TEST_CLONE); + assertTrue("Coprocessor should have been called on snapshot clone", + cp.wasCloneSnapshotCalled()); + assertFalse("Coprocessor restore should not have been called on snapshot clone", + cp.wasRestoreSnapshotCalled()); + admin.disableTable(TEST_CLONE); + assertTrue(admin.isTableDisabled(TEST_TABLE)); + admin.deleteTable(TEST_CLONE); + + // Test restore operation + cp.resetStates(); + admin.restoreSnapshot(TEST_SNAPSHOT); + assertTrue("Coprocessor should have been called on snapshot restore", + cp.wasRestoreSnapshotCalled()); + assertFalse("Coprocessor clone should not have been called on snapshot restore", + cp.wasCloneSnapshotCalled()); + + admin.deleteSnapshot(TEST_SNAPSHOT); + assertTrue("Coprocessor should have been called on snapshot delete", + cp.wasDeleteSnapshotCalled()); + } finally { + admin.deleteTable(TEST_TABLE); + } + } + private void modifyTableSync(HBaseAdmin admin, byte[] tableName, HTableDescriptor htd) throws IOException { admin.modifyTable(tableName, htd); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java index 2d17820a3b2..a5dfdff45c4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java @@ -203,6 +203,11 @@ public class TestCatalogJanitor { return this.mfs; } + @Override + public MasterCoprocessorHost getCoprocessorHost() { + return null; + } + @Override public ServerManager getServerManager() { return null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index c92e371e1e7..f5a4d31be98 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -110,6 +110,11 @@ public class TestAccessController { public static void setupBeforeClass() throws Exception { // setup configuration conf = TEST_UTIL.getConfiguration(); + conf.set("hbase.master.hfilecleaner.plugins", + "org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner," + + "org.apache.hadoop.hbase.master.snapshot.SnapshotHFileCleaner"); + conf.set("hbase.master.logcleaner.plugins", + "org.apache.hadoop.hbase.master.snapshot.SnapshotLogCleaner"); SecureTestUtil.enableSecurity(conf); TEST_UTIL.startMiniCluster(); @@ -1411,4 +1416,50 @@ public class TestAccessController { verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER); } + @Test + public void testSnapshot() throws Exception { + PrivilegedExceptionAction snapshotAction = new PrivilegedExceptionAction() { + public Object run() throws Exception { + ACCESS_CONTROLLER.preSnapshot(ObserverContext.createAndPrepare(CP_ENV, null), + null, null); + return null; + } + }; + + PrivilegedExceptionAction deleteAction = new PrivilegedExceptionAction() { + public Object run() throws Exception { + ACCESS_CONTROLLER.preDeleteSnapshot(ObserverContext.createAndPrepare(CP_ENV, null), + null); + return null; + } + }; + + PrivilegedExceptionAction restoreAction = new PrivilegedExceptionAction() { + public Object run() throws Exception { + ACCESS_CONTROLLER.preRestoreSnapshot(ObserverContext.createAndPrepare(CP_ENV, null), + null, null); + return null; + } + }; + + PrivilegedExceptionAction cloneAction = new PrivilegedExceptionAction() { + public Object run() throws Exception { + ACCESS_CONTROLLER.preCloneSnapshot(ObserverContext.createAndPrepare(CP_ENV, null), + null, null); + return null; + } + }; + + verifyAllowed(snapshotAction, SUPERUSER, USER_ADMIN); + verifyDenied(snapshotAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER); + + verifyAllowed(cloneAction, SUPERUSER, USER_ADMIN); + verifyDenied(deleteAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER); + + verifyAllowed(restoreAction, SUPERUSER, USER_ADMIN); + verifyDenied(restoreAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER); + + verifyAllowed(deleteAction, SUPERUSER, USER_ADMIN); + verifyDenied(cloneAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER); + } }