HBASE-18823 Apply RegionInfo to MasterObserver/RegionObserver/WALObserver

This commit is contained in:
Chia-Ping Tsai 2017-09-19 01:21:55 +08:00 committed by Apekshit Sharma
parent 37696fffe9
commit a6c3c645fd
23 changed files with 263 additions and 272 deletions

View File

@ -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()) {

View File

@ -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);

View File

@ -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);
}

View File

@ -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 {}
/**

View File

@ -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();

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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);

View File

@ -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);
}

View File

@ -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) {

View File

@ -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;
}

View File

@ -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();
}

View File

@ -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();
}

View File

@ -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;

View File

@ -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.

View File

@ -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 {
}
}

View File

@ -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;
}

View File

@ -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.");
}

View File

@ -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));

View File

@ -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) {

View File

@ -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) {