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:
parent
52b5f1968f
commit
71b0e2c0a8
|
@ -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<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 {
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<MasterCoprocessorEnvironment> 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<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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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<MasterCoprocessorEnvironment> 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<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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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<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 ---- */
|
||||
|
||||
|
|
|
@ -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<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
|
||||
public void preCreateTableHandler(
|
||||
ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
|
@ -523,7 +609,7 @@ public class TestMasterObserver {
|
|||
ObserverContext<MasterCoprocessorEnvironment> 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);
|
||||
|
|
|
@ -203,6 +203,11 @@ public class TestCatalogJanitor {
|
|||
return this.mfs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MasterCoprocessorHost getCoprocessorHost() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerManager getServerManager() {
|
||||
return null;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue