HBASE-16217 Pass through the calling user in ObserverContext
This commit is contained in:
parent
9d740f7b8b
commit
65834a1ced
@ -159,6 +159,7 @@ import org.apache.hadoop.hbase.quotas.ThrottleType;
|
|||||||
import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
|
import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
|
||||||
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
|
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
|
||||||
import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
|
import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.security.access.Permission;
|
import org.apache.hadoop.hbase.security.access.Permission;
|
||||||
import org.apache.hadoop.hbase.security.access.TablePermission;
|
import org.apache.hadoop.hbase.security.access.TablePermission;
|
||||||
import org.apache.hadoop.hbase.security.access.UserPermission;
|
import org.apache.hadoop.hbase.security.access.UserPermission;
|
||||||
@ -1874,12 +1875,12 @@ public final class ProtobufUtil {
|
|||||||
public static void mergeRegions(final RpcController controller,
|
public static void mergeRegions(final RpcController controller,
|
||||||
final AdminService.BlockingInterface admin,
|
final AdminService.BlockingInterface admin,
|
||||||
final HRegionInfo region_a, final HRegionInfo region_b,
|
final HRegionInfo region_a, final HRegionInfo region_b,
|
||||||
final boolean forcible, final UserGroupInformation user) throws IOException {
|
final boolean forcible, final User user) throws IOException {
|
||||||
final MergeRegionsRequest request = RequestConverter.buildMergeRegionsRequest(
|
final MergeRegionsRequest request = RequestConverter.buildMergeRegionsRequest(
|
||||||
region_a.getRegionName(), region_b.getRegionName(),forcible);
|
region_a.getRegionName(), region_b.getRegionName(),forcible);
|
||||||
if (user != null) {
|
if (user != null) {
|
||||||
try {
|
try {
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
user.runAs(new PrivilegedExceptionAction<Void>() {
|
||||||
@Override
|
@Override
|
||||||
public Void run() throws Exception {
|
public Void run() throws Exception {
|
||||||
admin.mergeRegions(controller, request);
|
admin.mergeRegions(controller, request);
|
||||||
|
@ -23,6 +23,10 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Carries the execution state for a given invocation of an Observer coprocessor
|
* Carries the execution state for a given invocation of an Observer coprocessor
|
||||||
@ -40,8 +44,10 @@ public class ObserverContext<E extends CoprocessorEnvironment> {
|
|||||||
private E env;
|
private E env;
|
||||||
private boolean bypass;
|
private boolean bypass;
|
||||||
private boolean complete;
|
private boolean complete;
|
||||||
|
private User caller;
|
||||||
|
|
||||||
public ObserverContext() {
|
public ObserverContext(User caller) {
|
||||||
|
this.caller = caller;
|
||||||
}
|
}
|
||||||
|
|
||||||
public E getEnvironment() {
|
public E getEnvironment() {
|
||||||
@ -91,6 +97,17 @@ public class ObserverContext<E extends CoprocessorEnvironment> {
|
|||||||
return current;
|
return current;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the active user for the coprocessor call.
|
||||||
|
* If an explicit {@code User} instance was provided to the constructor, that will be returned,
|
||||||
|
* otherwise if we are in the context of an RPC call, the remote user is used. May return null
|
||||||
|
* if the execution is outside of an RPC context.
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
public User getCaller() {
|
||||||
|
return caller;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Instantiates a new ObserverContext instance if the passed reference is
|
* Instantiates a new ObserverContext instance if the passed reference is
|
||||||
* <code>null</code> and sets the environment in the new or existing instance.
|
* <code>null</code> and sets the environment in the new or existing instance.
|
||||||
@ -103,10 +120,34 @@ public class ObserverContext<E extends CoprocessorEnvironment> {
|
|||||||
* @param <T> The environment type for the context
|
* @param <T> The environment type for the context
|
||||||
* @return An instance of <code>ObserverContext</code> with the environment set
|
* @return An instance of <code>ObserverContext</code> with the environment set
|
||||||
*/
|
*/
|
||||||
|
@Deprecated
|
||||||
|
// TODO: Remove this method, ObserverContext should not depend on RpcServer
|
||||||
public static <T extends CoprocessorEnvironment> ObserverContext<T> createAndPrepare(
|
public static <T extends CoprocessorEnvironment> ObserverContext<T> createAndPrepare(
|
||||||
T env, ObserverContext<T> context) {
|
T env, ObserverContext<T> context) {
|
||||||
if (context == null) {
|
if (context == null) {
|
||||||
context = new ObserverContext<T>();
|
context = new ObserverContext<T>(RpcServer.getRequestUser());
|
||||||
|
}
|
||||||
|
context.prepare(env);
|
||||||
|
return context;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Instantiates a new ObserverContext instance if the passed reference is
|
||||||
|
* <code>null</code> and sets the environment in the new or existing instance.
|
||||||
|
* This allows deferring the instantiation of a ObserverContext until it is
|
||||||
|
* actually needed.
|
||||||
|
*
|
||||||
|
* @param env The coprocessor environment to set
|
||||||
|
* @param context An existing ObserverContext instance to use, or <code>null</code>
|
||||||
|
* to create a new instance
|
||||||
|
* @param user The requesting caller for the execution context
|
||||||
|
* @param <T> The environment type for the context
|
||||||
|
* @return An instance of <code>ObserverContext</code> with the environment set
|
||||||
|
*/
|
||||||
|
public static <T extends CoprocessorEnvironment> ObserverContext<T> createAndPrepare(
|
||||||
|
T env, ObserverContext<T> context, User user) {
|
||||||
|
if (context == null) {
|
||||||
|
context = new ObserverContext<T>(user);
|
||||||
}
|
}
|
||||||
context.prepare(env);
|
context.prepare(env);
|
||||||
return context;
|
return context;
|
||||||
|
@ -44,10 +44,12 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
|
|||||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
||||||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||||
|
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
|
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Provides the coprocessor framework and environment for master oriented
|
* Provides the coprocessor framework and environment for master oriented
|
||||||
@ -240,9 +242,10 @@ public class MasterCoprocessorHost
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void preCreateTableAction(final HTableDescriptor htd, final HRegionInfo[] regions)
|
public void preCreateTableAction(final HTableDescriptor htd, final HRegionInfo[] regions,
|
||||||
|
final User user)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -253,8 +256,8 @@ public class MasterCoprocessorHost
|
|||||||
}
|
}
|
||||||
|
|
||||||
public void postCompletedCreateTableAction(
|
public void postCompletedCreateTableAction(
|
||||||
final HTableDescriptor htd, final HRegionInfo[] regions) throws IOException {
|
final HTableDescriptor htd, final HRegionInfo[] regions, final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -284,8 +287,8 @@ public class MasterCoprocessorHost
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void preDeleteTableAction(final TableName tableName) throws IOException {
|
public void preDeleteTableAction(final TableName tableName, final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -295,8 +298,9 @@ public class MasterCoprocessorHost
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void postCompletedDeleteTableAction(final TableName tableName) throws IOException {
|
public void postCompletedDeleteTableAction(final TableName tableName, final User user)
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
throws IOException {
|
||||||
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -326,8 +330,8 @@ public class MasterCoprocessorHost
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void preTruncateTableAction(final TableName tableName) throws IOException {
|
public void preTruncateTableAction(final TableName tableName, final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -337,8 +341,9 @@ public class MasterCoprocessorHost
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void postCompletedTruncateTableAction(final TableName tableName) throws IOException {
|
public void postCompletedTruncateTableAction(final TableName tableName, final User user)
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
throws IOException {
|
||||||
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -370,9 +375,10 @@ public class MasterCoprocessorHost
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void preModifyTableAction(final TableName tableName, final HTableDescriptor htd)
|
public void preModifyTableAction(final TableName tableName, final HTableDescriptor htd,
|
||||||
|
final User user)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -382,9 +388,10 @@ public class MasterCoprocessorHost
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void postCompletedModifyTableAction(final TableName tableName, final HTableDescriptor htd)
|
public void postCompletedModifyTableAction(final TableName tableName, final HTableDescriptor htd,
|
||||||
|
final User user)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -420,9 +427,10 @@ public class MasterCoprocessorHost
|
|||||||
|
|
||||||
public boolean preAddColumnFamilyAction(
|
public boolean preAddColumnFamilyAction(
|
||||||
final TableName tableName,
|
final TableName tableName,
|
||||||
final HColumnDescriptor columnFamily)
|
final HColumnDescriptor columnFamily,
|
||||||
|
final User user)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -434,9 +442,10 @@ public class MasterCoprocessorHost
|
|||||||
|
|
||||||
public void postCompletedAddColumnFamilyAction(
|
public void postCompletedAddColumnFamilyAction(
|
||||||
final TableName tableName,
|
final TableName tableName,
|
||||||
final HColumnDescriptor columnFamily)
|
final HColumnDescriptor columnFamily,
|
||||||
|
final User user)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -472,8 +481,9 @@ public class MasterCoprocessorHost
|
|||||||
|
|
||||||
public boolean preModifyColumnFamilyAction(
|
public boolean preModifyColumnFamilyAction(
|
||||||
final TableName tableName,
|
final TableName tableName,
|
||||||
final HColumnDescriptor columnFamily) throws IOException {
|
final HColumnDescriptor columnFamily,
|
||||||
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
final User user) throws IOException {
|
||||||
|
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -485,8 +495,9 @@ public class MasterCoprocessorHost
|
|||||||
|
|
||||||
public void postCompletedModifyColumnFamilyAction(
|
public void postCompletedModifyColumnFamilyAction(
|
||||||
final TableName tableName,
|
final TableName tableName,
|
||||||
final HColumnDescriptor columnFamily) throws IOException {
|
final HColumnDescriptor columnFamily,
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
final User user) throws IOException {
|
||||||
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -522,9 +533,10 @@ public class MasterCoprocessorHost
|
|||||||
|
|
||||||
public boolean preDeleteColumnFamilyAction(
|
public boolean preDeleteColumnFamilyAction(
|
||||||
final TableName tableName,
|
final TableName tableName,
|
||||||
final byte[] columnFamily)
|
final byte[] columnFamily,
|
||||||
|
final User user)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -535,8 +547,8 @@ public class MasterCoprocessorHost
|
|||||||
}
|
}
|
||||||
|
|
||||||
public void postCompletedDeleteColumnFamilyAction(
|
public void postCompletedDeleteColumnFamilyAction(
|
||||||
final TableName tableName, final byte[] columnFamily) throws IOException {
|
final TableName tableName, final byte[] columnFamily, final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -566,8 +578,8 @@ public class MasterCoprocessorHost
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void preEnableTableAction(final TableName tableName) throws IOException {
|
public void preEnableTableAction(final TableName tableName, final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -577,8 +589,9 @@ public class MasterCoprocessorHost
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void postCompletedEnableTableAction(final TableName tableName) throws IOException {
|
public void postCompletedEnableTableAction(final TableName tableName, final User user)
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
throws IOException {
|
||||||
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -608,8 +621,8 @@ public class MasterCoprocessorHost
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void preDisableTableAction(final TableName tableName) throws IOException {
|
public void preDisableTableAction(final TableName tableName, final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -619,8 +632,9 @@ public class MasterCoprocessorHost
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void postCompletedDisableTableAction(final TableName tableName) throws IOException {
|
public void postCompletedDisableTableAction(final TableName tableName, final User user)
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
throws IOException {
|
||||||
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -1168,6 +1182,11 @@ public class MasterCoprocessorHost
|
|||||||
private static abstract class CoprocessorOperation
|
private static abstract class CoprocessorOperation
|
||||||
extends ObserverContext<MasterCoprocessorEnvironment> {
|
extends ObserverContext<MasterCoprocessorEnvironment> {
|
||||||
public CoprocessorOperation() {
|
public CoprocessorOperation() {
|
||||||
|
this(RpcServer.getRequestUser());
|
||||||
|
}
|
||||||
|
|
||||||
|
public CoprocessorOperation(User user) {
|
||||||
|
super(user);
|
||||||
}
|
}
|
||||||
|
|
||||||
public abstract void call(MasterObserver oserver,
|
public abstract void call(MasterObserver oserver,
|
||||||
|
@ -67,13 +67,13 @@ import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSeque
|
|||||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
|
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.hbase.util.RetryCounter;
|
import org.apache.hadoop.hbase.util.RetryCounter;
|
||||||
import org.apache.hadoop.hbase.util.RetryCounterFactory;
|
import org.apache.hadoop.hbase.util.RetryCounterFactory;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
@ -884,7 +884,7 @@ public class ServerManager {
|
|||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public void sendRegionsMerge(ServerName server, HRegionInfo region_a,
|
public void sendRegionsMerge(ServerName server, HRegionInfo region_a,
|
||||||
HRegionInfo region_b, boolean forcible, final UserGroupInformation user) throws IOException {
|
HRegionInfo region_b, boolean forcible, final User user) throws IOException {
|
||||||
if (server == null)
|
if (server == null)
|
||||||
throw new NullPointerException("Passed server is null");
|
throw new NullPointerException("Passed server is null");
|
||||||
if (region_a == null || region_b == null)
|
if (region_a == null || region_b == null)
|
||||||
|
@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.master.procedure;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.security.PrivilegedExceptionAction;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
@ -39,7 +38,7 @@ import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.hbase.security.User;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The procedure to add a column family to an existing table.
|
* The procedure to add a column family to an existing table.
|
||||||
@ -55,7 +54,7 @@ public class AddColumnFamilyProcedure
|
|||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
private HTableDescriptor unmodifiedHTableDescriptor;
|
private HTableDescriptor unmodifiedHTableDescriptor;
|
||||||
private HColumnDescriptor cfDescriptor;
|
private HColumnDescriptor cfDescriptor;
|
||||||
private UserGroupInformation user;
|
private User user;
|
||||||
|
|
||||||
private List<HRegionInfo> regionInfoList;
|
private List<HRegionInfo> regionInfoList;
|
||||||
private Boolean traceEnabled;
|
private Boolean traceEnabled;
|
||||||
@ -72,8 +71,8 @@ public class AddColumnFamilyProcedure
|
|||||||
final HColumnDescriptor cfDescriptor) throws IOException {
|
final HColumnDescriptor cfDescriptor) throws IOException {
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.cfDescriptor = cfDescriptor;
|
this.cfDescriptor = cfDescriptor;
|
||||||
this.user = env.getRequestUser().getUGI();
|
this.user = env.getRequestUser();
|
||||||
this.setOwner(this.user.getShortUserName());
|
this.setOwner(this.user.getShortName());
|
||||||
this.unmodifiedHTableDescriptor = null;
|
this.unmodifiedHTableDescriptor = null;
|
||||||
this.regionInfoList = null;
|
this.regionInfoList = null;
|
||||||
this.traceEnabled = null;
|
this.traceEnabled = null;
|
||||||
@ -378,22 +377,16 @@ public class AddColumnFamilyProcedure
|
|||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
switch (state) {
|
||||||
@Override
|
case ADD_COLUMN_FAMILY_PRE_OPERATION:
|
||||||
public Void run() throws Exception {
|
cpHost.preAddColumnFamilyAction(tableName, cfDescriptor, user);
|
||||||
switch (state) {
|
break;
|
||||||
case ADD_COLUMN_FAMILY_PRE_OPERATION:
|
case ADD_COLUMN_FAMILY_POST_OPERATION:
|
||||||
cpHost.preAddColumnFamilyAction(tableName, cfDescriptor);
|
cpHost.postCompletedAddColumnFamilyAction(tableName, cfDescriptor, user);
|
||||||
break;
|
break;
|
||||||
case ADD_COLUMN_FAMILY_POST_OPERATION:
|
default:
|
||||||
cpHost.postCompletedAddColumnFamilyAction(tableName, cfDescriptor);
|
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||||
break;
|
}
|
||||||
default:
|
|
||||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
|
||||||
}
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
|||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CloneSnapshotState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CloneSnapshotState;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
@ -62,7 +63,6 @@ import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
|
|||||||
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
|
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
|
||||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||||
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
|
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
@ -74,7 +74,7 @@ public class CloneSnapshotProcedure
|
|||||||
|
|
||||||
private final AtomicBoolean aborted = new AtomicBoolean(false);
|
private final AtomicBoolean aborted = new AtomicBoolean(false);
|
||||||
|
|
||||||
private UserGroupInformation user;
|
private User user;
|
||||||
private HTableDescriptor hTableDescriptor;
|
private HTableDescriptor hTableDescriptor;
|
||||||
private SnapshotDescription snapshot;
|
private SnapshotDescription snapshot;
|
||||||
private List<HRegionInfo> newRegions = null;
|
private List<HRegionInfo> newRegions = null;
|
||||||
@ -106,8 +106,8 @@ public class CloneSnapshotProcedure
|
|||||||
throws IOException {
|
throws IOException {
|
||||||
this.hTableDescriptor = hTableDescriptor;
|
this.hTableDescriptor = hTableDescriptor;
|
||||||
this.snapshot = snapshot;
|
this.snapshot = snapshot;
|
||||||
this.user = env.getRequestUser().getUGI();
|
this.user = env.getRequestUser();
|
||||||
this.setOwner(this.user.getShortUserName());
|
this.setOwner(this.user.getShortName());
|
||||||
|
|
||||||
getMonitorStatus();
|
getMonitorStatus();
|
||||||
}
|
}
|
||||||
@ -372,13 +372,7 @@ public class CloneSnapshotProcedure
|
|||||||
|
|
||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
cpHost.preCreateTableAction(hTableDescriptor, null, user);
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
cpHost.preCreateTableAction(hTableDescriptor, null);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -394,13 +388,7 @@ public class CloneSnapshotProcedure
|
|||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final HRegionInfo[] regions = (newRegions == null) ? null :
|
final HRegionInfo[] regions = (newRegions == null) ? null :
|
||||||
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
cpHost.postCompletedCreateTableAction(hTableDescriptor, regions, user);
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
cpHost.postCompletedCreateTableAction(hTableDescriptor, regions);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -47,11 +47,11 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
|||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||||
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
|
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
|
||||||
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
|
||||||
@ -68,7 +68,7 @@ public class CreateTableProcedure
|
|||||||
|
|
||||||
private HTableDescriptor hTableDescriptor;
|
private HTableDescriptor hTableDescriptor;
|
||||||
private List<HRegionInfo> newRegions;
|
private List<HRegionInfo> newRegions;
|
||||||
private UserGroupInformation user;
|
private User user;
|
||||||
|
|
||||||
public CreateTableProcedure() {
|
public CreateTableProcedure() {
|
||||||
// Required by the Procedure framework to create the procedure on replay
|
// Required by the Procedure framework to create the procedure on replay
|
||||||
@ -87,8 +87,8 @@ public class CreateTableProcedure
|
|||||||
throws IOException {
|
throws IOException {
|
||||||
this.hTableDescriptor = hTableDescriptor;
|
this.hTableDescriptor = hTableDescriptor;
|
||||||
this.newRegions = newRegions != null ? Lists.newArrayList(newRegions) : null;
|
this.newRegions = newRegions != null ? Lists.newArrayList(newRegions) : null;
|
||||||
this.user = env.getRequestUser().getUGI();
|
this.user = env.getRequestUser();
|
||||||
this.setOwner(this.user.getShortUserName());
|
this.setOwner(this.user.getShortName());
|
||||||
|
|
||||||
// used for compatibility with clients without procedures
|
// used for compatibility with clients without procedures
|
||||||
// they need a sync TableExistsException
|
// they need a sync TableExistsException
|
||||||
@ -307,13 +307,7 @@ public class CreateTableProcedure
|
|||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final HRegionInfo[] regions = newRegions == null ? null :
|
final HRegionInfo[] regions = newRegions == null ? null :
|
||||||
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
cpHost.preCreateTableAction(hTableDescriptor, regions, user);
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
cpHost.preCreateTableAction(hTableDescriptor, regions);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -323,13 +317,7 @@ public class CreateTableProcedure
|
|||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final HRegionInfo[] regions = (newRegions == null) ? null :
|
final HRegionInfo[] regions = (newRegions == null) ? null :
|
||||||
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
cpHost.postCompletedCreateTableAction(hTableDescriptor, regions, user);
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
cpHost.postCompletedCreateTableAction(hTableDescriptor, regions);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -38,9 +38,9 @@ import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyState;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The procedure to delete a column family from an existing table.
|
* The procedure to delete a column family from an existing table.
|
||||||
@ -57,7 +57,7 @@ public class DeleteColumnFamilyProcedure
|
|||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
private byte [] familyName;
|
private byte [] familyName;
|
||||||
private boolean hasMob;
|
private boolean hasMob;
|
||||||
private UserGroupInformation user;
|
private User user;
|
||||||
|
|
||||||
private List<HRegionInfo> regionInfoList;
|
private List<HRegionInfo> regionInfoList;
|
||||||
private Boolean traceEnabled;
|
private Boolean traceEnabled;
|
||||||
@ -74,8 +74,8 @@ public class DeleteColumnFamilyProcedure
|
|||||||
final byte[] familyName) throws IOException {
|
final byte[] familyName) throws IOException {
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.familyName = familyName;
|
this.familyName = familyName;
|
||||||
this.user = env.getRequestUser().getUGI();
|
this.user = env.getRequestUser();
|
||||||
this.setOwner(this.user.getShortUserName());
|
this.setOwner(this.user.getShortName());
|
||||||
this.unmodifiedHTableDescriptor = null;
|
this.unmodifiedHTableDescriptor = null;
|
||||||
this.regionInfoList = null;
|
this.regionInfoList = null;
|
||||||
this.traceEnabled = null;
|
this.traceEnabled = null;
|
||||||
@ -403,22 +403,16 @@ public class DeleteColumnFamilyProcedure
|
|||||||
final DeleteColumnFamilyState state) throws IOException, InterruptedException {
|
final DeleteColumnFamilyState state) throws IOException, InterruptedException {
|
||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
switch (state) {
|
||||||
@Override
|
case DELETE_COLUMN_FAMILY_PRE_OPERATION:
|
||||||
public Void run() throws Exception {
|
cpHost.preDeleteColumnFamilyAction(tableName, familyName, user);
|
||||||
switch (state) {
|
break;
|
||||||
case DELETE_COLUMN_FAMILY_PRE_OPERATION:
|
case DELETE_COLUMN_FAMILY_POST_OPERATION:
|
||||||
cpHost.preDeleteColumnFamilyAction(tableName, familyName);
|
cpHost.postCompletedDeleteColumnFamilyAction(tableName, familyName, user);
|
||||||
break;
|
break;
|
||||||
case DELETE_COLUMN_FAMILY_POST_OPERATION:
|
default:
|
||||||
cpHost.postCompletedDeleteColumnFamilyAction(tableName, familyName);
|
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||||
break;
|
}
|
||||||
default:
|
|
||||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
|
||||||
}
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -55,8 +55,8 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
|||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class DeleteTableProcedure
|
public class DeleteTableProcedure
|
||||||
@ -65,7 +65,7 @@ public class DeleteTableProcedure
|
|||||||
private static final Log LOG = LogFactory.getLog(DeleteTableProcedure.class);
|
private static final Log LOG = LogFactory.getLog(DeleteTableProcedure.class);
|
||||||
|
|
||||||
private List<HRegionInfo> regions;
|
private List<HRegionInfo> regions;
|
||||||
private UserGroupInformation user;
|
private User user;
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
|
|
||||||
// used for compatibility with old clients
|
// used for compatibility with old clients
|
||||||
@ -84,8 +84,8 @@ public class DeleteTableProcedure
|
|||||||
public DeleteTableProcedure(final MasterProcedureEnv env, final TableName tableName,
|
public DeleteTableProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||||
final ProcedurePrepareLatch syncLatch) throws IOException {
|
final ProcedurePrepareLatch syncLatch) throws IOException {
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.user = env.getRequestUser().getUGI();
|
this.user = env.getRequestUser();
|
||||||
this.setOwner(this.user.getShortUserName());
|
this.setOwner(this.user.getShortName());
|
||||||
|
|
||||||
// used for compatibility with clients without procedures
|
// used for compatibility with clients without procedures
|
||||||
// they need a sync TableNotFoundException, TableNotDisabledException, ...
|
// they need a sync TableNotFoundException, TableNotDisabledException, ...
|
||||||
@ -266,13 +266,7 @@ public class DeleteTableProcedure
|
|||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final TableName tableName = this.tableName;
|
final TableName tableName = this.tableName;
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
cpHost.preDeleteTableAction(tableName, user);
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
cpHost.preDeleteTableAction(tableName);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@ -284,13 +278,7 @@ public class DeleteTableProcedure
|
|||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final TableName tableName = this.tableName;
|
final TableName tableName = this.tableName;
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
cpHost.postCompletedDeleteTableAction(tableName, user);
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
cpHost.postCompletedDeleteTableAction(tableName);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.htrace.Trace;
|
import org.apache.htrace.Trace;
|
||||||
@ -63,7 +64,7 @@ public class DisableTableProcedure
|
|||||||
|
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
private boolean skipTableStateCheck;
|
private boolean skipTableStateCheck;
|
||||||
private UserGroupInformation user;
|
private User user;
|
||||||
|
|
||||||
private Boolean traceEnabled = null;
|
private Boolean traceEnabled = null;
|
||||||
|
|
||||||
@ -105,8 +106,8 @@ public class DisableTableProcedure
|
|||||||
final ProcedurePrepareLatch syncLatch) throws IOException {
|
final ProcedurePrepareLatch syncLatch) throws IOException {
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.skipTableStateCheck = skipTableStateCheck;
|
this.skipTableStateCheck = skipTableStateCheck;
|
||||||
this.user = env.getRequestUser().getUGI();
|
this.user = env.getRequestUser();
|
||||||
this.setOwner(this.user.getShortUserName());
|
this.setOwner(this.user.getShortName());
|
||||||
|
|
||||||
// Compatible with 1.0: We use latch to make sure that this procedure implementation is
|
// Compatible with 1.0: We use latch to make sure that this procedure implementation is
|
||||||
// compatible with 1.0 asynchronized operations. We need to lock the table and check
|
// compatible with 1.0 asynchronized operations. We need to lock the table and check
|
||||||
@ -458,22 +459,16 @@ public class DisableTableProcedure
|
|||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
switch (state) {
|
||||||
@Override
|
case DISABLE_TABLE_PRE_OPERATION:
|
||||||
public Void run() throws Exception {
|
cpHost.preDisableTableAction(tableName, user);
|
||||||
switch (state) {
|
break;
|
||||||
case DISABLE_TABLE_PRE_OPERATION:
|
case DISABLE_TABLE_POST_OPERATION:
|
||||||
cpHost.preDisableTableAction(tableName);
|
cpHost.postCompletedDisableTableAction(tableName, user);
|
||||||
break;
|
break;
|
||||||
case DISABLE_TABLE_POST_OPERATION:
|
default:
|
||||||
cpHost.postCompletedDisableTableAction(tableName);
|
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||||
break;
|
}
|
||||||
default:
|
|
||||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
|
||||||
}
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -46,8 +46,8 @@ import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsState;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The procedure to Merge a region in a table.
|
* The procedure to Merge a region in a table.
|
||||||
@ -66,7 +66,7 @@ implements TableProcedureInterface {
|
|||||||
private String regionsToMergeListFullName;
|
private String regionsToMergeListFullName;
|
||||||
private String regionsToMergeListEncodedName;
|
private String regionsToMergeListEncodedName;
|
||||||
|
|
||||||
private UserGroupInformation user;
|
private User user;
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
private HRegionInfo [] regionsToMerge;
|
private HRegionInfo [] regionsToMerge;
|
||||||
private boolean forcible;
|
private boolean forcible;
|
||||||
@ -94,8 +94,8 @@ implements TableProcedureInterface {
|
|||||||
this.regionsToMerge = regionsToMerge;
|
this.regionsToMerge = regionsToMerge;
|
||||||
this.forcible = forcible;
|
this.forcible = forcible;
|
||||||
|
|
||||||
this.user = env.getRequestUser().getUGI();
|
this.user = env.getRequestUser();
|
||||||
this.setOwner(this.user.getShortUserName());
|
this.setOwner(this.user.getShortName());
|
||||||
|
|
||||||
this.timeout = -1;
|
this.timeout = -1;
|
||||||
this.regionsToMergeListFullName = getRegionsToMergeListFullNameString();
|
this.regionsToMergeListFullName = getRegionsToMergeListFullNameString();
|
||||||
|
@ -50,9 +50,9 @@ import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class EnableTableProcedure
|
public class EnableTableProcedure
|
||||||
@ -67,7 +67,7 @@ public class EnableTableProcedure
|
|||||||
|
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
private boolean skipTableStateCheck;
|
private boolean skipTableStateCheck;
|
||||||
private UserGroupInformation user;
|
private User user;
|
||||||
|
|
||||||
private Boolean traceEnabled = null;
|
private Boolean traceEnabled = null;
|
||||||
|
|
||||||
@ -103,8 +103,8 @@ public class EnableTableProcedure
|
|||||||
final ProcedurePrepareLatch syncLatch) throws IOException {
|
final ProcedurePrepareLatch syncLatch) throws IOException {
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.skipTableStateCheck = skipTableStateCheck;
|
this.skipTableStateCheck = skipTableStateCheck;
|
||||||
this.user = env.getRequestUser().getUGI();
|
this.user = env.getRequestUser();
|
||||||
this.setOwner(this.user.getShortUserName());
|
this.setOwner(this.user.getShortName());
|
||||||
|
|
||||||
// Compatible with 1.0: We use latch to make sure that this procedure implementation is
|
// Compatible with 1.0: We use latch to make sure that this procedure implementation is
|
||||||
// compatible with 1.0 asynchronized operations. We need to lock the table and check
|
// compatible with 1.0 asynchronized operations. We need to lock the table and check
|
||||||
@ -561,22 +561,16 @@ public class EnableTableProcedure
|
|||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
switch (state) {
|
||||||
@Override
|
case ENABLE_TABLE_PRE_OPERATION:
|
||||||
public Void run() throws Exception {
|
cpHost.preEnableTableAction(getTableName(), user);
|
||||||
switch (state) {
|
break;
|
||||||
case ENABLE_TABLE_PRE_OPERATION:
|
case ENABLE_TABLE_POST_OPERATION:
|
||||||
cpHost.preEnableTableAction(getTableName());
|
cpHost.postCompletedEnableTableAction(getTableName(), user);
|
||||||
break;
|
break;
|
||||||
case ENABLE_TABLE_POST_OPERATION:
|
default:
|
||||||
cpHost.postCompletedEnableTableAction(getTableName());
|
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||||
break;
|
}
|
||||||
default:
|
|
||||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
|
||||||
}
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFactory;
|
|||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
|
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@ -32,24 +33,24 @@ public final class MasterProcedureUtil {
|
|||||||
|
|
||||||
private MasterProcedureUtil() {}
|
private MasterProcedureUtil() {}
|
||||||
|
|
||||||
public static UserInformation toProtoUserInfo(UserGroupInformation ugi) {
|
public static UserInformation toProtoUserInfo(User user) {
|
||||||
UserInformation.Builder userInfoPB = UserInformation.newBuilder();
|
UserInformation.Builder userInfoPB = UserInformation.newBuilder();
|
||||||
userInfoPB.setEffectiveUser(ugi.getUserName());
|
userInfoPB.setEffectiveUser(user.getName());
|
||||||
if (ugi.getRealUser() != null) {
|
if (user.getUGI().getRealUser() != null) {
|
||||||
userInfoPB.setRealUser(ugi.getRealUser().getUserName());
|
userInfoPB.setRealUser(user.getUGI().getRealUser().getUserName());
|
||||||
}
|
}
|
||||||
return userInfoPB.build();
|
return userInfoPB.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static UserGroupInformation toUserInfo(UserInformation userInfoProto) {
|
public static User toUserInfo(UserInformation userInfoProto) {
|
||||||
if (userInfoProto.hasEffectiveUser()) {
|
if (userInfoProto.hasEffectiveUser()) {
|
||||||
String effectiveUser = userInfoProto.getEffectiveUser();
|
String effectiveUser = userInfoProto.getEffectiveUser();
|
||||||
if (userInfoProto.hasRealUser()) {
|
if (userInfoProto.hasRealUser()) {
|
||||||
String realUser = userInfoProto.getRealUser();
|
String realUser = userInfoProto.getRealUser();
|
||||||
UserGroupInformation realUserUgi = UserGroupInformation.createRemoteUser(realUser);
|
UserGroupInformation realUserUgi = UserGroupInformation.createRemoteUser(realUser);
|
||||||
return UserGroupInformation.createProxyUser(effectiveUser, realUserUgi);
|
return User.create(UserGroupInformation.createProxyUser(effectiveUser, realUserUgi));
|
||||||
}
|
}
|
||||||
return UserGroupInformation.createRemoteUser(effectiveUser);
|
return User.create(UserGroupInformation.createRemoteUser(effectiveUser));
|
||||||
}
|
}
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyState;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.hbase.security.User;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The procedure to modify a column family from an existing table.
|
* The procedure to modify a column family from an existing table.
|
||||||
@ -55,7 +55,7 @@ public class ModifyColumnFamilyProcedure
|
|||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
private HTableDescriptor unmodifiedHTableDescriptor;
|
private HTableDescriptor unmodifiedHTableDescriptor;
|
||||||
private HColumnDescriptor cfDescriptor;
|
private HColumnDescriptor cfDescriptor;
|
||||||
private UserGroupInformation user;
|
private User user;
|
||||||
|
|
||||||
private Boolean traceEnabled;
|
private Boolean traceEnabled;
|
||||||
|
|
||||||
@ -70,8 +70,8 @@ public class ModifyColumnFamilyProcedure
|
|||||||
final HColumnDescriptor cfDescriptor) throws IOException {
|
final HColumnDescriptor cfDescriptor) throws IOException {
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.cfDescriptor = cfDescriptor;
|
this.cfDescriptor = cfDescriptor;
|
||||||
this.user = env.getRequestUser().getUGI();
|
this.user = env.getRequestUser();
|
||||||
this.setOwner(this.user.getShortUserName());
|
this.setOwner(this.user.getShortName());
|
||||||
this.unmodifiedHTableDescriptor = null;
|
this.unmodifiedHTableDescriptor = null;
|
||||||
this.traceEnabled = null;
|
this.traceEnabled = null;
|
||||||
}
|
}
|
||||||
@ -359,22 +359,16 @@ public class ModifyColumnFamilyProcedure
|
|||||||
final ModifyColumnFamilyState state) throws IOException, InterruptedException {
|
final ModifyColumnFamilyState state) throws IOException, InterruptedException {
|
||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
switch (state) {
|
||||||
@Override
|
case MODIFY_COLUMN_FAMILY_PRE_OPERATION:
|
||||||
public Void run() throws Exception {
|
cpHost.preModifyColumnFamilyAction(tableName, cfDescriptor, user);
|
||||||
switch (state) {
|
break;
|
||||||
case MODIFY_COLUMN_FAMILY_PRE_OPERATION:
|
case MODIFY_COLUMN_FAMILY_POST_OPERATION:
|
||||||
cpHost.preModifyColumnFamilyAction(tableName, cfDescriptor);
|
cpHost.postCompletedModifyColumnFamilyAction(tableName, cfDescriptor, user);
|
||||||
break;
|
break;
|
||||||
case MODIFY_COLUMN_FAMILY_POST_OPERATION:
|
default:
|
||||||
cpHost.postCompletedModifyColumnFamilyAction(tableName, cfDescriptor);
|
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||||
break;
|
}
|
||||||
default:
|
|
||||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
|
||||||
}
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
@ -49,8 +49,8 @@ import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableState;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
|
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class ModifyTableProcedure
|
public class ModifyTableProcedure
|
||||||
@ -62,7 +62,7 @@ public class ModifyTableProcedure
|
|||||||
|
|
||||||
private HTableDescriptor unmodifiedHTableDescriptor = null;
|
private HTableDescriptor unmodifiedHTableDescriptor = null;
|
||||||
private HTableDescriptor modifiedHTableDescriptor;
|
private HTableDescriptor modifiedHTableDescriptor;
|
||||||
private UserGroupInformation user;
|
private User user;
|
||||||
private boolean deleteColumnFamilyInModify;
|
private boolean deleteColumnFamilyInModify;
|
||||||
|
|
||||||
private List<HRegionInfo> regionInfoList;
|
private List<HRegionInfo> regionInfoList;
|
||||||
@ -77,8 +77,8 @@ public class ModifyTableProcedure
|
|||||||
final HTableDescriptor htd) throws IOException {
|
final HTableDescriptor htd) throws IOException {
|
||||||
initilize();
|
initilize();
|
||||||
this.modifiedHTableDescriptor = htd;
|
this.modifiedHTableDescriptor = htd;
|
||||||
this.user = env.getRequestUser().getUGI();
|
this.user = env.getRequestUser();
|
||||||
this.setOwner(this.user.getShortUserName());
|
this.setOwner(this.user.getShortName());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void initilize() {
|
private void initilize() {
|
||||||
@ -467,22 +467,16 @@ public class ModifyTableProcedure
|
|||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
switch (state) {
|
||||||
@Override
|
case MODIFY_TABLE_PRE_OPERATION:
|
||||||
public Void run() throws Exception {
|
cpHost.preModifyTableAction(getTableName(), modifiedHTableDescriptor, user);
|
||||||
switch (state) {
|
break;
|
||||||
case MODIFY_TABLE_PRE_OPERATION:
|
case MODIFY_TABLE_POST_OPERATION:
|
||||||
cpHost.preModifyTableAction(getTableName(), modifiedHTableDescriptor);
|
cpHost.postCompletedModifyTableAction(getTableName(), modifiedHTableDescriptor, user);
|
||||||
break;
|
break;
|
||||||
case MODIFY_TABLE_POST_OPERATION:
|
default:
|
||||||
cpHost.postCompletedModifyTableAction(getTableName(), modifiedHTableDescriptor);
|
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||||
break;
|
}
|
||||||
default:
|
|
||||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
|
||||||
}
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -53,12 +53,12 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
|||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.RestoreSnapshotState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.RestoreSnapshotState;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
||||||
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
|
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
|
||||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||||
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
|
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class RestoreSnapshotProcedure
|
public class RestoreSnapshotProcedure
|
||||||
@ -75,7 +75,7 @@ public class RestoreSnapshotProcedure
|
|||||||
private Map<String, Pair<String, String>> parentsToChildrenPairMap =
|
private Map<String, Pair<String, String>> parentsToChildrenPairMap =
|
||||||
new HashMap<String, Pair<String, String>>();
|
new HashMap<String, Pair<String, String>>();
|
||||||
|
|
||||||
private UserGroupInformation user;
|
private User user;
|
||||||
private SnapshotDescription snapshot;
|
private SnapshotDescription snapshot;
|
||||||
|
|
||||||
// Monitor
|
// Monitor
|
||||||
@ -106,8 +106,8 @@ public class RestoreSnapshotProcedure
|
|||||||
// Snapshot information
|
// Snapshot information
|
||||||
this.snapshot = snapshot;
|
this.snapshot = snapshot;
|
||||||
// User and owner information
|
// User and owner information
|
||||||
this.user = env.getRequestUser().getUGI();
|
this.user = env.getRequestUser();
|
||||||
this.setOwner(this.user.getShortUserName());
|
this.setOwner(this.user.getShortName());
|
||||||
|
|
||||||
// Monitor
|
// Monitor
|
||||||
getMonitorStatus();
|
getMonitorStatus();
|
||||||
|
@ -41,8 +41,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
|||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class TruncateTableProcedure
|
public class TruncateTableProcedure
|
||||||
@ -52,7 +52,7 @@ public class TruncateTableProcedure
|
|||||||
|
|
||||||
private boolean preserveSplits;
|
private boolean preserveSplits;
|
||||||
private List<HRegionInfo> regions;
|
private List<HRegionInfo> regions;
|
||||||
private UserGroupInformation user;
|
private User user;
|
||||||
private HTableDescriptor hTableDescriptor;
|
private HTableDescriptor hTableDescriptor;
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
|
|
||||||
@ -64,8 +64,8 @@ public class TruncateTableProcedure
|
|||||||
boolean preserveSplits) throws IOException {
|
boolean preserveSplits) throws IOException {
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.preserveSplits = preserveSplits;
|
this.preserveSplits = preserveSplits;
|
||||||
this.user = env.getRequestUser().getUGI();
|
this.user = env.getRequestUser();
|
||||||
this.setOwner(this.user.getShortUserName());
|
this.setOwner(this.user.getShortName());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -261,13 +261,7 @@ public class TruncateTableProcedure
|
|||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final TableName tableName = getTableName();
|
final TableName tableName = getTableName();
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
cpHost.preTruncateTableAction(tableName, user);
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
cpHost.preTruncateTableAction(tableName);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@ -277,13 +271,7 @@ public class TruncateTableProcedure
|
|||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final TableName tableName = getTableName();
|
final TableName tableName = getTableName();
|
||||||
user.doAs(new PrivilegedExceptionAction<Void>() {
|
cpHost.postCompletedTruncateTableAction(tableName, user);
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
cpHost.postCompletedTruncateTableAction(tableName);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
@ -1284,23 +1284,7 @@ public class HStore implements Store {
|
|||||||
final StoreFile sf = moveFileIntoPlace(newFile);
|
final StoreFile sf = moveFileIntoPlace(newFile);
|
||||||
if (this.getCoprocessorHost() != null) {
|
if (this.getCoprocessorHost() != null) {
|
||||||
final Store thisStore = this;
|
final Store thisStore = this;
|
||||||
if (user == null) {
|
getCoprocessorHost().postCompact(thisStore, sf, cr, user);
|
||||||
getCoprocessorHost().postCompact(thisStore, sf, cr);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
|
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
getCoprocessorHost().postCompact(thisStore, sf, cr);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
assert sf != null;
|
assert sf != null;
|
||||||
sfs.add(sf);
|
sfs.add(sf);
|
||||||
@ -1507,7 +1491,7 @@ public class HStore implements Store {
|
|||||||
// Move the compaction into place.
|
// Move the compaction into place.
|
||||||
StoreFile sf = moveFileIntoPlace(newFile);
|
StoreFile sf = moveFileIntoPlace(newFile);
|
||||||
if (this.getCoprocessorHost() != null) {
|
if (this.getCoprocessorHost() != null) {
|
||||||
this.getCoprocessorHost().postCompact(this, sf, null);
|
this.getCoprocessorHost().postCompact(this, sf, null, null);
|
||||||
}
|
}
|
||||||
replaceStoreFiles(filesToCompact, Lists.newArrayList(sf));
|
replaceStoreFiles(filesToCompact, Lists.newArrayList(sf));
|
||||||
completeCompaction(filesToCompact);
|
completeCompaction(filesToCompact);
|
||||||
@ -1568,29 +1552,12 @@ public class HStore implements Store {
|
|||||||
this.lock.readLock().lock();
|
this.lock.readLock().lock();
|
||||||
try {
|
try {
|
||||||
synchronized (filesCompacting) {
|
synchronized (filesCompacting) {
|
||||||
final Store thisStore = this;
|
|
||||||
// First, see if coprocessor would want to override selection.
|
// First, see if coprocessor would want to override selection.
|
||||||
if (this.getCoprocessorHost() != null) {
|
if (this.getCoprocessorHost() != null) {
|
||||||
final List<StoreFile> candidatesForCoproc = compaction.preSelect(this.filesCompacting);
|
final List<StoreFile> candidatesForCoproc = compaction.preSelect(this.filesCompacting);
|
||||||
boolean override = false;
|
boolean override = false;
|
||||||
if (user == null) {
|
override = getCoprocessorHost().preCompactSelection(this, candidatesForCoproc,
|
||||||
override = getCoprocessorHost().preCompactSelection(this, candidatesForCoproc,
|
baseRequest, user);
|
||||||
baseRequest);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
override = user.getUGI().doAs(new PrivilegedExceptionAction<Boolean>() {
|
|
||||||
@Override
|
|
||||||
public Boolean run() throws Exception {
|
|
||||||
return getCoprocessorHost().preCompactSelection(thisStore, candidatesForCoproc,
|
|
||||||
baseRequest);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (override) {
|
if (override) {
|
||||||
// Coprocessor is overriding normal file selection.
|
// Coprocessor is overriding normal file selection.
|
||||||
compaction.forceSelect(new CompactionRequest(candidatesForCoproc));
|
compaction.forceSelect(new CompactionRequest(candidatesForCoproc));
|
||||||
@ -1618,25 +1585,8 @@ public class HStore implements Store {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (this.getCoprocessorHost() != null) {
|
if (this.getCoprocessorHost() != null) {
|
||||||
if (user == null) {
|
this.getCoprocessorHost().postCompactSelection(
|
||||||
this.getCoprocessorHost().postCompactSelection(
|
this, ImmutableList.copyOf(compaction.getRequest().getFiles()), baseRequest, user);
|
||||||
this, ImmutableList.copyOf(compaction.getRequest().getFiles()), baseRequest);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
|
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
getCoprocessorHost().postCompactSelection(
|
|
||||||
thisStore,ImmutableList.copyOf(compaction.getRequest().getFiles()),baseRequest);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Selected files; see if we have a compaction with some custom base request.
|
// Selected files; see if we have a compaction with some custom base request.
|
||||||
|
@ -74,9 +74,11 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
|||||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||||
import org.apache.hadoop.hbase.io.Reference;
|
import org.apache.hadoop.hbase.io.Reference;
|
||||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||||
|
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region.Operation;
|
import org.apache.hadoop.hbase.regionserver.Region.Operation;
|
||||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.wal.WALKey;
|
import org.apache.hadoop.hbase.wal.WALKey;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
@ -532,9 +534,9 @@ public class RegionCoprocessorHost
|
|||||||
*/
|
*/
|
||||||
public InternalScanner preCompactScannerOpen(final Store store,
|
public InternalScanner preCompactScannerOpen(final Store store,
|
||||||
final List<StoreFileScanner> scanners, final ScanType scanType, final long earliestPutTs,
|
final List<StoreFileScanner> scanners, final ScanType scanType, final long earliestPutTs,
|
||||||
final CompactionRequest request) throws IOException {
|
final CompactionRequest request, final User user) throws IOException {
|
||||||
return execOperationWithResult(null,
|
return execOperationWithResult(null,
|
||||||
coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>() {
|
coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -554,8 +556,8 @@ public class RegionCoprocessorHost
|
|||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public boolean preCompactSelection(final Store store, final List<StoreFile> candidates,
|
public boolean preCompactSelection(final Store store, final List<StoreFile> candidates,
|
||||||
final CompactionRequest request) throws IOException {
|
final CompactionRequest request, final User user) throws IOException {
|
||||||
return execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
return execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -572,9 +574,9 @@ public class RegionCoprocessorHost
|
|||||||
* @param request custom compaction
|
* @param request custom compaction
|
||||||
*/
|
*/
|
||||||
public void postCompactSelection(final Store store, final ImmutableList<StoreFile> selected,
|
public void postCompactSelection(final Store store, final ImmutableList<StoreFile> selected,
|
||||||
final CompactionRequest request) {
|
final CompactionRequest request, final User user) {
|
||||||
try {
|
try {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -595,9 +597,10 @@ public class RegionCoprocessorHost
|
|||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public InternalScanner preCompact(final Store store, final InternalScanner scanner,
|
public InternalScanner preCompact(final Store store, final InternalScanner scanner,
|
||||||
final ScanType scanType, final CompactionRequest request) throws IOException {
|
final ScanType scanType, final CompactionRequest request, final User user)
|
||||||
|
throws IOException {
|
||||||
return execOperationWithResult(false, scanner,
|
return execOperationWithResult(false, scanner,
|
||||||
coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>() {
|
coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -614,8 +617,8 @@ public class RegionCoprocessorHost
|
|||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public void postCompact(final Store store, final StoreFile resultFile,
|
public void postCompact(final Store store, final StoreFile resultFile,
|
||||||
final CompactionRequest request) throws IOException {
|
final CompactionRequest request, final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -704,8 +707,8 @@ public class RegionCoprocessorHost
|
|||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
// TODO: Deprecate this
|
// TODO: Deprecate this
|
||||||
public void preSplit() throws IOException {
|
public void preSplit(final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -718,8 +721,8 @@ public class RegionCoprocessorHost
|
|||||||
* Invoked just before a split
|
* Invoked just before a split
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public void preSplit(final byte[] splitRow) throws IOException {
|
public void preSplit(final byte[] splitRow, final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -734,8 +737,8 @@ public class RegionCoprocessorHost
|
|||||||
* @param r the new right-hand daughter region
|
* @param r the new right-hand daughter region
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public void postSplit(final Region l, final Region r) throws IOException {
|
public void postSplit(final Region l, final Region r, final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -745,8 +748,8 @@ public class RegionCoprocessorHost
|
|||||||
}
|
}
|
||||||
|
|
||||||
public boolean preSplitBeforePONR(final byte[] splitKey,
|
public boolean preSplitBeforePONR(final byte[] splitKey,
|
||||||
final List<Mutation> metaEntries) throws IOException {
|
final List<Mutation> metaEntries, final User user) throws IOException {
|
||||||
return execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
return execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -755,8 +758,8 @@ public class RegionCoprocessorHost
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void preSplitAfterPONR() throws IOException {
|
public void preSplitAfterPONR(final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -769,8 +772,8 @@ public class RegionCoprocessorHost
|
|||||||
* Invoked just before the rollback of a failed split is started
|
* Invoked just before the rollback of a failed split is started
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public void preRollBackSplit() throws IOException {
|
public void preRollBackSplit(final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -783,8 +786,8 @@ public class RegionCoprocessorHost
|
|||||||
* Invoked just after the rollback of a failed split is done
|
* Invoked just after the rollback of a failed split is done
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public void postRollBackSplit() throws IOException {
|
public void postRollBackSplit(final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -1656,6 +1659,14 @@ public class RegionCoprocessorHost
|
|||||||
|
|
||||||
private static abstract class CoprocessorOperation
|
private static abstract class CoprocessorOperation
|
||||||
extends ObserverContext<RegionCoprocessorEnvironment> {
|
extends ObserverContext<RegionCoprocessorEnvironment> {
|
||||||
|
public CoprocessorOperation() {
|
||||||
|
this(RpcServer.getRequestUser());
|
||||||
|
}
|
||||||
|
|
||||||
|
public CoprocessorOperation(User user) {
|
||||||
|
super(user);
|
||||||
|
}
|
||||||
|
|
||||||
public abstract void call(Coprocessor observer,
|
public abstract void call(Coprocessor observer,
|
||||||
ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException;
|
ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException;
|
||||||
public abstract boolean hasCall(Coprocessor observer);
|
public abstract boolean hasCall(Coprocessor observer);
|
||||||
@ -1663,6 +1674,13 @@ public class RegionCoprocessorHost
|
|||||||
}
|
}
|
||||||
|
|
||||||
private static abstract class RegionOperation extends CoprocessorOperation {
|
private static abstract class RegionOperation extends CoprocessorOperation {
|
||||||
|
public RegionOperation() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public RegionOperation(User user) {
|
||||||
|
super(user);
|
||||||
|
}
|
||||||
|
|
||||||
public abstract void call(RegionObserver observer,
|
public abstract void call(RegionObserver observer,
|
||||||
ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException;
|
ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException;
|
||||||
|
|
||||||
@ -1677,6 +1695,13 @@ public class RegionCoprocessorHost
|
|||||||
}
|
}
|
||||||
|
|
||||||
private static abstract class RegionOperationWithResult<T> extends RegionOperation {
|
private static abstract class RegionOperationWithResult<T> extends RegionOperation {
|
||||||
|
public RegionOperationWithResult() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public RegionOperationWithResult(User user) {
|
||||||
|
super (user);
|
||||||
|
}
|
||||||
|
|
||||||
private T result = null;
|
private T result = null;
|
||||||
public void setResult(final T result) { this.result = result; }
|
public void setResult(final T result) { this.result = result; }
|
||||||
public T getResult() { return this.result; }
|
public T getResult() { return this.result; }
|
||||||
|
@ -247,23 +247,7 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction {
|
|||||||
}
|
}
|
||||||
final HRegion mergedRegion = createMergedRegion(server, services, user);
|
final HRegion mergedRegion = createMergedRegion(server, services, user);
|
||||||
if (rsCoprocessorHost != null) {
|
if (rsCoprocessorHost != null) {
|
||||||
if (user == null) {
|
rsCoprocessorHost.postMergeCommit(this.region_a, this.region_b, mergedRegion, user);
|
||||||
rsCoprocessorHost.postMergeCommit(this.region_a, this.region_b, mergedRegion);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
|
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
rsCoprocessorHost.postMergeCommit(region_a, region_b, mergedRegion);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
stepsAfterPONR(server, services, mergedRegion, user);
|
stepsAfterPONR(server, services, mergedRegion, user);
|
||||||
|
|
||||||
@ -277,23 +261,7 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction {
|
|||||||
final HRegion mergedRegion, User user) throws IOException {
|
final HRegion mergedRegion, User user) throws IOException {
|
||||||
openMergedRegion(server, services, mergedRegion);
|
openMergedRegion(server, services, mergedRegion);
|
||||||
if (rsCoprocessorHost != null) {
|
if (rsCoprocessorHost != null) {
|
||||||
if (user == null) {
|
rsCoprocessorHost.postMerge(region_a, region_b, mergedRegion, user);
|
||||||
rsCoprocessorHost.postMerge(region_a, region_b, mergedRegion);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
|
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
rsCoprocessorHost.postMerge(region_a, region_b, mergedRegion);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -315,23 +283,7 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction {
|
|||||||
}
|
}
|
||||||
|
|
||||||
if (rsCoprocessorHost != null) {
|
if (rsCoprocessorHost != null) {
|
||||||
boolean ret = false;
|
boolean ret = rsCoprocessorHost.preMerge(region_a, region_b, user);
|
||||||
if (user == null) {
|
|
||||||
ret = rsCoprocessorHost.preMerge(region_a, region_b);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
ret = user.getUGI().doAs(new PrivilegedExceptionAction<Boolean>() {
|
|
||||||
@Override
|
|
||||||
public Boolean run() throws Exception {
|
|
||||||
return rsCoprocessorHost.preMerge(region_a, region_b);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (ret) {
|
if (ret) {
|
||||||
throw new IOException("Coprocessor bypassing regions " + this.region_a + " "
|
throw new IOException("Coprocessor bypassing regions " + this.region_a + " "
|
||||||
+ this.region_b + " merge.");
|
+ this.region_b + " merge.");
|
||||||
@ -347,23 +299,7 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction {
|
|||||||
@MetaMutationAnnotation
|
@MetaMutationAnnotation
|
||||||
final List<Mutation> metaEntries = new ArrayList<Mutation>();
|
final List<Mutation> metaEntries = new ArrayList<Mutation>();
|
||||||
if (rsCoprocessorHost != null) {
|
if (rsCoprocessorHost != null) {
|
||||||
boolean ret = false;
|
boolean ret = rsCoprocessorHost.preMergeCommit(region_a, region_b, metaEntries, user);
|
||||||
if (user == null) {
|
|
||||||
ret = rsCoprocessorHost.preMergeCommit(region_a, region_b, metaEntries);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
ret = user.getUGI().doAs(new PrivilegedExceptionAction<Boolean>() {
|
|
||||||
@Override
|
|
||||||
public Boolean run() throws Exception {
|
|
||||||
return rsCoprocessorHost.preMergeCommit(region_a, region_b, metaEntries);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (ret) {
|
if (ret) {
|
||||||
throw new IOException("Coprocessor bypassing regions " + this.region_a + " "
|
throw new IOException("Coprocessor bypassing regions " + this.region_a + " "
|
||||||
@ -658,23 +594,7 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction {
|
|||||||
this.rsServices = services;
|
this.rsServices = services;
|
||||||
// Coprocessor callback
|
// Coprocessor callback
|
||||||
if (rsCoprocessorHost != null) {
|
if (rsCoprocessorHost != null) {
|
||||||
if (user == null) {
|
rsCoprocessorHost.preRollBackMerge(region_a, region_b, user);
|
||||||
rsCoprocessorHost.preRollBackMerge(region_a, region_b);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
|
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
rsCoprocessorHost.preRollBackMerge(region_a, region_b);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
boolean result = true;
|
boolean result = true;
|
||||||
@ -759,23 +679,7 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction {
|
|||||||
}
|
}
|
||||||
// Coprocessor callback
|
// Coprocessor callback
|
||||||
if (rsCoprocessorHost != null) {
|
if (rsCoprocessorHost != null) {
|
||||||
if (user == null) {
|
rsCoprocessorHost.postRollBackMerge(region_a, region_b, user);
|
||||||
rsCoprocessorHost.postRollBackMerge(region_a, region_b);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
|
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
rsCoprocessorHost.postRollBackMerge(region_a, region_b);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
|
@ -39,8 +39,10 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
|||||||
import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
|
import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
|
||||||
import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
|
import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
|
||||||
import org.apache.hadoop.hbase.coprocessor.SingletonCoprocessorService;
|
import org.apache.hadoop.hbase.coprocessor.SingletonCoprocessorService;
|
||||||
|
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
|
||||||
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
|
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
|
|
||||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||||
@InterfaceStability.Evolving
|
@InterfaceStability.Evolving
|
||||||
@ -91,8 +93,8 @@ public class RegionServerCoprocessorHost extends
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean preMerge(final HRegion regionA, final HRegion regionB) throws IOException {
|
public boolean preMerge(final HRegion regionA, final HRegion regionB, final User user) throws IOException {
|
||||||
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionServerObserver oserver,
|
public void call(RegionServerObserver oserver,
|
||||||
ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
|
ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
|
||||||
@ -101,9 +103,10 @@ public class RegionServerCoprocessorHost extends
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void postMerge(final HRegion regionA, final HRegion regionB, final HRegion mergedRegion)
|
public void postMerge(final HRegion regionA, final HRegion regionB, final HRegion mergedRegion,
|
||||||
|
final User user)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionServerObserver oserver,
|
public void call(RegionServerObserver oserver,
|
||||||
ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
|
ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
|
||||||
@ -113,8 +116,9 @@ public class RegionServerCoprocessorHost extends
|
|||||||
}
|
}
|
||||||
|
|
||||||
public boolean preMergeCommit(final HRegion regionA, final HRegion regionB,
|
public boolean preMergeCommit(final HRegion regionA, final HRegion regionB,
|
||||||
final @MetaMutationAnnotation List<Mutation> metaEntries) throws IOException {
|
final @MetaMutationAnnotation List<Mutation> metaEntries, final User user)
|
||||||
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
throws IOException {
|
||||||
|
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionServerObserver oserver,
|
public void call(RegionServerObserver oserver,
|
||||||
ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
|
ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
|
||||||
@ -124,8 +128,8 @@ public class RegionServerCoprocessorHost extends
|
|||||||
}
|
}
|
||||||
|
|
||||||
public void postMergeCommit(final HRegion regionA, final HRegion regionB,
|
public void postMergeCommit(final HRegion regionA, final HRegion regionB,
|
||||||
final HRegion mergedRegion) throws IOException {
|
final HRegion mergedRegion, final User user) throws IOException {
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionServerObserver oserver,
|
public void call(RegionServerObserver oserver,
|
||||||
ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
|
ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
|
||||||
@ -134,8 +138,9 @@ public class RegionServerCoprocessorHost extends
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void preRollBackMerge(final HRegion regionA, final HRegion regionB) throws IOException {
|
public void preRollBackMerge(final HRegion regionA, final HRegion regionB, final User user)
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
throws IOException {
|
||||||
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionServerObserver oserver,
|
public void call(RegionServerObserver oserver,
|
||||||
ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
|
ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
|
||||||
@ -144,8 +149,9 @@ public class RegionServerCoprocessorHost extends
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void postRollBackMerge(final HRegion regionA, final HRegion regionB) throws IOException {
|
public void postRollBackMerge(final HRegion regionA, final HRegion regionB, final User user)
|
||||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
throws IOException {
|
||||||
|
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||||
@Override
|
@Override
|
||||||
public void call(RegionServerObserver oserver,
|
public void call(RegionServerObserver oserver,
|
||||||
ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
|
ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
|
||||||
@ -220,6 +226,11 @@ public class RegionServerCoprocessorHost extends
|
|||||||
private static abstract class CoprocessorOperation
|
private static abstract class CoprocessorOperation
|
||||||
extends ObserverContext<RegionServerCoprocessorEnvironment> {
|
extends ObserverContext<RegionServerCoprocessorEnvironment> {
|
||||||
public CoprocessorOperation() {
|
public CoprocessorOperation() {
|
||||||
|
this(RpcServer.getRequestUser());
|
||||||
|
}
|
||||||
|
|
||||||
|
public CoprocessorOperation(User user) {
|
||||||
|
super(user);
|
||||||
}
|
}
|
||||||
|
|
||||||
public abstract void call(RegionServerObserver oserver,
|
public abstract void call(RegionServerObserver oserver,
|
||||||
|
@ -151,7 +151,7 @@ public class SecureBulkLoadManager {
|
|||||||
|
|
||||||
if (bulkLoadObservers != null && bulkLoadObservers.size() != 0) {
|
if (bulkLoadObservers != null && bulkLoadObservers.size() != 0) {
|
||||||
ObserverContext<RegionCoprocessorEnvironment> ctx =
|
ObserverContext<RegionCoprocessorEnvironment> ctx =
|
||||||
new ObserverContext<RegionCoprocessorEnvironment>();
|
new ObserverContext<RegionCoprocessorEnvironment>(getActiveUser());
|
||||||
ctx.prepare((RegionCoprocessorEnvironment) region.getCoprocessorHost()
|
ctx.prepare((RegionCoprocessorEnvironment) region.getCoprocessorHost()
|
||||||
.findCoprocessorEnvironment(BulkLoadObserver.class).get(0));
|
.findCoprocessorEnvironment(BulkLoadObserver.class).get(0));
|
||||||
|
|
||||||
@ -173,7 +173,7 @@ public class SecureBulkLoadManager {
|
|||||||
|
|
||||||
if (bulkLoadObservers != null && bulkLoadObservers.size() != 0) {
|
if (bulkLoadObservers != null && bulkLoadObservers.size() != 0) {
|
||||||
ObserverContext<RegionCoprocessorEnvironment> ctx =
|
ObserverContext<RegionCoprocessorEnvironment> ctx =
|
||||||
new ObserverContext<RegionCoprocessorEnvironment>();
|
new ObserverContext<RegionCoprocessorEnvironment>(getActiveUser());
|
||||||
ctx.prepare((RegionCoprocessorEnvironment) region.getCoprocessorHost()
|
ctx.prepare((RegionCoprocessorEnvironment) region.getCoprocessorHost()
|
||||||
.findCoprocessorEnvironment(BulkLoadObserver.class).get(0));
|
.findCoprocessorEnvironment(BulkLoadObserver.class).get(0));
|
||||||
|
|
||||||
|
@ -228,26 +228,9 @@ public class SplitTransactionImpl implements SplitTransaction {
|
|||||||
|
|
||||||
// Coprocessor callback
|
// Coprocessor callback
|
||||||
if (this.parent.getCoprocessorHost() != null) {
|
if (this.parent.getCoprocessorHost() != null) {
|
||||||
if (user == null) {
|
// TODO: Remove one of these
|
||||||
// TODO: Remove one of these
|
parent.getCoprocessorHost().preSplit(user);
|
||||||
parent.getCoprocessorHost().preSplit();
|
parent.getCoprocessorHost().preSplit(splitrow, user);
|
||||||
parent.getCoprocessorHost().preSplit(splitrow);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
|
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
parent.getCoprocessorHost().preSplit();
|
|
||||||
parent.getCoprocessorHost().preSplit(splitrow);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
transition(SplitTransactionPhase.AFTER_PRE_SPLIT_HOOK);
|
transition(SplitTransactionPhase.AFTER_PRE_SPLIT_HOOK);
|
||||||
@ -264,22 +247,7 @@ public class SplitTransactionImpl implements SplitTransaction {
|
|||||||
final List<Mutation> metaEntries = new ArrayList<Mutation>();
|
final List<Mutation> metaEntries = new ArrayList<Mutation>();
|
||||||
boolean ret = false;
|
boolean ret = false;
|
||||||
if (this.parent.getCoprocessorHost() != null) {
|
if (this.parent.getCoprocessorHost() != null) {
|
||||||
if (user == null) {
|
ret = parent.getCoprocessorHost().preSplitBeforePONR(splitrow, metaEntries, user);
|
||||||
ret = parent.getCoprocessorHost().preSplitBeforePONR(splitrow, metaEntries);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
ret = user.getUGI().doAs(new PrivilegedExceptionAction<Boolean>() {
|
|
||||||
@Override
|
|
||||||
public Boolean run() throws Exception {
|
|
||||||
return parent.getCoprocessorHost().preSplitBeforePONR(splitrow, metaEntries);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (ret) {
|
if (ret) {
|
||||||
throw new IOException("Coprocessor bypassing region "
|
throw new IOException("Coprocessor bypassing region "
|
||||||
+ parent.getRegionInfo().getRegionNameAsString() + " split.");
|
+ parent.getRegionInfo().getRegionNameAsString() + " split.");
|
||||||
@ -510,23 +478,7 @@ public class SplitTransactionImpl implements SplitTransaction {
|
|||||||
final RegionServerServices services, final PairOfSameType<Region> regions, User user)
|
final RegionServerServices services, final PairOfSameType<Region> regions, User user)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (this.parent.getCoprocessorHost() != null) {
|
if (this.parent.getCoprocessorHost() != null) {
|
||||||
if (user == null) {
|
parent.getCoprocessorHost().preSplitAfterPONR(user);
|
||||||
parent.getCoprocessorHost().preSplitAfterPONR();
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
|
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
parent.getCoprocessorHost().preSplitAfterPONR();
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
openDaughters(server, services, regions.getFirst(), regions.getSecond());
|
openDaughters(server, services, regions.getFirst(), regions.getSecond());
|
||||||
@ -535,23 +487,7 @@ public class SplitTransactionImpl implements SplitTransaction {
|
|||||||
|
|
||||||
// Coprocessor callback
|
// Coprocessor callback
|
||||||
if (parent.getCoprocessorHost() != null) {
|
if (parent.getCoprocessorHost() != null) {
|
||||||
if (user == null) {
|
this.parent.getCoprocessorHost().postSplit(regions.getFirst(), regions.getSecond(), user);
|
||||||
this.parent.getCoprocessorHost().postSplit(regions.getFirst(), regions.getSecond());
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
|
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
parent.getCoprocessorHost().postSplit(regions.getFirst(), regions.getSecond());
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
transition(SplitTransactionPhase.AFTER_POST_SPLIT_HOOK);
|
transition(SplitTransactionPhase.AFTER_POST_SPLIT_HOOK);
|
||||||
@ -780,23 +716,7 @@ public class SplitTransactionImpl implements SplitTransaction {
|
|||||||
this.rsServices = services;
|
this.rsServices = services;
|
||||||
// Coprocessor callback
|
// Coprocessor callback
|
||||||
if (this.parent.getCoprocessorHost() != null) {
|
if (this.parent.getCoprocessorHost() != null) {
|
||||||
if (user == null) {
|
this.parent.getCoprocessorHost().preRollBackSplit(user);
|
||||||
this.parent.getCoprocessorHost().preRollBackSplit();
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
|
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
parent.getCoprocessorHost().preRollBackSplit();
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
boolean result = true;
|
boolean result = true;
|
||||||
@ -875,23 +795,7 @@ public class SplitTransactionImpl implements SplitTransaction {
|
|||||||
}
|
}
|
||||||
// Coprocessor callback
|
// Coprocessor callback
|
||||||
if (this.parent.getCoprocessorHost() != null) {
|
if (this.parent.getCoprocessorHost() != null) {
|
||||||
if (user == null) {
|
this.parent.getCoprocessorHost().postRollBackSplit(user);
|
||||||
this.parent.getCoprocessorHost().postRollBackSplit();
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
|
|
||||||
@Override
|
|
||||||
public Void run() throws Exception {
|
|
||||||
parent.getCoprocessorHost().postRollBackSplit();
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
@ -357,24 +357,8 @@ public abstract class Compactor<T extends CellSink> {
|
|||||||
if (store.getCoprocessorHost() == null) {
|
if (store.getCoprocessorHost() == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
if (user == null) {
|
return store.getCoprocessorHost().preCompactScannerOpen(store, scanners, scanType,
|
||||||
return store.getCoprocessorHost().preCompactScannerOpen(store, scanners, scanType,
|
earliestPutTs, request, user);
|
||||||
earliestPutTs, request);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
return user.getUGI().doAs(new PrivilegedExceptionAction<InternalScanner>() {
|
|
||||||
@Override
|
|
||||||
public InternalScanner run() throws Exception {
|
|
||||||
return store.getCoprocessorHost().preCompactScannerOpen(store, scanners,
|
|
||||||
scanType, earliestPutTs, request);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -389,22 +373,7 @@ public abstract class Compactor<T extends CellSink> {
|
|||||||
if (store.getCoprocessorHost() == null) {
|
if (store.getCoprocessorHost() == null) {
|
||||||
return scanner;
|
return scanner;
|
||||||
}
|
}
|
||||||
if (user == null) {
|
return store.getCoprocessorHost().preCompact(store, scanner, scanType, request, user);
|
||||||
return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
return user.getUGI().doAs(new PrivilegedExceptionAction<InternalScanner>() {
|
|
||||||
@Override
|
|
||||||
public InternalScanner run() throws Exception {
|
|
||||||
return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
InterruptedIOException iioe = new InterruptedIOException();
|
|
||||||
iioe.initCause(ie);
|
|
||||||
throw iioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -414,8 +414,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
* If we are in the context of an RPC call, the remote user is used,
|
* If we are in the context of an RPC call, the remote user is used,
|
||||||
* otherwise the currently logged in user is used.
|
* otherwise the currently logged in user is used.
|
||||||
*/
|
*/
|
||||||
private User getActiveUser() throws IOException {
|
private User getActiveUser(ObserverContext ctx) throws IOException {
|
||||||
User user = RpcServer.getRequestUser();
|
User user = ctx.getCaller();
|
||||||
if (user == null) {
|
if (user == null) {
|
||||||
// for non-rpc handling, fallback to system user
|
// for non-rpc handling, fallback to system user
|
||||||
user = userProvider.getCurrent();
|
user = userProvider.getCurrent();
|
||||||
@ -432,9 +432,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
* @throws IOException if obtaining the current user fails
|
* @throws IOException if obtaining the current user fails
|
||||||
* @throws AccessDeniedException if user has no authorization
|
* @throws AccessDeniedException if user has no authorization
|
||||||
*/
|
*/
|
||||||
private void requirePermission(String request, TableName tableName, byte[] family,
|
private void requirePermission(User user, String request, TableName tableName, byte[] family,
|
||||||
byte[] qualifier, Action... permissions) throws IOException {
|
byte[] qualifier, Action... permissions) throws IOException {
|
||||||
User user = getActiveUser();
|
|
||||||
AuthResult result = null;
|
AuthResult result = null;
|
||||||
|
|
||||||
for (Action permission : permissions) {
|
for (Action permission : permissions) {
|
||||||
@ -463,9 +462,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
* @throws IOException if obtaining the current user fails
|
* @throws IOException if obtaining the current user fails
|
||||||
* @throws AccessDeniedException if user has no authorization
|
* @throws AccessDeniedException if user has no authorization
|
||||||
*/
|
*/
|
||||||
private void requireTablePermission(String request, TableName tableName, byte[] family,
|
private void requireTablePermission(User user, String request, TableName tableName, byte[] family,
|
||||||
byte[] qualifier, Action... permissions) throws IOException {
|
byte[] qualifier, Action... permissions) throws IOException {
|
||||||
User user = getActiveUser();
|
|
||||||
AuthResult result = null;
|
AuthResult result = null;
|
||||||
|
|
||||||
for (Action permission : permissions) {
|
for (Action permission : permissions) {
|
||||||
@ -495,9 +493,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
* @throws IOException if obtaining the current user fails
|
* @throws IOException if obtaining the current user fails
|
||||||
* @throws AccessDeniedException if user has no authorization
|
* @throws AccessDeniedException if user has no authorization
|
||||||
*/
|
*/
|
||||||
private void requireAccess(String request, TableName tableName,
|
private void requireAccess(User user, String request, TableName tableName,
|
||||||
Action... permissions) throws IOException {
|
Action... permissions) throws IOException {
|
||||||
User user = getActiveUser();
|
|
||||||
AuthResult result = null;
|
AuthResult result = null;
|
||||||
|
|
||||||
for (Action permission : permissions) {
|
for (Action permission : permissions) {
|
||||||
@ -523,8 +520,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
* @throws IOException if obtaining the current user fails
|
* @throws IOException if obtaining the current user fails
|
||||||
* @throws AccessDeniedException if authorization is denied
|
* @throws AccessDeniedException if authorization is denied
|
||||||
*/
|
*/
|
||||||
private void requirePermission(String request, Action perm) throws IOException {
|
private void requirePermission(User user, String request, Action perm) throws IOException {
|
||||||
requireGlobalPermission(request, perm, null, null);
|
requireGlobalPermission(user, request, perm, null, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -535,9 +532,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
* @param tableName Affected table name.
|
* @param tableName Affected table name.
|
||||||
* @param familyMap Affected column families.
|
* @param familyMap Affected column families.
|
||||||
*/
|
*/
|
||||||
private void requireGlobalPermission(String request, Action perm, TableName tableName,
|
private void requireGlobalPermission(User user, String request, Action perm, TableName tableName,
|
||||||
Map<byte[], ? extends Collection<byte[]>> familyMap) throws IOException {
|
Map<byte[], ? extends Collection<byte[]>> familyMap) throws IOException {
|
||||||
User user = getActiveUser();
|
|
||||||
AuthResult result = null;
|
AuthResult result = null;
|
||||||
if (authManager.authorize(user, perm)) {
|
if (authManager.authorize(user, perm)) {
|
||||||
result = AuthResult.allow(request, "Global check allowed", user, perm, tableName, familyMap);
|
result = AuthResult.allow(request, "Global check allowed", user, perm, tableName, familyMap);
|
||||||
@ -562,9 +558,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
* @param perm Action being requested
|
* @param perm Action being requested
|
||||||
* @param namespace
|
* @param namespace
|
||||||
*/
|
*/
|
||||||
private void requireGlobalPermission(String request, Action perm,
|
private void requireGlobalPermission(User user, String request, Action perm,
|
||||||
String namespace) throws IOException {
|
String namespace) throws IOException {
|
||||||
User user = getActiveUser();
|
|
||||||
AuthResult authResult = null;
|
AuthResult authResult = null;
|
||||||
if (authManager.authorize(user, perm)) {
|
if (authManager.authorize(user, perm)) {
|
||||||
authResult = AuthResult.allow(request, "Global check allowed", user, perm, null);
|
authResult = AuthResult.allow(request, "Global check allowed", user, perm, null);
|
||||||
@ -587,9 +582,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
* @param namespace
|
* @param namespace
|
||||||
* @param permissions Actions being requested
|
* @param permissions Actions being requested
|
||||||
*/
|
*/
|
||||||
public void requireNamespacePermission(String request, String namespace,
|
public void requireNamespacePermission(User user, String request, String namespace,
|
||||||
Action... permissions) throws IOException {
|
Action... permissions) throws IOException {
|
||||||
User user = getActiveUser();
|
|
||||||
AuthResult result = null;
|
AuthResult result = null;
|
||||||
|
|
||||||
for (Action permission : permissions) {
|
for (Action permission : permissions) {
|
||||||
@ -615,10 +609,10 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
* @param namespace
|
* @param namespace
|
||||||
* @param permissions Actions being requested
|
* @param permissions Actions being requested
|
||||||
*/
|
*/
|
||||||
public void requireNamespacePermission(String request, String namespace, TableName tableName,
|
public void requireNamespacePermission(User user, String request, String namespace,
|
||||||
Map<byte[], ? extends Collection<byte[]>> familyMap, Action... permissions)
|
TableName tableName, Map<byte[], ? extends Collection<byte[]>> familyMap,
|
||||||
|
Action... permissions)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
User user = getActiveUser();
|
|
||||||
AuthResult result = null;
|
AuthResult result = null;
|
||||||
|
|
||||||
for (Action permission : permissions) {
|
for (Action permission : permissions) {
|
||||||
@ -711,14 +705,13 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
* @return false if cell ACLs failed to grant access, true otherwise
|
* @return false if cell ACLs failed to grant access, true otherwise
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private boolean checkCoveringPermission(OpType request, RegionCoprocessorEnvironment e,
|
private boolean checkCoveringPermission(User user, OpType request, RegionCoprocessorEnvironment e,
|
||||||
byte[] row, Map<byte[], ? extends Collection<?>> familyMap, long opTs, Action... actions)
|
byte[] row, Map<byte[], ? extends Collection<?>> familyMap, long opTs, Action... actions)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (!cellFeaturesEnabled) {
|
if (!cellFeaturesEnabled) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
long cellGrants = 0;
|
long cellGrants = 0;
|
||||||
User user = getActiveUser();
|
|
||||||
long latestCellTs = 0;
|
long latestCellTs = 0;
|
||||||
Get get = new Get(row);
|
Get get = new Get(row);
|
||||||
// Only in case of Put/Delete op, consider TS within cell (if set for individual cells).
|
// Only in case of Put/Delete op, consider TS within cell (if set for individual cells).
|
||||||
@ -1000,8 +993,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
for (byte[] family: families) {
|
for (byte[] family: families) {
|
||||||
familyMap.put(family, null);
|
familyMap.put(family, null);
|
||||||
}
|
}
|
||||||
requireNamespacePermission("createTable", desc.getTableName().getNamespaceAsString(),
|
requireNamespacePermission(getActiveUser(c), "createTable",
|
||||||
desc.getTableName(), familyMap, Action.CREATE);
|
desc.getTableName().getNamespaceAsString(), desc.getTableName(), familyMap, Action.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -1035,7 +1028,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
String owner = desc.getOwnerString();
|
String owner = desc.getOwnerString();
|
||||||
// default the table owner to current user, if not specified.
|
// default the table owner to current user, if not specified.
|
||||||
if (owner == null)
|
if (owner == null)
|
||||||
owner = getActiveUser().getShortName();
|
owner = getActiveUser(c).getShortName();
|
||||||
final UserPermission userperm = new UserPermission(Bytes.toBytes(owner),
|
final UserPermission userperm = new UserPermission(Bytes.toBytes(owner),
|
||||||
desc.getTableName(), null, Action.values());
|
desc.getTableName(), null, Action.values());
|
||||||
// switch to the real hbase master user for doing the RPC on the ACL table
|
// switch to the real hbase master user for doing the RPC on the ACL table
|
||||||
@ -1054,7 +1047,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName)
|
public void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("deleteTable", tableName, null, null, Action.ADMIN, Action.CREATE);
|
requirePermission(getActiveUser(c), "deleteTable", tableName, null, null,
|
||||||
|
Action.ADMIN, Action.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -1074,7 +1068,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> c,
|
public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> c,
|
||||||
final TableName tableName) throws IOException {
|
final TableName tableName) throws IOException {
|
||||||
requirePermission("truncateTable", tableName, null, null, Action.ADMIN, Action.CREATE);
|
requirePermission(getActiveUser(c), "truncateTable", tableName, null, null,
|
||||||
|
Action.ADMIN, Action.CREATE);
|
||||||
|
|
||||||
final Configuration conf = c.getEnvironment().getConfiguration();
|
final Configuration conf = c.getEnvironment().getConfiguration();
|
||||||
User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
|
User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
|
||||||
@ -1111,7 +1106,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName,
|
public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName,
|
||||||
HTableDescriptor htd) throws IOException {
|
HTableDescriptor htd) throws IOException {
|
||||||
requirePermission("modifyTable", tableName, null, null, Action.ADMIN, Action.CREATE);
|
requirePermission(getActiveUser(c), "modifyTable", tableName, null, null,
|
||||||
|
Action.ADMIN, Action.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -1120,7 +1116,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
final Configuration conf = c.getEnvironment().getConfiguration();
|
final Configuration conf = c.getEnvironment().getConfiguration();
|
||||||
// default the table owner to current user, if not specified.
|
// default the table owner to current user, if not specified.
|
||||||
final String owner = (htd.getOwnerString() != null) ? htd.getOwnerString() :
|
final String owner = (htd.getOwnerString() != null) ? htd.getOwnerString() :
|
||||||
getActiveUser().getShortName();
|
getActiveUser(c).getShortName();
|
||||||
User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
|
User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
|
||||||
@Override
|
@Override
|
||||||
public Void run() throws Exception {
|
public Void run() throws Exception {
|
||||||
@ -1136,22 +1132,23 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
public void preAddColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preAddColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
TableName tableName, HColumnDescriptor columnFamily)
|
TableName tableName, HColumnDescriptor columnFamily)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requireTablePermission("addColumn", tableName, columnFamily.getName(), null, Action.ADMIN,
|
requireTablePermission(getActiveUser(ctx), "addColumn", tableName, columnFamily.getName(), null,
|
||||||
Action.CREATE);
|
Action.ADMIN, Action.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preModifyColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preModifyColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
TableName tableName, HColumnDescriptor columnFamily)
|
TableName tableName, HColumnDescriptor columnFamily)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("modifyColumn", tableName, columnFamily.getName(), null, Action.ADMIN,
|
requirePermission(getActiveUser(ctx), "modifyColumn", tableName, columnFamily.getName(), null,
|
||||||
Action.CREATE);
|
Action.ADMIN, Action.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preDeleteColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preDeleteColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
TableName tableName, byte[] columnFamily) throws IOException {
|
TableName tableName, byte[] columnFamily) throws IOException {
|
||||||
requirePermission("deleteColumn", tableName, columnFamily, null, Action.ADMIN, Action.CREATE);
|
requirePermission(getActiveUser(ctx), "deleteColumn", tableName, columnFamily, null,
|
||||||
|
Action.ADMIN, Action.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -1170,7 +1167,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName)
|
public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("enableTable", tableName, null, null, Action.ADMIN, Action.CREATE);
|
requirePermission(getActiveUser(c), "enableTable", tableName, null, null,
|
||||||
|
Action.ADMIN, Action.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -1184,7 +1182,8 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
throw new AccessDeniedException("Not allowed to disable "
|
throw new AccessDeniedException("Not allowed to disable "
|
||||||
+ AccessControlLists.ACL_TABLE_NAME + " table with AccessController installed");
|
+ AccessControlLists.ACL_TABLE_NAME + " table with AccessController installed");
|
||||||
}
|
}
|
||||||
requirePermission("disableTable", tableName, null, null, Action.ADMIN, Action.CREATE);
|
requirePermission(getActiveUser(c), "disableTable", tableName, null, null,
|
||||||
|
Action.ADMIN, Action.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -1192,10 +1191,10 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
ObserverContext<MasterCoprocessorEnvironment> ctx,
|
ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procEnv,
|
final ProcedureExecutor<MasterProcedureEnv> procEnv,
|
||||||
final long procId) throws IOException {
|
final long procId) throws IOException {
|
||||||
if (!procEnv.isProcedureOwner(procId, getActiveUser())) {
|
if (!procEnv.isProcedureOwner(procId, getActiveUser(ctx))) {
|
||||||
// If the user is not the procedure owner, then we should further probe whether
|
// If the user is not the procedure owner, then we should further probe whether
|
||||||
// he can abort the procedure.
|
// he can abort the procedure.
|
||||||
requirePermission("abortProcedure", Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "abortProcedure", Action.ADMIN);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1223,14 +1222,14 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
// Retains only those which passes authorization checks, as the checks weren't done as part
|
// Retains only those which passes authorization checks, as the checks weren't done as part
|
||||||
// of preListProcedures.
|
// of preListProcedures.
|
||||||
Iterator<ProcedureInfo> itr = procInfoList.iterator();
|
Iterator<ProcedureInfo> itr = procInfoList.iterator();
|
||||||
User user = getActiveUser();
|
User user = getActiveUser(ctx);
|
||||||
while (itr.hasNext()) {
|
while (itr.hasNext()) {
|
||||||
ProcedureInfo procInfo = itr.next();
|
ProcedureInfo procInfo = itr.next();
|
||||||
try {
|
try {
|
||||||
if (!ProcedureInfo.isProcedureOwner(procInfo, user)) {
|
if (!ProcedureInfo.isProcedureOwner(procInfo, user)) {
|
||||||
// If the user is not the procedure owner, then we should further probe whether
|
// If the user is not the procedure owner, then we should further probe whether
|
||||||
// he can see the procedure.
|
// he can see the procedure.
|
||||||
requirePermission("listProcedures", Action.ADMIN);
|
requirePermission(user, "listProcedures", Action.ADMIN);
|
||||||
}
|
}
|
||||||
} catch (AccessDeniedException e) {
|
} catch (AccessDeniedException e) {
|
||||||
itr.remove();
|
itr.remove();
|
||||||
@ -1241,31 +1240,32 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo region,
|
public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo region,
|
||||||
ServerName srcServer, ServerName destServer) throws IOException {
|
ServerName srcServer, ServerName destServer) throws IOException {
|
||||||
requirePermission("move", region.getTable(), null, null, Action.ADMIN);
|
requirePermission(getActiveUser(c), "move", region.getTable(), null, null, Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo)
|
public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("assign", regionInfo.getTable(), null, null, Action.ADMIN);
|
requirePermission(getActiveUser(c), "assign", regionInfo.getTable(), null, null, Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo,
|
public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo,
|
||||||
boolean force) throws IOException {
|
boolean force) throws IOException {
|
||||||
requirePermission("unassign", regionInfo.getTable(), null, null, Action.ADMIN);
|
requirePermission(getActiveUser(c), "unassign", regionInfo.getTable(), null, null, Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> c,
|
public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> c,
|
||||||
HRegionInfo regionInfo) throws IOException {
|
HRegionInfo regionInfo) throws IOException {
|
||||||
requirePermission("regionOffline", regionInfo.getTable(), null, null, Action.ADMIN);
|
requirePermission(getActiveUser(c), "regionOffline", regionInfo.getTable(), null, null,
|
||||||
|
Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final boolean newValue, final MasterSwitchType switchType) throws IOException {
|
final boolean newValue, final MasterSwitchType switchType) throws IOException {
|
||||||
requirePermission("setSplitOrMergeEnabled", Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "setSplitOrMergeEnabled", Action.ADMIN);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1277,26 +1277,26 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preBalance(ObserverContext<MasterCoprocessorEnvironment> c)
|
public void preBalance(ObserverContext<MasterCoprocessorEnvironment> c)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("balance", Action.ADMIN);
|
requirePermission(getActiveUser(c), "balance", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> c,
|
public boolean preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> c,
|
||||||
boolean newValue) throws IOException {
|
boolean newValue) throws IOException {
|
||||||
requirePermission("balanceSwitch", Action.ADMIN);
|
requirePermission(getActiveUser(c), "balanceSwitch", Action.ADMIN);
|
||||||
return newValue;
|
return newValue;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preShutdown(ObserverContext<MasterCoprocessorEnvironment> c)
|
public void preShutdown(ObserverContext<MasterCoprocessorEnvironment> c)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("shutdown", Action.ADMIN);
|
requirePermission(getActiveUser(c), "shutdown", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> c)
|
public void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> c)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("stopMaster", Action.ADMIN);
|
requirePermission(getActiveUser(c), "stopMaster", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -1315,18 +1315,19 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
|
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("snapshot", hTableDescriptor.getTableName(), null, null,
|
requirePermission(getActiveUser(ctx), "snapshot", hTableDescriptor.getTableName(), null, null,
|
||||||
Permission.Action.ADMIN);
|
Permission.Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final SnapshotDescription snapshot) throws IOException {
|
final SnapshotDescription snapshot) throws IOException {
|
||||||
if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, getActiveUser())) {
|
User user = getActiveUser(ctx);
|
||||||
|
if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, user)) {
|
||||||
// list it, if user is the owner of snapshot
|
// list it, if user is the owner of snapshot
|
||||||
// TODO: We are not logging this for audit
|
// TODO: We are not logging this for audit
|
||||||
} else {
|
} else {
|
||||||
requirePermission("listSnapshot", Action.ADMIN);
|
requirePermission(user, "listSnapshot", Action.ADMIN);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1334,42 +1335,44 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
|
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("clone", Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "clone", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
|
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, getActiveUser())) {
|
User user = getActiveUser(ctx);
|
||||||
requirePermission("restoreSnapshot", hTableDescriptor.getTableName(), null, null,
|
if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, user)) {
|
||||||
|
requirePermission(user, "restoreSnapshot", hTableDescriptor.getTableName(), null, null,
|
||||||
Permission.Action.ADMIN);
|
Permission.Action.ADMIN);
|
||||||
} else {
|
} else {
|
||||||
requirePermission("restoreSnapshot", Action.ADMIN);
|
requirePermission(user, "restoreSnapshot", Action.ADMIN);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final SnapshotDescription snapshot) throws IOException {
|
final SnapshotDescription snapshot) throws IOException {
|
||||||
if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, getActiveUser())) {
|
User user = getActiveUser(ctx);
|
||||||
|
if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, user)) {
|
||||||
// Snapshot owner is allowed to delete the snapshot
|
// Snapshot owner is allowed to delete the snapshot
|
||||||
// TODO: We are not logging this for audit
|
// TODO: We are not logging this for audit
|
||||||
} else {
|
} else {
|
||||||
requirePermission("deleteSnapshot", Action.ADMIN);
|
requirePermission(user, "deleteSnapshot", Action.ADMIN);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
NamespaceDescriptor ns) throws IOException {
|
NamespaceDescriptor ns) throws IOException {
|
||||||
requireGlobalPermission("createNamespace", Action.ADMIN, ns.getName());
|
requireGlobalPermission(getActiveUser(ctx), "createNamespace", Action.ADMIN, ns.getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace)
|
public void preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requireGlobalPermission("deleteNamespace", Action.ADMIN, namespace);
|
requireGlobalPermission(getActiveUser(ctx), "deleteNamespace", Action.ADMIN, namespace);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -1392,13 +1395,13 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
NamespaceDescriptor ns) throws IOException {
|
NamespaceDescriptor ns) throws IOException {
|
||||||
// We require only global permission so that
|
// We require only global permission so that
|
||||||
// a user with NS admin cannot altering namespace configurations. i.e. namespace quota
|
// a user with NS admin cannot altering namespace configurations. i.e. namespace quota
|
||||||
requireGlobalPermission("modifyNamespace", Action.ADMIN, ns.getName());
|
requireGlobalPermission(getActiveUser(ctx), "modifyNamespace", Action.ADMIN, ns.getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace)
|
public void preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requireNamespacePermission("getNamespaceDescriptor", namespace, Action.ADMIN);
|
requireNamespacePermission(getActiveUser(ctx), "getNamespaceDescriptor", namespace, Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -1407,10 +1410,11 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
// Retains only those which passes authorization checks, as the checks weren't done as part
|
// Retains only those which passes authorization checks, as the checks weren't done as part
|
||||||
// of preGetTableDescriptors.
|
// of preGetTableDescriptors.
|
||||||
Iterator<NamespaceDescriptor> itr = descriptors.iterator();
|
Iterator<NamespaceDescriptor> itr = descriptors.iterator();
|
||||||
|
User user = getActiveUser(ctx);
|
||||||
while (itr.hasNext()) {
|
while (itr.hasNext()) {
|
||||||
NamespaceDescriptor desc = itr.next();
|
NamespaceDescriptor desc = itr.next();
|
||||||
try {
|
try {
|
||||||
requireNamespacePermission("listNamespaces", desc.getName(), Action.ADMIN);
|
requireNamespacePermission(user, "listNamespaces", desc.getName(), Action.ADMIN);
|
||||||
} catch (AccessDeniedException e) {
|
} catch (AccessDeniedException e) {
|
||||||
itr.remove();
|
itr.remove();
|
||||||
}
|
}
|
||||||
@ -1420,24 +1424,25 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preTableFlush(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preTableFlush(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final TableName tableName) throws IOException {
|
final TableName tableName) throws IOException {
|
||||||
requirePermission("flushTable", tableName, null, null, Action.ADMIN, Action.CREATE);
|
requirePermission(getActiveUser(ctx), "flushTable", tableName, null, null,
|
||||||
|
Action.ADMIN, Action.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
/* ---- RegionObserver implementation ---- */
|
/* ---- RegionObserver implementation ---- */
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preOpen(ObserverContext<RegionCoprocessorEnvironment> e)
|
public void preOpen(ObserverContext<RegionCoprocessorEnvironment> c)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
RegionCoprocessorEnvironment env = e.getEnvironment();
|
RegionCoprocessorEnvironment env = c.getEnvironment();
|
||||||
final Region region = env.getRegion();
|
final Region region = env.getRegion();
|
||||||
if (region == null) {
|
if (region == null) {
|
||||||
LOG.error("NULL region from RegionCoprocessorEnvironment in preOpen()");
|
LOG.error("NULL region from RegionCoprocessorEnvironment in preOpen()");
|
||||||
} else {
|
} else {
|
||||||
HRegionInfo regionInfo = region.getRegionInfo();
|
HRegionInfo regionInfo = region.getRegionInfo();
|
||||||
if (regionInfo.getTable().isSystemTable()) {
|
if (regionInfo.getTable().isSystemTable()) {
|
||||||
checkSystemOrSuperUser();
|
checkSystemOrSuperUser(getActiveUser(c));
|
||||||
} else {
|
} else {
|
||||||
requirePermission("preOpen", Action.ADMIN);
|
requirePermission(getActiveUser(c), "preOpen", Action.ADMIN);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1481,28 +1486,30 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preFlush(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
|
public void preFlush(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {
|
||||||
requirePermission("flush", getTableName(e.getEnvironment()), null, null, Action.ADMIN,
|
requirePermission(getActiveUser(c), "flush", getTableName(c.getEnvironment()), null, null,
|
||||||
Action.CREATE);
|
Action.ADMIN, Action.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preSplit(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
|
public void preSplit(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {
|
||||||
requirePermission("split", getTableName(e.getEnvironment()), null, null, Action.ADMIN);
|
requirePermission(getActiveUser(c), "split", getTableName(c.getEnvironment()), null, null,
|
||||||
|
Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preSplit(ObserverContext<RegionCoprocessorEnvironment> e,
|
public void preSplit(ObserverContext<RegionCoprocessorEnvironment> c,
|
||||||
byte[] splitRow) throws IOException {
|
byte[] splitRow) throws IOException {
|
||||||
requirePermission("split", getTableName(e.getEnvironment()), null, null, Action.ADMIN);
|
requirePermission(getActiveUser(c), "split", getTableName(c.getEnvironment()), null, null,
|
||||||
|
Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
|
public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c,
|
||||||
final Store store, final InternalScanner scanner, final ScanType scanType)
|
final Store store, final InternalScanner scanner, final ScanType scanType)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("compact", getTableName(e.getEnvironment()), null, null, Action.ADMIN,
|
requirePermission(getActiveUser(c), "compact", getTableName(c.getEnvironment()), null, null,
|
||||||
Action.CREATE);
|
Action.ADMIN, Action.CREATE);
|
||||||
return scanner;
|
return scanner;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1513,7 +1520,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
if (filter != null && filter instanceof AccessControlFilter) {
|
if (filter != null && filter instanceof AccessControlFilter) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
User user = getActiveUser();
|
User user = getActiveUser(c);
|
||||||
RegionCoprocessorEnvironment env = c.getEnvironment();
|
RegionCoprocessorEnvironment env = c.getEnvironment();
|
||||||
Map<byte[],? extends Collection<byte[]>> families = null;
|
Map<byte[],? extends Collection<byte[]>> families = null;
|
||||||
switch (opType) {
|
switch (opType) {
|
||||||
@ -1626,7 +1633,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c,
|
public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||||
final Put put, final WALEdit edit, final Durability durability)
|
final Put put, final WALEdit edit, final Durability durability)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
User user = getActiveUser();
|
User user = getActiveUser(c);
|
||||||
checkForReservedTagPresence(user, put);
|
checkForReservedTagPresence(user, put);
|
||||||
|
|
||||||
// Require WRITE permission to the table, CF, or top visible value, if any.
|
// Require WRITE permission to the table, CF, or top visible value, if any.
|
||||||
@ -1681,7 +1688,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
// by a tombstone already) then we have to disallow this operation.
|
// by a tombstone already) then we have to disallow this operation.
|
||||||
RegionCoprocessorEnvironment env = c.getEnvironment();
|
RegionCoprocessorEnvironment env = c.getEnvironment();
|
||||||
Map<byte[],? extends Collection<Cell>> families = delete.getFamilyCellMap();
|
Map<byte[],? extends Collection<Cell>> families = delete.getFamilyCellMap();
|
||||||
User user = getActiveUser();
|
User user = getActiveUser(c);
|
||||||
AuthResult authResult = permissionGranted(OpType.DELETE, user, env, families, Action.WRITE);
|
AuthResult authResult = permissionGranted(OpType.DELETE, user, env, families, Action.WRITE);
|
||||||
logResult(authResult);
|
logResult(authResult);
|
||||||
if (!authResult.isAllowed()) {
|
if (!authResult.isAllowed()) {
|
||||||
@ -1699,6 +1706,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
||||||
if (cellFeaturesEnabled && !compatibleEarlyTermination) {
|
if (cellFeaturesEnabled && !compatibleEarlyTermination) {
|
||||||
TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
|
TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
|
||||||
|
User user = getActiveUser(c);
|
||||||
for (int i = 0; i < miniBatchOp.size(); i++) {
|
for (int i = 0; i < miniBatchOp.size(); i++) {
|
||||||
Mutation m = miniBatchOp.getOperation(i);
|
Mutation m = miniBatchOp.getOperation(i);
|
||||||
if (m.getAttribute(CHECK_COVERING_PERM) != null) {
|
if (m.getAttribute(CHECK_COVERING_PERM) != null) {
|
||||||
@ -1706,19 +1714,19 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
// perm check
|
// perm check
|
||||||
OpType opType;
|
OpType opType;
|
||||||
if (m instanceof Put) {
|
if (m instanceof Put) {
|
||||||
checkForReservedTagPresence(getActiveUser(), m);
|
checkForReservedTagPresence(user, m);
|
||||||
opType = OpType.PUT;
|
opType = OpType.PUT;
|
||||||
} else {
|
} else {
|
||||||
opType = OpType.DELETE;
|
opType = OpType.DELETE;
|
||||||
}
|
}
|
||||||
AuthResult authResult = null;
|
AuthResult authResult = null;
|
||||||
if (checkCoveringPermission(opType, c.getEnvironment(), m.getRow(),
|
if (checkCoveringPermission(user, opType, c.getEnvironment(), m.getRow(),
|
||||||
m.getFamilyCellMap(), m.getTimeStamp(), Action.WRITE)) {
|
m.getFamilyCellMap(), m.getTimeStamp(), Action.WRITE)) {
|
||||||
authResult = AuthResult.allow(opType.toString(), "Covering cell set",
|
authResult = AuthResult.allow(opType.toString(), "Covering cell set",
|
||||||
getActiveUser(), Action.WRITE, table, m.getFamilyCellMap());
|
user, Action.WRITE, table, m.getFamilyCellMap());
|
||||||
} else {
|
} else {
|
||||||
authResult = AuthResult.deny(opType.toString(), "Covering cell set",
|
authResult = AuthResult.deny(opType.toString(), "Covering cell set",
|
||||||
getActiveUser(), Action.WRITE, table, m.getFamilyCellMap());
|
user, Action.WRITE, table, m.getFamilyCellMap());
|
||||||
}
|
}
|
||||||
logResult(authResult);
|
logResult(authResult);
|
||||||
if (authorizationEnabled && !authResult.isAllowed()) {
|
if (authorizationEnabled && !authResult.isAllowed()) {
|
||||||
@ -1745,7 +1753,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
final CompareFilter.CompareOp compareOp,
|
final CompareFilter.CompareOp compareOp,
|
||||||
final ByteArrayComparable comparator, final Put put,
|
final ByteArrayComparable comparator, final Put put,
|
||||||
final boolean result) throws IOException {
|
final boolean result) throws IOException {
|
||||||
User user = getActiveUser();
|
User user = getActiveUser(c);
|
||||||
checkForReservedTagPresence(user, put);
|
checkForReservedTagPresence(user, put);
|
||||||
|
|
||||||
// Require READ and WRITE permissions on the table, CF, and KV to update
|
// Require READ and WRITE permissions on the table, CF, and KV to update
|
||||||
@ -1785,13 +1793,14 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
|
TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
|
||||||
Map<byte[], ? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
|
Map<byte[], ? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
|
||||||
AuthResult authResult = null;
|
AuthResult authResult = null;
|
||||||
if (checkCoveringPermission(OpType.CHECK_AND_PUT, c.getEnvironment(), row, families,
|
User user = getActiveUser(c);
|
||||||
|
if (checkCoveringPermission(user, OpType.CHECK_AND_PUT, c.getEnvironment(), row, families,
|
||||||
HConstants.LATEST_TIMESTAMP, Action.READ)) {
|
HConstants.LATEST_TIMESTAMP, Action.READ)) {
|
||||||
authResult = AuthResult.allow(OpType.CHECK_AND_PUT.toString(), "Covering cell set",
|
authResult = AuthResult.allow(OpType.CHECK_AND_PUT.toString(), "Covering cell set",
|
||||||
getActiveUser(), Action.READ, table, families);
|
user, Action.READ, table, families);
|
||||||
} else {
|
} else {
|
||||||
authResult = AuthResult.deny(OpType.CHECK_AND_PUT.toString(), "Covering cell set",
|
authResult = AuthResult.deny(OpType.CHECK_AND_PUT.toString(), "Covering cell set",
|
||||||
getActiveUser(), Action.READ, table, families);
|
user, Action.READ, table, families);
|
||||||
}
|
}
|
||||||
logResult(authResult);
|
logResult(authResult);
|
||||||
if (authorizationEnabled && !authResult.isAllowed()) {
|
if (authorizationEnabled && !authResult.isAllowed()) {
|
||||||
@ -1816,7 +1825,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
// by the delete
|
// by the delete
|
||||||
RegionCoprocessorEnvironment env = c.getEnvironment();
|
RegionCoprocessorEnvironment env = c.getEnvironment();
|
||||||
Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
|
Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
|
||||||
User user = getActiveUser();
|
User user = getActiveUser(c);
|
||||||
AuthResult authResult = permissionGranted(OpType.CHECK_AND_DELETE, user, env, families,
|
AuthResult authResult = permissionGranted(OpType.CHECK_AND_DELETE, user, env, families,
|
||||||
Action.READ, Action.WRITE);
|
Action.READ, Action.WRITE);
|
||||||
logResult(authResult);
|
logResult(authResult);
|
||||||
@ -1843,13 +1852,14 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
|
TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
|
||||||
Map<byte[], ? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
|
Map<byte[], ? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
|
||||||
AuthResult authResult = null;
|
AuthResult authResult = null;
|
||||||
if (checkCoveringPermission(OpType.CHECK_AND_DELETE, c.getEnvironment(), row, families,
|
User user = getActiveUser(c);
|
||||||
|
if (checkCoveringPermission(user, OpType.CHECK_AND_DELETE, c.getEnvironment(), row, families,
|
||||||
HConstants.LATEST_TIMESTAMP, Action.READ)) {
|
HConstants.LATEST_TIMESTAMP, Action.READ)) {
|
||||||
authResult = AuthResult.allow(OpType.CHECK_AND_DELETE.toString(), "Covering cell set",
|
authResult = AuthResult.allow(OpType.CHECK_AND_DELETE.toString(), "Covering cell set",
|
||||||
getActiveUser(), Action.READ, table, families);
|
user, Action.READ, table, families);
|
||||||
} else {
|
} else {
|
||||||
authResult = AuthResult.deny(OpType.CHECK_AND_DELETE.toString(), "Covering cell set",
|
authResult = AuthResult.deny(OpType.CHECK_AND_DELETE.toString(), "Covering cell set",
|
||||||
getActiveUser(), Action.READ, table, families);
|
user, Action.READ, table, families);
|
||||||
}
|
}
|
||||||
logResult(authResult);
|
logResult(authResult);
|
||||||
if (authorizationEnabled && !authResult.isAllowed()) {
|
if (authorizationEnabled && !authResult.isAllowed()) {
|
||||||
@ -1868,11 +1878,11 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
// incremented value
|
// incremented value
|
||||||
RegionCoprocessorEnvironment env = c.getEnvironment();
|
RegionCoprocessorEnvironment env = c.getEnvironment();
|
||||||
Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
|
Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
|
||||||
User user = getActiveUser();
|
User user = getActiveUser(c);
|
||||||
AuthResult authResult = permissionGranted(OpType.INCREMENT_COLUMN_VALUE, user, env, families,
|
AuthResult authResult = permissionGranted(OpType.INCREMENT_COLUMN_VALUE, user, env, families,
|
||||||
Action.WRITE);
|
Action.WRITE);
|
||||||
if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
|
if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
|
||||||
authResult.setAllowed(checkCoveringPermission(OpType.INCREMENT_COLUMN_VALUE, env, row,
|
authResult.setAllowed(checkCoveringPermission(user, OpType.INCREMENT_COLUMN_VALUE, env, row,
|
||||||
families, HConstants.LATEST_TIMESTAMP, Action.WRITE));
|
families, HConstants.LATEST_TIMESTAMP, Action.WRITE));
|
||||||
authResult.setReason("Covering cell set");
|
authResult.setReason("Covering cell set");
|
||||||
}
|
}
|
||||||
@ -1886,7 +1896,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append)
|
public Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
User user = getActiveUser();
|
User user = getActiveUser(c);
|
||||||
checkForReservedTagPresence(user, append);
|
checkForReservedTagPresence(user, append);
|
||||||
|
|
||||||
// Require WRITE permission to the table, CF, and the KV to be appended
|
// Require WRITE permission to the table, CF, and the KV to be appended
|
||||||
@ -1923,13 +1933,14 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
// perm check
|
// perm check
|
||||||
TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
|
TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
|
||||||
AuthResult authResult = null;
|
AuthResult authResult = null;
|
||||||
if (checkCoveringPermission(OpType.APPEND, c.getEnvironment(), append.getRow(),
|
User user = getActiveUser(c);
|
||||||
|
if (checkCoveringPermission(user, OpType.APPEND, c.getEnvironment(), append.getRow(),
|
||||||
append.getFamilyCellMap(), HConstants.LATEST_TIMESTAMP, Action.WRITE)) {
|
append.getFamilyCellMap(), HConstants.LATEST_TIMESTAMP, Action.WRITE)) {
|
||||||
authResult = AuthResult.allow(OpType.APPEND.toString(), "Covering cell set",
|
authResult = AuthResult.allow(OpType.APPEND.toString(), "Covering cell set",
|
||||||
getActiveUser(), Action.WRITE, table, append.getFamilyCellMap());
|
user, Action.WRITE, table, append.getFamilyCellMap());
|
||||||
} else {
|
} else {
|
||||||
authResult = AuthResult.deny(OpType.APPEND.toString(), "Covering cell set",
|
authResult = AuthResult.deny(OpType.APPEND.toString(), "Covering cell set",
|
||||||
getActiveUser(), Action.WRITE, table, append.getFamilyCellMap());
|
user, Action.WRITE, table, append.getFamilyCellMap());
|
||||||
}
|
}
|
||||||
logResult(authResult);
|
logResult(authResult);
|
||||||
if (authorizationEnabled && !authResult.isAllowed()) {
|
if (authorizationEnabled && !authResult.isAllowed()) {
|
||||||
@ -1944,7 +1955,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
|
public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||||
final Increment increment)
|
final Increment increment)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
User user = getActiveUser();
|
User user = getActiveUser(c);
|
||||||
checkForReservedTagPresence(user, increment);
|
checkForReservedTagPresence(user, increment);
|
||||||
|
|
||||||
// Require WRITE permission to the table, CF, and the KV to be replaced by
|
// Require WRITE permission to the table, CF, and the KV to be replaced by
|
||||||
@ -1983,13 +1994,14 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
// perm check
|
// perm check
|
||||||
TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
|
TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
|
||||||
AuthResult authResult = null;
|
AuthResult authResult = null;
|
||||||
if (checkCoveringPermission(OpType.INCREMENT, c.getEnvironment(), increment.getRow(),
|
User user = getActiveUser(c);
|
||||||
|
if (checkCoveringPermission(user, OpType.INCREMENT, c.getEnvironment(), increment.getRow(),
|
||||||
increment.getFamilyCellMap(), increment.getTimeRange().getMax(), Action.WRITE)) {
|
increment.getFamilyCellMap(), increment.getTimeRange().getMax(), Action.WRITE)) {
|
||||||
authResult = AuthResult.allow(OpType.INCREMENT.toString(), "Covering cell set",
|
authResult = AuthResult.allow(OpType.INCREMENT.toString(), "Covering cell set",
|
||||||
getActiveUser(), Action.WRITE, table, increment.getFamilyCellMap());
|
user, Action.WRITE, table, increment.getFamilyCellMap());
|
||||||
} else {
|
} else {
|
||||||
authResult = AuthResult.deny(OpType.INCREMENT.toString(), "Covering cell set",
|
authResult = AuthResult.deny(OpType.INCREMENT.toString(), "Covering cell set",
|
||||||
getActiveUser(), Action.WRITE, table, increment.getFamilyCellMap());
|
user, Action.WRITE, table, increment.getFamilyCellMap());
|
||||||
}
|
}
|
||||||
logResult(authResult);
|
logResult(authResult);
|
||||||
if (authorizationEnabled && !authResult.isAllowed()) {
|
if (authorizationEnabled && !authResult.isAllowed()) {
|
||||||
@ -2067,7 +2079,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public RegionScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
|
public RegionScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||||
final Scan scan, final RegionScanner s) throws IOException {
|
final Scan scan, final RegionScanner s) throws IOException {
|
||||||
User user = getActiveUser();
|
User user = getActiveUser(c);
|
||||||
if (user != null && user.getShortName() != null) {
|
if (user != null && user.getShortName() != null) {
|
||||||
// store reference to scanner owner for later checks
|
// store reference to scanner owner for later checks
|
||||||
scannerOwners.put(s, user.getShortName());
|
scannerOwners.put(s, user.getShortName());
|
||||||
@ -2127,8 +2139,9 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
|
public void preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
|
||||||
List<Pair<byte[], String>> familyPaths) throws IOException {
|
List<Pair<byte[], String>> familyPaths) throws IOException {
|
||||||
|
User user = getActiveUser(ctx);
|
||||||
for(Pair<byte[],String> el : familyPaths) {
|
for(Pair<byte[],String> el : familyPaths) {
|
||||||
requirePermission("preBulkLoadHFile",
|
requirePermission(user, "preBulkLoadHFile",
|
||||||
ctx.getEnvironment().getRegion().getTableDesc().getTableName(),
|
ctx.getEnvironment().getRegion().getTableDesc().getTableName(),
|
||||||
el.getFirst(),
|
el.getFirst(),
|
||||||
null,
|
null,
|
||||||
@ -2146,7 +2159,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void prePrepareBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx,
|
public void prePrepareBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx,
|
||||||
PrepareBulkLoadRequest request) throws IOException {
|
PrepareBulkLoadRequest request) throws IOException {
|
||||||
requireAccess("prePareBulkLoad",
|
requireAccess(getActiveUser(ctx), "prePrepareBulkLoad",
|
||||||
ctx.getEnvironment().getRegion().getTableDesc().getTableName(), Action.CREATE);
|
ctx.getEnvironment().getRegion().getTableDesc().getTableName(), Action.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2160,7 +2173,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preCleanupBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx,
|
public void preCleanupBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx,
|
||||||
CleanupBulkLoadRequest request) throws IOException {
|
CleanupBulkLoadRequest request) throws IOException {
|
||||||
requireAccess("preCleanupBulkLoad",
|
requireAccess(getActiveUser(ctx), "preCleanupBulkLoad",
|
||||||
ctx.getEnvironment().getRegion().getTableDesc().getTableName(), Action.CREATE);
|
ctx.getEnvironment().getRegion().getTableDesc().getTableName(), Action.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2172,10 +2185,10 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
// Don't intercept calls to our own AccessControlService, we check for
|
// Don't intercept calls to our own AccessControlService, we check for
|
||||||
// appropriate permissions in the service handlers
|
// appropriate permissions in the service handlers
|
||||||
if (shouldCheckExecPermission && !(service instanceof AccessControlService)) {
|
if (shouldCheckExecPermission && !(service instanceof AccessControlService)) {
|
||||||
requirePermission("invoke(" + service.getDescriptorForType().getName() + "." +
|
requirePermission(getActiveUser(ctx),
|
||||||
methodName + ")",
|
"invoke(" + service.getDescriptorForType().getName() + "." + methodName + ")",
|
||||||
getTableName(ctx.getEnvironment()), null, null,
|
getTableName(ctx.getEnvironment()), null, null,
|
||||||
Action.EXEC);
|
Action.EXEC);
|
||||||
}
|
}
|
||||||
return request;
|
return request;
|
||||||
}
|
}
|
||||||
@ -2202,15 +2215,16 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Received request to grant access permission " + perm.toString());
|
LOG.debug("Received request to grant access permission " + perm.toString());
|
||||||
}
|
}
|
||||||
|
User caller = RpcServer.getRequestUser();
|
||||||
|
|
||||||
switch(request.getUserPermission().getPermission().getType()) {
|
switch(request.getUserPermission().getPermission().getType()) {
|
||||||
case Global :
|
case Global :
|
||||||
case Table :
|
case Table :
|
||||||
requirePermission("grant", perm.getTableName(), perm.getFamily(),
|
requirePermission(caller, "grant", perm.getTableName(),
|
||||||
perm.getQualifier(), Action.ADMIN);
|
perm.getFamily(), perm.getQualifier(), Action.ADMIN);
|
||||||
break;
|
break;
|
||||||
case Namespace :
|
case Namespace :
|
||||||
requireNamespacePermission("grant", perm.getNamespace(), Action.ADMIN);
|
requireNamespacePermission(caller, "grant", perm.getNamespace(), Action.ADMIN);
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2253,15 +2267,16 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Received request to revoke access permission " + perm.toString());
|
LOG.debug("Received request to revoke access permission " + perm.toString());
|
||||||
}
|
}
|
||||||
|
User caller = RpcServer.getRequestUser();
|
||||||
|
|
||||||
switch(request.getUserPermission().getPermission().getType()) {
|
switch(request.getUserPermission().getPermission().getType()) {
|
||||||
case Global :
|
case Global :
|
||||||
case Table :
|
case Table :
|
||||||
requirePermission("revoke", perm.getTableName(), perm.getFamily(),
|
requirePermission(caller, "revoke", perm.getTableName(), perm.getFamily(),
|
||||||
perm.getQualifier(), Action.ADMIN);
|
perm.getQualifier(), Action.ADMIN);
|
||||||
break;
|
break;
|
||||||
case Namespace :
|
case Namespace :
|
||||||
requireNamespacePermission("revoke", perm.getNamespace(), Action.ADMIN);
|
requireNamespacePermission(caller, "revoke", perm.getNamespace(), Action.ADMIN);
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2300,11 +2315,13 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
if (!initialized) {
|
if (!initialized) {
|
||||||
throw new CoprocessorException("AccessController not yet initialized");
|
throw new CoprocessorException("AccessController not yet initialized");
|
||||||
}
|
}
|
||||||
|
User caller = RpcServer.getRequestUser();
|
||||||
|
|
||||||
List<UserPermission> perms = null;
|
List<UserPermission> perms = null;
|
||||||
if (request.getType() == AccessControlProtos.Permission.Type.Table) {
|
if (request.getType() == AccessControlProtos.Permission.Type.Table) {
|
||||||
final TableName table = request.hasTableName() ?
|
final TableName table = request.hasTableName() ?
|
||||||
ProtobufUtil.toTableName(request.getTableName()) : null;
|
ProtobufUtil.toTableName(request.getTableName()) : null;
|
||||||
requirePermission("userPermissions", table, null, null, Action.ADMIN);
|
requirePermission(caller, "userPermissions", table, null, null, Action.ADMIN);
|
||||||
perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
|
perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
|
||||||
@Override
|
@Override
|
||||||
public List<UserPermission> run() throws Exception {
|
public List<UserPermission> run() throws Exception {
|
||||||
@ -2313,7 +2330,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
});
|
});
|
||||||
} else if (request.getType() == AccessControlProtos.Permission.Type.Namespace) {
|
} else if (request.getType() == AccessControlProtos.Permission.Type.Namespace) {
|
||||||
final String namespace = request.getNamespaceName().toStringUtf8();
|
final String namespace = request.getNamespaceName().toStringUtf8();
|
||||||
requireNamespacePermission("userPermissions", namespace, Action.ADMIN);
|
requireNamespacePermission(caller, "userPermissions", namespace, Action.ADMIN);
|
||||||
perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
|
perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
|
||||||
@Override
|
@Override
|
||||||
public List<UserPermission> run() throws Exception {
|
public List<UserPermission> run() throws Exception {
|
||||||
@ -2322,7 +2339,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
}
|
}
|
||||||
});
|
});
|
||||||
} else {
|
} else {
|
||||||
requirePermission("userPermissions", Action.ADMIN);
|
requirePermission(caller, "userPermissions", Action.ADMIN);
|
||||||
perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
|
perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
|
||||||
@Override
|
@Override
|
||||||
public List<UserPermission> run() throws Exception {
|
public List<UserPermission> run() throws Exception {
|
||||||
@ -2359,7 +2376,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
}
|
}
|
||||||
AccessControlProtos.CheckPermissionsResponse response = null;
|
AccessControlProtos.CheckPermissionsResponse response = null;
|
||||||
try {
|
try {
|
||||||
User user = getActiveUser();
|
User user = RpcServer.getRequestUser();
|
||||||
TableName tableName = regionEnv.getRegion().getTableDesc().getTableName();
|
TableName tableName = regionEnv.getRegion().getTableDesc().getTableName();
|
||||||
for (Permission permission : permissions) {
|
for (Permission permission : permissions) {
|
||||||
if (permission instanceof TablePermission) {
|
if (permission instanceof TablePermission) {
|
||||||
@ -2453,17 +2470,16 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preClose(ObserverContext<RegionCoprocessorEnvironment> e, boolean abortRequested)
|
public void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("preClose", Action.ADMIN);
|
requirePermission(getActiveUser(c), "preClose", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void checkSystemOrSuperUser() throws IOException {
|
private void checkSystemOrSuperUser(User activeUser) throws IOException {
|
||||||
// No need to check if we're not going to throw
|
// No need to check if we're not going to throw
|
||||||
if (!authorizationEnabled) {
|
if (!authorizationEnabled) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
User activeUser = getActiveUser();
|
|
||||||
if (!Superusers.isSuperUser(activeUser)) {
|
if (!Superusers.isSuperUser(activeUser)) {
|
||||||
throw new AccessDeniedException("User '" + (activeUser != null ?
|
throw new AccessDeniedException("User '" + (activeUser != null ?
|
||||||
activeUser.getShortName() : "null") + "' is not system or super user.");
|
activeUser.getShortName() : "null") + "' is not system or super user.");
|
||||||
@ -2472,9 +2488,9 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preStopRegionServer(
|
public void preStopRegionServer(
|
||||||
ObserverContext<RegionServerCoprocessorEnvironment> env)
|
ObserverContext<RegionServerCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("preStopRegionServer", Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "preStopRegionServer", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<byte[], ? extends Collection<byte[]>> makeFamilyMap(byte[] family,
|
private Map<byte[], ? extends Collection<byte[]>> makeFamilyMap(byte[] family,
|
||||||
@ -2502,7 +2518,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
// Skip checks for a table that does not exist
|
// Skip checks for a table that does not exist
|
||||||
if (!masterServices.getTableStateManager().isTablePresent(tableName))
|
if (!masterServices.getTableStateManager().isTablePresent(tableName))
|
||||||
continue;
|
continue;
|
||||||
requirePermission("getTableDescriptors", tableName, null, null,
|
requirePermission(getActiveUser(ctx), "getTableDescriptors", tableName, null, null,
|
||||||
Action.ADMIN, Action.CREATE);
|
Action.ADMIN, Action.CREATE);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -2523,7 +2539,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
while (itr.hasNext()) {
|
while (itr.hasNext()) {
|
||||||
HTableDescriptor htd = itr.next();
|
HTableDescriptor htd = itr.next();
|
||||||
try {
|
try {
|
||||||
requirePermission("getTableDescriptors", htd.getTableName(), null, null,
|
requirePermission(getActiveUser(ctx), "getTableDescriptors", htd.getTableName(), null, null,
|
||||||
Action.ADMIN, Action.CREATE);
|
Action.ADMIN, Action.CREATE);
|
||||||
} catch (AccessDeniedException e) {
|
} catch (AccessDeniedException e) {
|
||||||
itr.remove();
|
itr.remove();
|
||||||
@ -2539,7 +2555,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
while (itr.hasNext()) {
|
while (itr.hasNext()) {
|
||||||
HTableDescriptor htd = itr.next();
|
HTableDescriptor htd = itr.next();
|
||||||
try {
|
try {
|
||||||
requireAccess("getTableNames", htd.getTableName(), Action.values());
|
requireAccess(getActiveUser(ctx), "getTableNames", htd.getTableName(), Action.values());
|
||||||
} catch (AccessDeniedException e) {
|
} catch (AccessDeniedException e) {
|
||||||
itr.remove();
|
itr.remove();
|
||||||
}
|
}
|
||||||
@ -2549,15 +2565,15 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
HRegionInfo regionA, HRegionInfo regionB) throws IOException {
|
HRegionInfo regionA, HRegionInfo regionB) throws IOException {
|
||||||
requirePermission("mergeRegions", regionA.getTable(), null, null,
|
requirePermission(getActiveUser(ctx), "mergeRegions", regionA.getTable(), null, null,
|
||||||
Action.ADMIN);
|
Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx, Region regionA,
|
public void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx, Region regionA,
|
||||||
Region regionB) throws IOException {
|
Region regionB) throws IOException {
|
||||||
requirePermission("mergeRegions", regionA.getTableDesc().getTableName(), null, null,
|
requirePermission(getActiveUser(ctx), "mergeRegions", regionA.getTableDesc().getTableName(),
|
||||||
Action.ADMIN);
|
null, null, Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -2583,7 +2599,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
|
public void preRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
requirePermission("preRollLogWriterRequest", Permission.Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "preRollLogWriterRequest", Permission.Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -2593,31 +2609,31 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final String userName, final Quotas quotas) throws IOException {
|
final String userName, final Quotas quotas) throws IOException {
|
||||||
requirePermission("setUserQuota", Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "setUserQuota", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final String userName, final TableName tableName, final Quotas quotas) throws IOException {
|
final String userName, final TableName tableName, final Quotas quotas) throws IOException {
|
||||||
requirePermission("setUserTableQuota", tableName, null, null, Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "setUserTableQuota", tableName, null, null, Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final String userName, final String namespace, final Quotas quotas) throws IOException {
|
final String userName, final String namespace, final Quotas quotas) throws IOException {
|
||||||
requirePermission("setUserNamespaceQuota", Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "setUserNamespaceQuota", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final TableName tableName, final Quotas quotas) throws IOException {
|
final TableName tableName, final Quotas quotas) throws IOException {
|
||||||
requirePermission("setTableQuota", tableName, null, null, Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "setTableQuota", tableName, null, null, Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
final String namespace, final Quotas quotas) throws IOException {
|
final String namespace, final Quotas quotas) throws IOException {
|
||||||
requirePermission("setNamespaceQuota", Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "setNamespaceQuota", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -2629,7 +2645,7 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preReplicateLogEntries(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
|
public void preReplicateLogEntries(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
|
||||||
List<WALEntry> entries, CellScanner cells) throws IOException {
|
List<WALEntry> entries, CellScanner cells) throws IOException {
|
||||||
requirePermission("replicateLogEntries", Action.WRITE);
|
requirePermission(getActiveUser(ctx), "replicateLogEntries", Action.WRITE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -2640,30 +2656,30 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||||||
@Override
|
@Override
|
||||||
public void preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
Set<HostAndPort> servers, String targetGroup) throws IOException {
|
Set<HostAndPort> servers, String targetGroup) throws IOException {
|
||||||
requirePermission("moveServers", Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "moveServers", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
Set<TableName> tables, String targetGroup) throws IOException {
|
Set<TableName> tables, String targetGroup) throws IOException {
|
||||||
requirePermission("moveTables", Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "moveTables", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
String name) throws IOException {
|
String name) throws IOException {
|
||||||
requirePermission("addRSGroup", Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "addRSGroup", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
String name) throws IOException {
|
String name) throws IOException {
|
||||||
requirePermission("removeRSGroup", Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "removeRSGroup", Action.ADMIN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
public void preBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||||
String groupName) throws IOException {
|
String groupName) throws IOException {
|
||||||
requirePermission("balanceRSGroup", Action.ADMIN);
|
requirePermission(getActiveUser(ctx), "balanceRSGroup", Action.ADMIN);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user