HBASE-7367 Snapshot coprocessor and ACL security (Matteo Bertozzi)

git-svn-id: https://svn.apache.org/repos/asf/hbase/branches/hbase-7290@1445790 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jonathan Hsieh 2013-02-13 18:15:04 +00:00
parent 52b5f1968f
commit 71b0e2c0a8
9 changed files with 599 additions and 6 deletions

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
@ -287,4 +288,50 @@ public class BaseMasterObserver implements MasterObserver {
HRegionInfo region, ServerName srcServer, ServerName destServer) HRegionInfo region, ServerName srcServer, ServerName destServer)
throws IOException { throws IOException {
} }
@Override
public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void postSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void postCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void postRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void preDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException {
}
@Override
public void postDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException {
}
} }

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.RegionPlan; 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 * Defines coprocessor hooks for interacting with operations on the
@ -496,4 +497,100 @@ public interface MasterObserver extends Coprocessor {
*/ */
void postStartMaster(final ObserverContext<MasterCoprocessorEnvironment> ctx) void postStartMaster(final ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException; 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException;
} }

View File

@ -2036,6 +2036,7 @@ Server {
return zooKeeper; return zooKeeper;
} }
@Override
public MasterCoprocessorHost getCoprocessorHost() { public MasterCoprocessorHost getCoprocessorHost() {
return cpHost; return cpHost;
} }
@ -2483,6 +2484,14 @@ Server {
snapshot = snapshot.toBuilder().setVersion(SnapshotDescriptionUtils.SNAPSHOT_LAYOUT_VERSION) snapshot = snapshot.toBuilder().setVersion(SnapshotDescriptionUtils.SNAPSHOT_LAYOUT_VERSION)
.build(); .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 the table is enabled, then have the RS run actually the snapshot work
if (this.assignmentManager.getZKTable().isEnabledTable(snapshot.getTable())) { if (this.assignmentManager.getZKTable().isEnabledTable(snapshot.getTable())) {
LOG.debug("Table enabled, starting distributed snapshot."); LOG.debug("Table enabled, starting distributed snapshot.");
@ -2506,6 +2515,15 @@ Server {
"Table is not entirely open or closed", new TablePartiallyOpenException( "Table is not entirely open or closed", new TablePartiallyOpenException(
snapshot.getTable() + " isn't fully open."), snapshot)); 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 // send back the max amount of time the client should wait for the snapshot to complete
long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(), long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(),
SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME); SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
@ -2566,6 +2584,10 @@ Server {
public DeleteSnapshotResponse deleteSnapshot(RpcController controller, public DeleteSnapshotResponse deleteSnapshot(RpcController controller,
DeleteSnapshotRequest request) throws ServiceException { DeleteSnapshotRequest request) throws ServiceException {
try { try {
if (cpHost != null) {
cpHost.preDeleteSnapshot(request.getSnapshot());
}
// check to see if it is completed // check to see if it is completed
if (!isSnapshotCompleted(request.getSnapshot())) { if (!isSnapshotCompleted(request.getSnapshot())) {
throw new SnapshotDoesNotExistException(request.getSnapshot()); throw new SnapshotDoesNotExistException(request.getSnapshot());
@ -2581,6 +2603,11 @@ Server {
if (!this.getMasterFileSystem().getFileSystem().delete(snapshotDir, true)) { if (!this.getMasterFileSystem().getFileSystem().delete(snapshotDir, true)) {
throw new ServiceException("Failed to delete snapshot directory: " + snapshotDir); throw new ServiceException("Failed to delete snapshot directory: " + snapshotDir);
} }
if (cpHost != null) {
cpHost.postDeleteSnapshot(request.getSnapshot());
}
return DeleteSnapshotResponse.newBuilder().build(); return DeleteSnapshotResponse.newBuilder().build();
} catch (IOException e) { } catch (IOException e) {
throw new ServiceException(e); throw new ServiceException(e);
@ -2701,13 +2728,30 @@ Server {
snapshot.getTable() + "' must be disabled in order to perform a restore operation.")); snapshot.getTable() + "' must be disabled in order to perform a restore operation."));
} }
if (cpHost != null) {
cpHost.preRestoreSnapshot(snapshot, snapshotTableDesc);
}
snapshotManager.restoreSnapshot(snapshot, snapshotTableDesc); snapshotManager.restoreSnapshot(snapshot, snapshotTableDesc);
LOG.info("Restore snapshot=" + snapshot.getName() + " as table=" + tableName); LOG.info("Restore snapshot=" + snapshot.getName() + " as table=" + tableName);
if (cpHost != null) {
cpHost.postRestoreSnapshot(snapshot, snapshotTableDesc);
}
} else { } else {
HTableDescriptor htd = RestoreSnapshotHelper.cloneTableSchema(snapshotTableDesc, HTableDescriptor htd = RestoreSnapshotHelper.cloneTableSchema(snapshotTableDesc,
Bytes.toBytes(tableName)); Bytes.toBytes(tableName));
if (cpHost != null) {
cpHost.preCloneSnapshot(snapshot, htd);
}
snapshotManager.cloneSnapshot(snapshot, htd); snapshotManager.cloneSnapshot(snapshot, htd);
LOG.info("Clone snapshot=" + snapshot.getName() + " as table=" + tableName); LOG.info("Clone snapshot=" + snapshot.getName() + " as table=" + tableName);
if (cpHost != null) {
cpHost.postCloneSnapshot(snapshot, htd);
}
} }
return RestoreSnapshotResponse.newBuilder().build(); return RestoreSnapshotResponse.newBuilder().build();

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.coprocessor.*; import org.apache.hadoop.hbase.coprocessor.*;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
@ -159,6 +160,7 @@ public class MasterCoprocessorHost
} }
} }
} }
public void preDeleteTable(byte[] tableName) throws IOException { public void preDeleteTable(byte[] tableName) throws IOException {
ObserverContext<MasterCoprocessorEnvironment> ctx = null; ObserverContext<MasterCoprocessorEnvironment> ctx = null;
for (MasterEnvironment env: coprocessors) { for (MasterEnvironment env: coprocessors) {
@ -934,4 +936,146 @@ public class MasterCoprocessorHost
} }
} }
} }
public void preSnapshot(final SnapshotDescription snapshot,
final HTableDescriptor hTableDescriptor) throws IOException {
ObserverContext<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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<MasterCoprocessorEnvironment> 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;
}
}
}
}
} }

View File

@ -54,6 +54,11 @@ public interface MasterServices extends Server {
*/ */
public ExecutorService getExecutorService(); public ExecutorService getExecutorService();
/**
* @return Master's instance of {@link MasterCoprocessorHost}
*/
public MasterCoprocessorHost getCoprocessorHost();
/** /**
* Check table is modifiable; i.e. exists and is offline. * Check table is modifiable; i.e. exists and is offline.
* @param tableName Name of table to check. * @param tableName Name of table to check.

View File

@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; 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.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.RegionScanner;
@ -775,6 +776,55 @@ public class AccessController extends BaseRegionObserver
AccessControlLists.init(ctx.getEnvironment().getMasterServices()); AccessControlLists.init(ctx.getEnvironment().getMasterServices());
} }
@Override
public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
requirePermission(Permission.Action.ADMIN);
}
@Override
public void postSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
requirePermission(Permission.Action.ADMIN);
}
@Override
public void postCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
requirePermission(Permission.Action.ADMIN);
}
@Override
public void postRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void preDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException {
requirePermission(Permission.Action.ADMIN);
}
@Override
public void postDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException {
}
/* ---- RegionObserver implementation ---- */ /* ---- RegionObserver implementation ---- */

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.RegionState; 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.RequestConverter;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
@ -62,7 +63,7 @@ import org.junit.experimental.categories.Category;
public class TestMasterObserver { public class TestMasterObserver {
private static final Log LOG = LogFactory.getLog(TestMasterObserver.class); 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 { public static class CPMasterObserver implements MasterObserver {
@ -98,6 +99,14 @@ public class TestMasterObserver {
private boolean postStartMasterCalled; private boolean postStartMasterCalled;
private boolean startCalled; private boolean startCalled;
private boolean stopCalled; 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 preCreateTableHandlerCalled;
private boolean postCreateTableHandlerCalled; private boolean postCreateTableHandlerCalled;
private boolean preDeleteTableHandlerCalled; private boolean preDeleteTableHandlerCalled;
@ -115,7 +124,6 @@ public class TestMasterObserver {
private boolean preModifyTableHandlerCalled; private boolean preModifyTableHandlerCalled;
private boolean postModifyTableHandlerCalled; private boolean postModifyTableHandlerCalled;
public void enableBypass(boolean bypass) { public void enableBypass(boolean bypass) {
this.bypass = bypass; this.bypass = bypass;
} }
@ -147,6 +155,14 @@ public class TestMasterObserver {
postBalanceCalled = false; postBalanceCalled = false;
preBalanceSwitchCalled = false; preBalanceSwitchCalled = false;
postBalanceSwitchCalled = false; postBalanceSwitchCalled = false;
preSnapshotCalled = false;
postSnapshotCalled = false;
preCloneSnapshotCalled = false;
postCloneSnapshotCalled = false;
preRestoreSnapshotCalled = false;
postRestoreSnapshotCalled = false;
preDeleteSnapshotCalled = false;
postDeleteSnapshotCalled = false;
preCreateTableHandlerCalled = false; preCreateTableHandlerCalled = false;
postCreateTableHandlerCalled = false; postCreateTableHandlerCalled = false;
preDeleteTableHandlerCalled = false; preDeleteTableHandlerCalled = false;
@ -508,6 +524,76 @@ public class TestMasterObserver {
public boolean wasStopped() { return stopCalled; } public boolean wasStopped() { return stopCalled; }
@Override
public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
preSnapshotCalled = true;
}
@Override
public void postSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
postSnapshotCalled = true;
}
public boolean wasSnapshotCalled() {
return preSnapshotCalled && postSnapshotCalled;
}
@Override
public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
preCloneSnapshotCalled = true;
}
@Override
public void postCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
postCloneSnapshotCalled = true;
}
public boolean wasCloneSnapshotCalled() {
return preCloneSnapshotCalled && postCloneSnapshotCalled;
}
@Override
public void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
preRestoreSnapshotCalled = true;
}
@Override
public void postRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
postRestoreSnapshotCalled = true;
}
public boolean wasRestoreSnapshotCalled() {
return preRestoreSnapshotCalled && postRestoreSnapshotCalled;
}
@Override
public void preDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException {
preDeleteSnapshotCalled = true;
}
@Override
public void postDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException {
postDeleteSnapshotCalled = true;
}
public boolean wasDeleteSnapshotCalled() {
return preDeleteSnapshotCalled && postDeleteSnapshotCalled;
}
@Override @Override
public void preCreateTableHandler( public void preCreateTableHandler(
ObserverContext<MasterCoprocessorEnvironment> env, ObserverContext<MasterCoprocessorEnvironment> env,
@ -523,7 +609,7 @@ public class TestMasterObserver {
ObserverContext<MasterCoprocessorEnvironment> ctx, ObserverContext<MasterCoprocessorEnvironment> ctx,
HTableDescriptor desc, HRegionInfo[] regions) throws IOException { HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
postCreateTableHandlerCalled = true; postCreateTableHandlerCalled = true;
countDown.countDown(); tableCreationLatch.countDown();
} }
public boolean wasPreCreateTableHandlerCalled(){ public boolean wasPreCreateTableHandlerCalled(){
@ -711,7 +797,9 @@ public class TestMasterObserver {
} }
private static HBaseTestingUtility UTIL = new HBaseTestingUtility(); 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_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_FAMILY = Bytes.toBytes("fam1");
private static byte[] TEST_FAMILY2 = Bytes.toBytes("fam2"); private static byte[] TEST_FAMILY2 = Bytes.toBytes("fam2");
private static byte[] TEST_FAMILY3 = Bytes.toBytes("fam3"); private static byte[] TEST_FAMILY3 = Bytes.toBytes("fam3");
@ -721,6 +809,11 @@ public class TestMasterObserver {
Configuration conf = UTIL.getConfiguration(); Configuration conf = UTIL.getConfiguration();
conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
CPMasterObserver.class.getName()); 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 // We need more than one data server on this test
UTIL.startMiniCluster(2); UTIL.startMiniCluster(2);
} }
@ -765,16 +858,17 @@ public class TestMasterObserver {
htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
HBaseAdmin admin = UTIL.getHBaseAdmin(); HBaseAdmin admin = UTIL.getHBaseAdmin();
tableCreationLatch = new CountDownLatch(1);
admin.createTable(htd); admin.createTable(htd);
// preCreateTable can't bypass default action. // preCreateTable can't bypass default action.
assertTrue("Test table should be created", cp.wasCreateTableCalled()); assertTrue("Test table should be created", cp.wasCreateTableCalled());
countDown.await(); tableCreationLatch.await();
assertTrue("Table pre create handler called.", cp assertTrue("Table pre create handler called.", cp
.wasPreCreateTableHandlerCalled()); .wasPreCreateTableHandlerCalled());
assertTrue("Table create handler should be called.", assertTrue("Table create handler should be called.",
cp.wasCreateTableHandlerCalled()); cp.wasCreateTableHandlerCalled());
countDown = new CountDownLatch(1); tableCreationLatch = new CountDownLatch(1);
admin.disableTable(TEST_TABLE); admin.disableTable(TEST_TABLE);
assertTrue(admin.isTableDisabled(TEST_TABLE)); assertTrue(admin.isTableDisabled(TEST_TABLE));
// preDisableTable can't bypass default action. // preDisableTable can't bypass default action.
@ -831,7 +925,7 @@ public class TestMasterObserver {
admin.createTable(htd); admin.createTable(htd);
assertTrue("Test table should be created", cp.wasCreateTableCalled()); assertTrue("Test table should be created", cp.wasCreateTableCalled());
countDown.await(); tableCreationLatch.await();
assertTrue("Table pre create handler called.", cp assertTrue("Table pre create handler called.", cp
.wasPreCreateTableHandlerCalled()); .wasPreCreateTableHandlerCalled());
assertTrue("Table create handler should be called.", assertTrue("Table create handler should be called.",
@ -908,6 +1002,62 @@ public class TestMasterObserver {
cp.wasDeleteTableHandlerCalled()); 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) private void modifyTableSync(HBaseAdmin admin, byte[] tableName, HTableDescriptor htd)
throws IOException { throws IOException {
admin.modifyTable(tableName, htd); admin.modifyTable(tableName, htd);

View File

@ -203,6 +203,11 @@ public class TestCatalogJanitor {
return this.mfs; return this.mfs;
} }
@Override
public MasterCoprocessorHost getCoprocessorHost() {
return null;
}
@Override @Override
public ServerManager getServerManager() { public ServerManager getServerManager() {
return null; return null;

View File

@ -110,6 +110,11 @@ public class TestAccessController {
public static void setupBeforeClass() throws Exception { public static void setupBeforeClass() throws Exception {
// setup configuration // setup configuration
conf = TEST_UTIL.getConfiguration(); 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); SecureTestUtil.enableSecurity(conf);
TEST_UTIL.startMiniCluster(); TEST_UTIL.startMiniCluster();
@ -1411,4 +1416,50 @@ public class TestAccessController {
verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER); 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);
}
} }