HBASE-18823 Apply RegionInfo to MasterObserver/RegionObserver/WALObserver
This commit is contained in:
parent
37696fffe9
commit
a6c3c645fd
|
@ -27,17 +27,17 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.backup.impl.BackupManager;
|
||||
import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionObserver;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* An Observer to facilitate backup operations
|
||||
|
@ -61,7 +61,7 @@ public class BackupObserver implements RegionObserver {
|
|||
try (Connection connection = ConnectionFactory.createConnection(cfg);
|
||||
BackupSystemTable tbl = new BackupSystemTable(connection)) {
|
||||
List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
|
||||
HRegionInfo info = ctx.getEnvironment().getRegionInfo();
|
||||
RegionInfo info = ctx.getEnvironment().getRegionInfo();
|
||||
TableName tableName = info.getTable();
|
||||
if (!fullyBackedUpTables.contains(tableName)) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
|
@ -87,7 +87,7 @@ public class BackupObserver implements RegionObserver {
|
|||
try (Connection connection = ConnectionFactory.createConnection(cfg);
|
||||
BackupSystemTable tbl = new BackupSystemTable(connection)) {
|
||||
List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
|
||||
HRegionInfo info = ctx.getEnvironment().getRegionInfo();
|
||||
RegionInfo info = ctx.getEnvironment().getRegionInfo();
|
||||
TableName tableName = info.getTable();
|
||||
if (!fullyBackedUpTables.contains(tableName)) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
|
|
|
@ -23,11 +23,11 @@ import java.io.IOException;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.metrics.Counter;
|
||||
import org.apache.hadoop.hbase.metrics.Gauge;
|
||||
|
@ -68,7 +68,7 @@ public class ExampleMasterObserverWithMetrics implements MasterObserver {
|
|||
|
||||
@Override
|
||||
public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {
|
||||
// we rely on the fact that there is only 1 instance of our MasterObserver. We keep track of
|
||||
// when the operation starts before the operation is executing.
|
||||
this.createTableStartTime = System.currentTimeMillis();
|
||||
|
@ -76,7 +76,7 @@ public class ExampleMasterObserverWithMetrics implements MasterObserver {
|
|||
|
||||
@Override
|
||||
public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {
|
||||
if (this.createTableStartTime > 0) {
|
||||
long time = System.currentTimeMillis() - this.createTableStartTime;
|
||||
LOG.info("Create table took: " + time);
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.io.IOException;
|
|||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
|
@ -30,11 +29,9 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
|
||||
|
@ -68,7 +65,9 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.MoveTablesR
|
|||
import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RSGroupAdminService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupResponse;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService {
|
||||
|
@ -309,7 +308,7 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
|
|||
// Assign table to default RSGroup.
|
||||
@Override
|
||||
public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {
|
||||
assignTableToGroup(desc);
|
||||
}
|
||||
|
||||
|
|
|
@ -25,16 +25,14 @@ import java.util.Set;
|
|||
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.MetaMutationAnnotation;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.locking.LockProcedure;
|
||||
|
@ -47,6 +45,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
|||
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -88,7 +88,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
* @param regions the initial regions created for the table
|
||||
*/
|
||||
default void preCreateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {}
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called after the createTable operation has been requested. Called as part
|
||||
|
@ -98,7 +98,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
* @param regions the initial regions created for the table
|
||||
*/
|
||||
default void postCreateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {}
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called before a new table is created by
|
||||
|
@ -113,7 +113,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
default void preCreateTableAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final TableDescriptor desc,
|
||||
final HRegionInfo[] regions) throws IOException {}
|
||||
final RegionInfo[] regions) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called after the createTable operation has been requested. Called as part
|
||||
|
@ -127,7 +127,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
default void postCompletedCreateTableAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final TableDescriptor desc,
|
||||
final HRegionInfo[] regions) throws IOException {}
|
||||
final RegionInfo[] regions) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a
|
||||
|
@ -550,24 +550,24 @@ public interface MasterObserver extends Coprocessor {
|
|||
/**
|
||||
* Called prior to moving a given region from one region server to another.
|
||||
* @param ctx the environment to interact with the framework and master
|
||||
* @param region the HRegionInfo
|
||||
* @param region the RegionInfo
|
||||
* @param srcServer the source ServerName
|
||||
* @param destServer the destination ServerName
|
||||
*/
|
||||
default void preMove(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo region, final ServerName srcServer,
|
||||
final RegionInfo region, final ServerName srcServer,
|
||||
final ServerName destServer)
|
||||
throws IOException {}
|
||||
|
||||
/**
|
||||
* Called after the region move has been requested.
|
||||
* @param ctx the environment to interact with the framework and master
|
||||
* @param region the HRegionInfo
|
||||
* @param region the RegionInfo
|
||||
* @param srcServer the source ServerName
|
||||
* @param destServer the destination ServerName
|
||||
*/
|
||||
default void postMove(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo region, final ServerName srcServer,
|
||||
final RegionInfo region, final ServerName srcServer,
|
||||
final ServerName destServer)
|
||||
throws IOException {}
|
||||
|
||||
|
@ -577,7 +577,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
* @param regionInfo the regionInfo of the region
|
||||
*/
|
||||
default void preAssign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo regionInfo) throws IOException {}
|
||||
final RegionInfo regionInfo) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called after the region assignment has been requested.
|
||||
|
@ -585,7 +585,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
* @param regionInfo the regionInfo of the region
|
||||
*/
|
||||
default void postAssign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo regionInfo) throws IOException {}
|
||||
final RegionInfo regionInfo) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called prior to unassigning a given region.
|
||||
|
@ -594,7 +594,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
* @param force whether to force unassignment or not
|
||||
*/
|
||||
default void preUnassign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo regionInfo, final boolean force) throws IOException {}
|
||||
final RegionInfo regionInfo, final boolean force) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called after the region unassignment has been requested.
|
||||
|
@ -603,7 +603,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
* @param force whether to force unassignment or not
|
||||
*/
|
||||
default void postUnassign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo regionInfo, final boolean force) throws IOException {}
|
||||
final RegionInfo regionInfo, final boolean force) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called prior to marking a given region as offline. <code>ctx.bypass()</code> will not have any
|
||||
|
@ -612,7 +612,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
* @param regionInfo
|
||||
*/
|
||||
default void preRegionOffline(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo regionInfo) throws IOException {}
|
||||
final RegionInfo regionInfo) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called after the region has been marked offline.
|
||||
|
@ -620,7 +620,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
* @param regionInfo
|
||||
*/
|
||||
default void postRegionOffline(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo regionInfo) throws IOException {}
|
||||
final RegionInfo regionInfo) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called prior to requesting rebalancing of the cluster regions, though after
|
||||
|
@ -692,8 +692,8 @@ public interface MasterObserver extends Coprocessor {
|
|||
*/
|
||||
default void postCompletedSplitRegionAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> c,
|
||||
final HRegionInfo regionInfoA,
|
||||
final HRegionInfo regionInfoB) throws IOException {}
|
||||
final RegionInfo regionInfoA,
|
||||
final RegionInfo regionInfoB) throws IOException {}
|
||||
|
||||
/**
|
||||
* This will be called before PONR step as part of split transaction. Calling
|
||||
|
@ -733,7 +733,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
*/
|
||||
default void preMergeRegionsAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge) throws IOException {}
|
||||
final RegionInfo[] regionsToMerge) throws IOException {}
|
||||
|
||||
/**
|
||||
* called after the regions merge.
|
||||
|
@ -741,8 +741,8 @@ public interface MasterObserver extends Coprocessor {
|
|||
*/
|
||||
default void postCompletedMergeRegionsAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge,
|
||||
final HRegionInfo mergedRegion) throws IOException {}
|
||||
final RegionInfo[] regionsToMerge,
|
||||
final RegionInfo mergedRegion) throws IOException {}
|
||||
|
||||
/**
|
||||
* This will be called before PONR step as part of regions merge transaction. Calling
|
||||
|
@ -753,7 +753,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
*/
|
||||
default void preMergeRegionsCommitAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge,
|
||||
final RegionInfo[] regionsToMerge,
|
||||
@MetaMutationAnnotation List<Mutation> metaEntries) throws IOException {}
|
||||
|
||||
/**
|
||||
|
@ -762,8 +762,8 @@ public interface MasterObserver extends Coprocessor {
|
|||
*/
|
||||
default void postMergeRegionsCommitAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge,
|
||||
final HRegionInfo mergedRegion) throws IOException {}
|
||||
final RegionInfo[] regionsToMerge,
|
||||
final RegionInfo mergedRegion) throws IOException {}
|
||||
|
||||
/**
|
||||
* This will be called after the roll back of the regions merge.
|
||||
|
@ -771,7 +771,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
*/
|
||||
default void postRollBackMergeRegionsAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge) throws IOException {}
|
||||
final RegionInfo[] regionsToMerge) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called prior to modifying the flag used to enable/disable region balancing.
|
||||
|
@ -1173,7 +1173,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
*/
|
||||
default void preMergeRegions(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge) throws IOException {}
|
||||
final RegionInfo[] regionsToMerge) throws IOException {}
|
||||
|
||||
/**
|
||||
* called after merge regions request.
|
||||
|
@ -1182,7 +1182,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
*/
|
||||
default void postMergeRegions(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> c,
|
||||
final HRegionInfo[] regionsToMerge) throws IOException {}
|
||||
final RegionInfo[] regionsToMerge) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called before servers are moved to target region server group
|
||||
|
@ -1407,7 +1407,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
* @param ctx the environment to interact with the framework and master
|
||||
*/
|
||||
default void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
|
||||
TableName tableName, HRegionInfo[] regionInfos, LockType type,
|
||||
TableName tableName, RegionInfo[] regionInfos, LockType type,
|
||||
String description) throws IOException {}
|
||||
|
||||
/**
|
||||
|
@ -1415,7 +1415,7 @@ public interface MasterObserver extends Coprocessor {
|
|||
* @param ctx the environment to interact with the framework and master
|
||||
*/
|
||||
default void postRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
|
||||
TableName tableName, HRegionInfo[] regionInfos, LockType type,
|
||||
TableName tableName, RegionInfo[] regionInfos, LockType type,
|
||||
String description) throws IOException {}
|
||||
|
||||
/**
|
||||
|
|
|
@ -23,12 +23,12 @@ import java.util.concurrent.ConcurrentMap;
|
|||
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.metrics.MetricRegistry;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
|
@ -37,7 +37,7 @@ public interface RegionCoprocessorEnvironment extends CoprocessorEnvironment {
|
|||
Region getRegion();
|
||||
|
||||
/** @return region information for the region this coprocessor is running on */
|
||||
HRegionInfo getRegionInfo();
|
||||
RegionInfo getRegionInfo();
|
||||
|
||||
/** @return reference to the region server services */
|
||||
RegionServerServices getRegionServerServices();
|
||||
|
|
|
@ -30,9 +30,6 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CompareOperator;
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
|
@ -40,6 +37,7 @@ import org.apache.hadoop.hbase.client.Get;
|
|||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||
|
@ -63,6 +61,8 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Coprocessors implement this interface to observe and mediate client actions on the region.
|
||||
|
@ -943,7 +943,7 @@ public interface RegionObserver extends Coprocessor {
|
|||
* @param edits the file of recovered edits
|
||||
*/
|
||||
default void preReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, Path edits) throws IOException {}
|
||||
RegionInfo info, Path edits) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called after replaying WALs for this region.
|
||||
|
@ -952,7 +952,7 @@ public interface RegionObserver extends Coprocessor {
|
|||
* @param edits the file of recovered edits
|
||||
*/
|
||||
default void postReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, Path edits) throws IOException {}
|
||||
RegionInfo info, Path edits) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called before a {@link WALEdit}
|
||||
|
@ -960,7 +960,7 @@ public interface RegionObserver extends Coprocessor {
|
|||
* @param ctx the environment provided by the region server
|
||||
*/
|
||||
default void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
|
||||
RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called after a {@link WALEdit}
|
||||
|
@ -968,7 +968,7 @@ public interface RegionObserver extends Coprocessor {
|
|||
* @param ctx the environment provided by the region server
|
||||
*/
|
||||
default void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
|
||||
RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called before bulkLoadHFile. Users can create a StoreFile instance to
|
||||
|
|
|
@ -24,11 +24,11 @@ import java.io.IOException;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
/**
|
||||
* It's provided to have a way for coprocessors to observe, rewrite,
|
||||
|
@ -75,7 +75,7 @@ public interface WALObserver extends Coprocessor {
|
|||
*/
|
||||
// TODO: return value is not used
|
||||
default boolean preWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -84,7 +84,7 @@ public interface WALObserver extends Coprocessor {
|
|||
* is writen to WAL.
|
||||
*/
|
||||
default void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
|
||||
RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
|
||||
|
||||
/**
|
||||
* Called before rolling the current WAL
|
||||
|
|
|
@ -28,15 +28,14 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.MetaMutationAnnotation;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
|
||||
|
@ -57,6 +56,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
|||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Provides the coprocessor framework and environment for master oriented
|
||||
|
@ -242,7 +242,7 @@ public class MasterCoprocessorHost
|
|||
|
||||
/* Implementation of hooks for invoking MasterObservers */
|
||||
|
||||
public void preCreateTable(final TableDescriptor htd, final HRegionInfo[] regions)
|
||||
public void preCreateTable(final TableDescriptor htd, final RegionInfo[] regions)
|
||||
throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
|
@ -253,7 +253,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public void postCreateTable(final TableDescriptor htd, final HRegionInfo[] regions)
|
||||
public void postCreateTable(final TableDescriptor htd, final RegionInfo[] regions)
|
||||
throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
|
@ -264,7 +264,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public void preCreateTableAction(final TableDescriptor htd, final HRegionInfo[] regions,
|
||||
public void preCreateTableAction(final TableDescriptor htd, final RegionInfo[] regions,
|
||||
final User user)
|
||||
throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||
|
@ -277,7 +277,7 @@ public class MasterCoprocessorHost
|
|||
}
|
||||
|
||||
public void postCompletedCreateTableAction(
|
||||
final TableDescriptor htd, final HRegionInfo[] regions, final User user) throws IOException {
|
||||
final TableDescriptor htd, final RegionInfo[] regions, final User user) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||
@Override
|
||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||
|
@ -704,8 +704,8 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public boolean preMove(final HRegionInfo region, final ServerName srcServer,
|
||||
final ServerName destServer) throws IOException {
|
||||
public boolean preMove(final RegionInfo region, final ServerName srcServer,
|
||||
final ServerName destServer) throws IOException {
|
||||
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||
|
@ -715,7 +715,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public void postMove(final HRegionInfo region, final ServerName srcServer,
|
||||
public void postMove(final RegionInfo region, final ServerName srcServer,
|
||||
final ServerName destServer) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
|
@ -726,7 +726,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public boolean preAssign(final HRegionInfo regionInfo) throws IOException {
|
||||
public boolean preAssign(final RegionInfo regionInfo) throws IOException {
|
||||
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||
|
@ -736,7 +736,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public void postAssign(final HRegionInfo regionInfo) throws IOException {
|
||||
public void postAssign(final RegionInfo regionInfo) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||
|
@ -746,7 +746,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public boolean preUnassign(final HRegionInfo regionInfo, final boolean force)
|
||||
public boolean preUnassign(final RegionInfo regionInfo, final boolean force)
|
||||
throws IOException {
|
||||
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
|
@ -757,7 +757,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public void postUnassign(final HRegionInfo regionInfo, final boolean force) throws IOException {
|
||||
public void postUnassign(final RegionInfo regionInfo, final boolean force) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||
|
@ -767,7 +767,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public void preRegionOffline(final HRegionInfo regionInfo) throws IOException {
|
||||
public void preRegionOffline(final RegionInfo regionInfo) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||
|
@ -777,7 +777,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public void postRegionOffline(final HRegionInfo regionInfo) throws IOException {
|
||||
public void postRegionOffline(final RegionInfo regionInfo) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||
|
@ -787,7 +787,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public void preMergeRegions(final HRegionInfo[] regionsToMerge)
|
||||
public void preMergeRegions(final RegionInfo[] regionsToMerge)
|
||||
throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
|
@ -798,7 +798,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public void postMergeRegions(final HRegionInfo[] regionsToMerge)
|
||||
public void postMergeRegions(final RegionInfo[] regionsToMerge)
|
||||
throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
|
@ -897,8 +897,8 @@ public class MasterCoprocessorHost
|
|||
* @throws IOException
|
||||
*/
|
||||
public void postCompletedSplitRegionAction(
|
||||
final HRegionInfo regionInfoA,
|
||||
final HRegionInfo regionInfoB,
|
||||
final RegionInfo regionInfoA,
|
||||
final RegionInfo regionInfoB,
|
||||
final User user) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||
@Override
|
||||
|
@ -966,7 +966,7 @@ public class MasterCoprocessorHost
|
|||
* @throws IOException
|
||||
*/
|
||||
public boolean preMergeRegionsAction(
|
||||
final HRegionInfo[] regionsToMerge, final User user) throws IOException {
|
||||
final RegionInfo[] regionsToMerge, final User user) throws IOException {
|
||||
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||
@Override
|
||||
public void call(MasterObserver oserver,
|
||||
|
@ -984,8 +984,8 @@ public class MasterCoprocessorHost
|
|||
* @throws IOException
|
||||
*/
|
||||
public void postCompletedMergeRegionsAction(
|
||||
final HRegionInfo[] regionsToMerge,
|
||||
final HRegionInfo mergedRegion,
|
||||
final RegionInfo[] regionsToMerge,
|
||||
final RegionInfo mergedRegion,
|
||||
final User user) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||
@Override
|
||||
|
@ -1004,7 +1004,7 @@ public class MasterCoprocessorHost
|
|||
* @throws IOException
|
||||
*/
|
||||
public boolean preMergeRegionsCommit(
|
||||
final HRegionInfo[] regionsToMerge,
|
||||
final RegionInfo[] regionsToMerge,
|
||||
final @MetaMutationAnnotation List<Mutation> metaEntries,
|
||||
final User user) throws IOException {
|
||||
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||
|
@ -1024,8 +1024,8 @@ public class MasterCoprocessorHost
|
|||
* @throws IOException
|
||||
*/
|
||||
public void postMergeRegionsCommit(
|
||||
final HRegionInfo[] regionsToMerge,
|
||||
final HRegionInfo mergedRegion,
|
||||
final RegionInfo[] regionsToMerge,
|
||||
final RegionInfo mergedRegion,
|
||||
final User user) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||
@Override
|
||||
|
@ -1043,7 +1043,7 @@ public class MasterCoprocessorHost
|
|||
* @throws IOException
|
||||
*/
|
||||
public void postRollBackMergeRegionsAction(
|
||||
final HRegionInfo[] regionsToMerge, final User user) throws IOException {
|
||||
final RegionInfo[] regionsToMerge, final User user) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
|
||||
@Override
|
||||
public void call(MasterObserver oserver,
|
||||
|
@ -1809,7 +1809,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public void preRequestLock(String namespace, TableName tableName, HRegionInfo[] regionInfos,
|
||||
public void preRequestLock(String namespace, TableName tableName, RegionInfo[] regionInfos,
|
||||
LockType type, String description) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
|
@ -1820,7 +1820,7 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public void postRequestLock(String namespace, TableName tableName, HRegionInfo[] regionInfos,
|
||||
public void postRequestLock(String namespace, TableName tableName, RegionInfo[] regionInfos,
|
||||
LockType type, String description) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
|
||||
@Override
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
@ -32,6 +29,8 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.regex.Matcher;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Service;
|
||||
import org.apache.commons.collections4.map.AbstractReferenceMap;
|
||||
import org.apache.commons.collections4.map.ReferenceMap;
|
||||
import org.apache.commons.lang3.ClassUtils;
|
||||
|
@ -46,9 +45,6 @@ import org.apache.hadoop.hbase.Coprocessor;
|
|||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
|
@ -56,6 +52,7 @@ import org.apache.hadoop.hbase.client.Get;
|
|||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
|
@ -75,17 +72,17 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
|
|||
import org.apache.hadoop.hbase.metrics.MetricRegistry;
|
||||
import org.apache.hadoop.hbase.regionserver.Region.Operation;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Implements the coprocessor environment and runtime support for coprocessors
|
||||
|
@ -156,7 +153,7 @@ public class RegionCoprocessorHost
|
|||
}
|
||||
|
||||
@Override
|
||||
public HRegionInfo getRegionInfo() {
|
||||
public RegionInfo getRegionInfo() {
|
||||
return region.getRegionInfo();
|
||||
}
|
||||
|
||||
|
@ -1264,7 +1261,7 @@ public class RegionCoprocessorHost
|
|||
* @param edits the file of recovered edits
|
||||
* @throws IOException Exception
|
||||
*/
|
||||
public void preReplayWALs(final HRegionInfo info, final Path edits) throws IOException {
|
||||
public void preReplayWALs(final RegionInfo info, final Path edits) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
||||
@Override
|
||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||
|
@ -1279,7 +1276,7 @@ public class RegionCoprocessorHost
|
|||
* @param edits the file of recovered edits
|
||||
* @throws IOException Exception
|
||||
*/
|
||||
public void postReplayWALs(final HRegionInfo info, final Path edits) throws IOException {
|
||||
public void postReplayWALs(final RegionInfo info, final Path edits) throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
||||
@Override
|
||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||
|
@ -1296,7 +1293,7 @@ public class RegionCoprocessorHost
|
|||
* @return true if default behavior should be bypassed, false otherwise
|
||||
* @throws IOException
|
||||
*/
|
||||
public boolean preWALRestore(final HRegionInfo info, final WALKey logKey,
|
||||
public boolean preWALRestore(final RegionInfo info, final WALKey logKey,
|
||||
final WALEdit logEdit) throws IOException {
|
||||
return execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
||||
@Override
|
||||
|
@ -1313,7 +1310,7 @@ public class RegionCoprocessorHost
|
|||
* @param logEdit
|
||||
* @throws IOException
|
||||
*/
|
||||
public void postWALRestore(final HRegionInfo info, final WALKey logKey, final WALEdit logEdit)
|
||||
public void postWALRestore(final RegionInfo info, final WALKey logKey, final WALEdit logEdit)
|
||||
throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
||||
@Override
|
||||
|
|
|
@ -18,6 +18,20 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.security.access;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
|
@ -35,7 +49,6 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
|||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
|
@ -43,7 +56,6 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -54,6 +66,7 @@ import org.apache.hadoop.hbase.client.MasterSwitchType;
|
|||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Query;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
|
@ -93,8 +106,6 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
|
|||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
||||
import org.apache.hadoop.hbase.security.AccessDeniedException;
|
||||
|
@ -120,21 +131,9 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Provides basic authorization checks for data access and administrative
|
||||
|
@ -303,7 +302,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
AuthResult permissionGranted(String request, User user, Action permRequest,
|
||||
RegionCoprocessorEnvironment e,
|
||||
Map<byte [], ? extends Collection<?>> families) {
|
||||
HRegionInfo hri = e.getRegion().getRegionInfo();
|
||||
RegionInfo hri = e.getRegion().getRegionInfo();
|
||||
TableName tableName = hri.getTable();
|
||||
|
||||
// 1. All users need read access to hbase:meta table.
|
||||
|
@ -645,7 +644,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
RegionCoprocessorEnvironment env,
|
||||
Map<byte[], ? extends Collection<byte[]>> familyMap)
|
||||
throws IOException {
|
||||
HRegionInfo hri = env.getRegion().getRegionInfo();
|
||||
RegionInfo hri = env.getRegion().getRegionInfo();
|
||||
TableName tableName = hri.getTable();
|
||||
|
||||
if (user == null) {
|
||||
|
@ -988,7 +987,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
|
||||
@Override
|
||||
public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> c,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {
|
||||
Set<byte[]> families = desc.getColumnFamilyNames();
|
||||
Map<byte[], Set<byte[]>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
||||
for (byte[] family: families) {
|
||||
|
@ -1002,7 +1001,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
public void postCompletedCreateTableAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> c,
|
||||
final TableDescriptor desc,
|
||||
final HRegionInfo[] regions) throws IOException {
|
||||
final RegionInfo[] regions) throws IOException {
|
||||
// When AC is used, it should be configured as the 1st CP.
|
||||
// In Master, the table operations like create, are handled by a Thread pool but the max size
|
||||
// for this pool is 1. So if multiple CPs create tables on startup, these creations will happen
|
||||
|
@ -1251,26 +1250,26 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
}
|
||||
|
||||
@Override
|
||||
public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo region,
|
||||
public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo region,
|
||||
ServerName srcServer, ServerName destServer) throws IOException {
|
||||
requirePermission(getActiveUser(c), "move", region.getTable(), null, null, Action.ADMIN);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo)
|
||||
public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo)
|
||||
throws IOException {
|
||||
requirePermission(getActiveUser(c), "assign", regionInfo.getTable(), null, null, Action.ADMIN);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo,
|
||||
public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo,
|
||||
boolean force) throws IOException {
|
||||
requirePermission(getActiveUser(c), "unassign", regionInfo.getTable(), null, null, Action.ADMIN);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> c,
|
||||
HRegionInfo regionInfo) throws IOException {
|
||||
RegionInfo regionInfo) throws IOException {
|
||||
requirePermission(getActiveUser(c), "regionOffline", regionInfo.getTable(), null, null,
|
||||
Action.ADMIN);
|
||||
}
|
||||
|
@ -1478,7 +1477,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
if (region == null) {
|
||||
LOG.error("NULL region from RegionCoprocessorEnvironment in preOpen()");
|
||||
} else {
|
||||
HRegionInfo regionInfo = region.getRegionInfo();
|
||||
RegionInfo regionInfo = region.getRegionInfo();
|
||||
if (regionInfo.getTable().isSystemTable()) {
|
||||
checkSystemOrSuperUser(getActiveUser(c));
|
||||
} else {
|
||||
|
@ -2467,7 +2466,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
}
|
||||
|
||||
private TableName getTableName(Region region) {
|
||||
HRegionInfo regionInfo = region.getRegionInfo();
|
||||
RegionInfo regionInfo = region.getRegionInfo();
|
||||
if (regionInfo != null) {
|
||||
return regionInfo.getTable();
|
||||
}
|
||||
|
@ -2569,7 +2568,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
|
||||
@Override
|
||||
public void preMergeRegions(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge) throws IOException {
|
||||
final RegionInfo[] regionsToMerge) throws IOException {
|
||||
requirePermission(getActiveUser(ctx), "mergeRegions", regionsToMerge[0].getTable(), null, null,
|
||||
Action.ADMIN);
|
||||
}
|
||||
|
@ -2718,7 +2717,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
|
||||
@Override
|
||||
public void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
|
||||
TableName tableName, HRegionInfo[] regionInfos, LockType type, String description)
|
||||
TableName tableName, RegionInfo[] regionInfos, LockType type, String description)
|
||||
throws IOException {
|
||||
// There are operations in the CREATE and ADMIN domain which may require lock, READ
|
||||
// or WRITE. So for any lock request, we check for these two perms irrespective of lock type.
|
||||
|
@ -2734,7 +2733,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
}
|
||||
|
||||
private void checkLockPermissions(User user, String namespace,
|
||||
TableName tableName, HRegionInfo[] regionInfos, String reason)
|
||||
TableName tableName, RegionInfo[] regionInfos, String reason)
|
||||
throws IOException {
|
||||
if (namespace != null && !namespace.isEmpty()) {
|
||||
requireNamespacePermission(user, reason, namespace, Action.ADMIN, Action.CREATE);
|
||||
|
|
|
@ -25,21 +25,19 @@ import java.util.regex.Matcher;
|
|||
import org.apache.commons.io.FilenameUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Master observer for restricting coprocessor assignments.
|
||||
|
@ -61,7 +59,7 @@ public class CoprocessorWhitelistMasterObserver implements MasterObserver {
|
|||
|
||||
@Override
|
||||
public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
TableDescriptor htd, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor htd, RegionInfo[] regions) throws IOException {
|
||||
verifyCoprocessors(ctx, htd);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,10 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
@ -42,6 +38,9 @@ import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
|||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -53,9 +52,9 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
@Category({ MasterTests.class, MediumTests.class })
|
||||
public class TestEnableTable {
|
||||
|
@ -198,7 +197,7 @@ public class TestEnableTable {
|
|||
public void postCompletedCreateTableAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final TableDescriptor desc,
|
||||
final HRegionInfo[] regions) throws IOException {
|
||||
final RegionInfo[] regions) throws IOException {
|
||||
// the AccessController test, some times calls only and directly the
|
||||
// postCompletedCreateTableAction()
|
||||
if (tableCreationLatch != null) {
|
||||
|
|
|
@ -28,10 +28,10 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
|
||||
/**
|
||||
|
@ -83,13 +83,13 @@ public class SampleRegionWALObserver implements WALObserver, RegionObserver {
|
|||
|
||||
@Override
|
||||
public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> env,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
postWALWriteCalled = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean preWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> env,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
boolean bypass = false;
|
||||
// check table name matches or not.
|
||||
if (!Bytes.equals(info.getTable().toBytes(), this.tableName)) {
|
||||
|
@ -132,7 +132,7 @@ public class SampleRegionWALObserver implements WALObserver, RegionObserver {
|
|||
*/
|
||||
@Override
|
||||
public void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
preWALRestoreCalled = true;
|
||||
}
|
||||
|
||||
|
@ -154,7 +154,7 @@ public class SampleRegionWALObserver implements WALObserver, RegionObserver {
|
|||
*/
|
||||
@Override
|
||||
public void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
postWALRestoreCalled = true;
|
||||
}
|
||||
|
||||
|
|
|
@ -19,11 +19,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -37,7 +32,6 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CompareOperator;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
|
@ -45,6 +39,7 @@ import org.apache.hadoop.hbase.client.Get;
|
|||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||
|
@ -62,13 +57,16 @@ import org.apache.hadoop.hbase.regionserver.Store;
|
|||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileReader;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
/**
|
||||
* A sample region observer that tests the RegionObserver interface.
|
||||
|
@ -619,19 +617,19 @@ public class SimpleRegionObserver implements RegionObserver {
|
|||
|
||||
@Override
|
||||
public void preReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> env,
|
||||
HRegionInfo info, Path edits) throws IOException {
|
||||
RegionInfo info, Path edits) throws IOException {
|
||||
ctPreReplayWALs.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> env,
|
||||
HRegionInfo info, Path edits) throws IOException {
|
||||
RegionInfo info, Path edits) throws IOException {
|
||||
ctPostReplayWALs.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
String tableName = logKey.getTablename().getNameAsString();
|
||||
if (tableName.equals(TABLE_SKIPPED)) {
|
||||
// skip recovery of TABLE_SKIPPED for testing purpose
|
||||
|
@ -643,7 +641,7 @@ public class SimpleRegionObserver implements RegionObserver {
|
|||
|
||||
@Override
|
||||
public void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
ctPostWALRestore.incrementAndGet();
|
||||
}
|
||||
|
||||
|
|
|
@ -18,16 +18,13 @@
|
|||
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -35,7 +32,6 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -45,6 +41,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
|
|||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
|
@ -61,23 +58,25 @@ import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRo
|
|||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
/**
|
||||
* Testing of coprocessor metrics end-to-end.
|
||||
*/
|
||||
|
@ -102,14 +101,14 @@ public class TestCoprocessorMetrics {
|
|||
|
||||
@Override
|
||||
public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {
|
||||
// we rely on the fact that there is only 1 instance of our MasterObserver
|
||||
this.start = System.currentTimeMillis();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {
|
||||
if (this.start > 0) {
|
||||
long time = System.currentTimeMillis() - start;
|
||||
LOG.info("Create table took: " + time);
|
||||
|
@ -162,7 +161,7 @@ public class TestCoprocessorMetrics {
|
|||
|
||||
@Override
|
||||
public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, org.apache.hadoop.hbase.wal.WALKey logKey,
|
||||
RegionInfo info, org.apache.hadoop.hbase.wal.WALKey logKey,
|
||||
WALEdit logEdit) throws IOException {
|
||||
walEditsCount.increment();
|
||||
}
|
||||
|
|
|
@ -19,11 +19,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -32,11 +27,11 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
|||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
|
@ -50,6 +45,11 @@ import org.junit.BeforeClass;
|
|||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
/**
|
||||
* Tests unhandled exceptions thrown by coprocessors running on master.
|
||||
* Expected result is that the master will abort with an informative
|
||||
|
@ -105,7 +105,7 @@ public class TestMasterCoprocessorExceptionWithAbort {
|
|||
|
||||
@Override
|
||||
public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {
|
||||
// cause a NullPointerException and don't catch it: this will cause the
|
||||
// master to abort().
|
||||
Integer i;
|
||||
|
|
|
@ -19,10 +19,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -30,11 +26,11 @@ import org.apache.hadoop.hbase.Abortable;
|
|||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
|
@ -48,6 +44,10 @@ import org.junit.BeforeClass;
|
|||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
/**
|
||||
* Tests unhandled exceptions thrown by coprocessors running on master.
|
||||
* Expected result is that the master will remove the buggy coprocessor from
|
||||
|
@ -82,7 +82,7 @@ public class TestMasterCoprocessorExceptionWithRemove {
|
|||
@SuppressWarnings("null")
|
||||
@Override
|
||||
public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {
|
||||
// Cause a NullPointerException and don't catch it: this should cause the
|
||||
// master to throw an o.apache.hadoop.hbase.DoNotRetryIOException to the
|
||||
// client.
|
||||
|
|
|
@ -19,11 +19,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -50,6 +45,7 @@ import org.apache.hadoop.hbase.client.Connection;
|
|||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
|
@ -82,6 +78,11 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
|
||||
/**
|
||||
* Tests invocation of the {@link org.apache.hadoop.hbase.coprocessor.MasterObserver}
|
||||
|
@ -284,14 +285,14 @@ public class TestMasterObserver {
|
|||
@Override
|
||||
public void preMergeRegions(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge) throws IOException {
|
||||
final RegionInfo[] regionsToMerge) throws IOException {
|
||||
preMergeRegionsCalled = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postMergeRegions(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge) throws IOException {
|
||||
final RegionInfo[] regionsToMerge) throws IOException {
|
||||
postMergeRegionsCalled = true;
|
||||
}
|
||||
|
||||
|
@ -301,7 +302,7 @@ public class TestMasterObserver {
|
|||
|
||||
@Override
|
||||
public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {
|
||||
if (bypass) {
|
||||
env.bypass();
|
||||
}
|
||||
|
@ -310,7 +311,7 @@ public class TestMasterObserver {
|
|||
|
||||
@Override
|
||||
public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {
|
||||
postCreateTableCalled = true;
|
||||
}
|
||||
|
||||
|
@ -697,7 +698,7 @@ public class TestMasterObserver {
|
|||
|
||||
@Override
|
||||
public void preMove(ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
HRegionInfo region, ServerName srcServer, ServerName destServer)
|
||||
RegionInfo region, ServerName srcServer, ServerName destServer)
|
||||
throws IOException {
|
||||
if (bypass) {
|
||||
env.bypass();
|
||||
|
@ -706,7 +707,7 @@ public class TestMasterObserver {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void postMove(ObserverContext<MasterCoprocessorEnvironment> env, HRegionInfo region,
|
||||
public void postMove(ObserverContext<MasterCoprocessorEnvironment> env, RegionInfo region,
|
||||
ServerName srcServer, ServerName destServer)
|
||||
throws IOException {
|
||||
postMoveCalled = true;
|
||||
|
@ -722,7 +723,7 @@ public class TestMasterObserver {
|
|||
|
||||
@Override
|
||||
public void preAssign(ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
final HRegionInfo regionInfo) throws IOException {
|
||||
final RegionInfo regionInfo) throws IOException {
|
||||
if (bypass) {
|
||||
env.bypass();
|
||||
}
|
||||
|
@ -731,7 +732,7 @@ public class TestMasterObserver {
|
|||
|
||||
@Override
|
||||
public void postAssign(ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
final HRegionInfo regionInfo) throws IOException {
|
||||
final RegionInfo regionInfo) throws IOException {
|
||||
postAssignCalled = true;
|
||||
}
|
||||
|
||||
|
@ -745,7 +746,7 @@ public class TestMasterObserver {
|
|||
|
||||
@Override
|
||||
public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
final HRegionInfo regionInfo, final boolean force) throws IOException {
|
||||
final RegionInfo regionInfo, final boolean force) throws IOException {
|
||||
if (bypass) {
|
||||
env.bypass();
|
||||
}
|
||||
|
@ -754,7 +755,7 @@ public class TestMasterObserver {
|
|||
|
||||
@Override
|
||||
public void postUnassign(ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
final HRegionInfo regionInfo, final boolean force) throws IOException {
|
||||
final RegionInfo regionInfo, final boolean force) throws IOException {
|
||||
postUnassignCalled = true;
|
||||
}
|
||||
|
||||
|
@ -768,13 +769,13 @@ public class TestMasterObserver {
|
|||
|
||||
@Override
|
||||
public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
final HRegionInfo regionInfo) throws IOException {
|
||||
final RegionInfo regionInfo) throws IOException {
|
||||
preRegionOfflineCalled = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postRegionOffline(ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
final HRegionInfo regionInfo) throws IOException {
|
||||
final RegionInfo regionInfo) throws IOException {
|
||||
postRegionOfflineCalled = true;
|
||||
}
|
||||
|
||||
|
@ -977,7 +978,7 @@ public class TestMasterObserver {
|
|||
public void preCreateTableAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> env,
|
||||
final TableDescriptor desc,
|
||||
final HRegionInfo[] regions) throws IOException {
|
||||
final RegionInfo[] regions) throws IOException {
|
||||
if (bypass) {
|
||||
env.bypass();
|
||||
}
|
||||
|
@ -988,7 +989,7 @@ public class TestMasterObserver {
|
|||
public void postCompletedCreateTableAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final TableDescriptor desc,
|
||||
final HRegionInfo[] regions) throws IOException {
|
||||
final RegionInfo[] regions) throws IOException {
|
||||
postCompletedCreateTableActionCalled = true;
|
||||
tableCreationLatch.countDown();
|
||||
}
|
||||
|
@ -1368,14 +1369,14 @@ public class TestMasterObserver {
|
|||
|
||||
@Override
|
||||
public void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
|
||||
TableName tableName, HRegionInfo[] regionInfos, LockType type,
|
||||
TableName tableName, RegionInfo[] regionInfos, LockType type,
|
||||
String description) throws IOException {
|
||||
preRequestLockCalled = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
|
||||
TableName tableName, HRegionInfo[] regionInfos, LockType type,
|
||||
TableName tableName, RegionInfo[] regionInfos, LockType type,
|
||||
String description) throws IOException {
|
||||
postRequestLockCalled = true;
|
||||
}
|
||||
|
@ -1414,8 +1415,8 @@ public class TestMasterObserver {
|
|||
@Override
|
||||
public void postCompletedSplitRegionAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> c,
|
||||
final HRegionInfo regionInfoA,
|
||||
final HRegionInfo regionInfoB) throws IOException {
|
||||
final RegionInfo regionInfoA,
|
||||
final RegionInfo regionInfoB) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1438,34 +1439,34 @@ public class TestMasterObserver {
|
|||
@Override
|
||||
public void preMergeRegionsAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge) throws IOException {
|
||||
final RegionInfo[] regionsToMerge) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postCompletedMergeRegionsAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> c,
|
||||
final HRegionInfo[] regionsToMerge,
|
||||
final HRegionInfo mergedRegion) throws IOException {
|
||||
final RegionInfo[] regionsToMerge,
|
||||
final RegionInfo mergedRegion) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preMergeRegionsCommitAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge,
|
||||
final RegionInfo[] regionsToMerge,
|
||||
final List<Mutation> metaEntries) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postMergeRegionsCommitAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge,
|
||||
final HRegionInfo mergedRegion) throws IOException {
|
||||
final RegionInfo[] regionsToMerge,
|
||||
final RegionInfo mergedRegion) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postRollBackMergeRegionsAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge) throws IOException {
|
||||
final RegionInfo[] regionsToMerge) throws IOException {
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
@ -31,20 +29,21 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
|
@ -54,7 +53,8 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
|
||||
@Category(MediumTests.class)
|
||||
public class TestRegionObserverForAddingMutationsFromCoprocessors {
|
||||
|
@ -272,7 +272,7 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors {
|
|||
static WALEdit savedEdit = null;
|
||||
@Override
|
||||
public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
if (info.getTable().equals(TableName.valueOf("testCPMutationsAreWrittenToWALEdit"))) {
|
||||
savedEdit = logEdit;
|
||||
}
|
||||
|
|
|
@ -18,13 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.namespace;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -54,6 +47,7 @@ import org.apache.hadoop.hbase.Waiter;
|
|||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
|
@ -87,6 +81,13 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestRule;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
@Category(MediumTests.class)
|
||||
public class TestNamespaceAuditor {
|
||||
@Rule public final TestRule timeout = CategoryBasedTimeout.builder().
|
||||
|
@ -303,7 +304,7 @@ public class TestNamespaceAuditor {
|
|||
@Override
|
||||
public synchronized void preMergeRegionsAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final HRegionInfo[] regionsToMerge) throws IOException {
|
||||
final RegionInfo[] regionsToMerge) throws IOException {
|
||||
notifyAll();
|
||||
if (shouldFailMerge) {
|
||||
throw new IOException("fail merge");
|
||||
|
@ -540,7 +541,7 @@ public class TestNamespaceAuditor {
|
|||
|
||||
@Override
|
||||
public void preCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {
|
||||
if (throwExceptionInPreCreateTableAction) {
|
||||
throw new IOException("Throw exception as it is demanded.");
|
||||
}
|
||||
|
|
|
@ -17,12 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.closeRegion;
|
||||
import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.openRegion;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Queue;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
|
@ -39,6 +33,7 @@ import org.apache.hadoop.hbase.RegionLocations;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
|
@ -47,22 +42,23 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
|||
import org.apache.hadoop.hbase.coprocessor.WALCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.WALObserver;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationEndpoint.ReplicateContext;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeer;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
||||
import org.apache.hadoop.hbase.replication.WALEntryFilter;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.RegionReplicaReplicationEndpoint.RegionReplicaReplayCallable;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -72,7 +68,11 @@ import org.junit.BeforeClass;
|
|||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.closeRegion;
|
||||
import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.openRegion;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* Tests RegionReplicaReplicationEndpoint. Unlike TestRegionReplicaReplicationEndpoint this
|
||||
|
@ -154,7 +154,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
|
|||
}
|
||||
@Override
|
||||
public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
// only keep primary region's edits
|
||||
if (logKey.getTablename().equals(tableName) && info.getReplicaId() == 0) {
|
||||
entries.add(new Entry(logKey, logEdit));
|
||||
|
|
|
@ -18,9 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.security.access;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
import java.security.PrivilegedActionException;
|
||||
|
@ -30,30 +27,32 @@ import java.util.Map;
|
|||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import com.google.protobuf.BlockingRpcChannel;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotEnabledException;
|
||||
import org.apache.hadoop.hbase.Waiter.Predicate;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
||||
|
@ -63,12 +62,12 @@ import org.apache.hadoop.hbase.regionserver.Region;
|
|||
import org.apache.hadoop.hbase.security.AccessDeniedException;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.security.access.Permission.Action;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||
|
||||
import com.google.protobuf.BlockingRpcChannel;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
/**
|
||||
* Utility methods for testing security
|
||||
|
@ -630,7 +629,7 @@ public class SecureTestUtil {
|
|||
@Override
|
||||
public void postCompletedCreateTableAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
TableDescriptor desc, HRegionInfo[] regions) throws IOException {
|
||||
TableDescriptor desc, RegionInfo[] regions) throws IOException {
|
||||
// the AccessController test, some times calls only and directly the
|
||||
// postCompletedCreateTableAction()
|
||||
if (tableCreationLatch != null) {
|
||||
|
|
|
@ -18,9 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
|
@ -40,36 +37,37 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ClusterStatus;
|
||||
import org.apache.hadoop.hbase.ClusterStatus.Option;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ClusterStatus.Option;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.mob.MobFileName;
|
||||
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.assignment.RegionStates;
|
||||
import org.apache.hadoop.hbase.mob.MobFileName;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
|
||||
import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
|
||||
import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo;
|
||||
|
@ -77,6 +75,9 @@ import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
|
|||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
/**
|
||||
* This is the base class for HBaseFsck's ability to detect reasons for inconsistent tables.
|
||||
*
|
||||
|
@ -601,7 +602,7 @@ public class BaseTestHBaseFsck {
|
|||
public void postCompletedCreateTableAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final TableDescriptor desc,
|
||||
final HRegionInfo[] regions) throws IOException {
|
||||
final RegionInfo[] regions) throws IOException {
|
||||
// the AccessController test, some times calls only and directly the
|
||||
// postCompletedCreateTableAction()
|
||||
if (tableCreationLatch != null) {
|
||||
|
|
Loading…
Reference in New Issue