diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java index 6d9efadef212..a86cfde090f4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java @@ -19,7 +19,9 @@ import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.Optional; +import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -30,7 +32,12 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.BalanceRequest; +import org.apache.hadoop.hbase.client.CheckAndMutate; +import org.apache.hadoop.hbase.client.CheckAndMutateResult; import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +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; @@ -55,14 +62,23 @@ import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterServices; +import org.apache.hadoop.hbase.net.Address; +import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings; import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker; +import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; +import org.apache.hadoop.hbase.regionserver.ScanOptions; +import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.util.FSUtils; 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.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -107,52 +123,136 @@ public void start(CoprocessorEnvironment env) throws IOException { public void stop(CoprocessorEnvironment env) { } + @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.UNITTEST) + public void setGlobalReadOnlyEnabled(boolean enabled) { + this.globalReadOnlyEnabled = enabled; + } + /* ---- RegionObserver Overrides ---- */ @Override public Optional getRegionObserver() { return Optional.of(this); } + @Override + public void preFlush(final ObserverContext c, + FlushLifeCycleTracker tracker) throws IOException { + internalReadOnlyGuard(); + RegionObserver.super.preFlush(c, tracker); + } + + @Override + public void preFlushScannerOpen(ObserverContext c, + Store store, ScanOptions options, FlushLifeCycleTracker tracker) throws IOException { + internalReadOnlyGuard(); + RegionObserver.super.preFlushScannerOpen(c, store, options, tracker); + } + + @Override + public InternalScanner preFlush(ObserverContext c, + Store store, InternalScanner scanner, FlushLifeCycleTracker tracker) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preFlush(c, store, scanner, tracker); + } + + @Override + public void preMemStoreCompaction(ObserverContext c, + Store store) throws IOException { + internalReadOnlyGuard(); + RegionObserver.super.preMemStoreCompaction(c, store); + } + + @Override + public void preMemStoreCompactionCompactScannerOpen( + ObserverContext c, Store store, ScanOptions options) + throws IOException { + internalReadOnlyGuard(); + RegionObserver.super.preMemStoreCompactionCompactScannerOpen(c, store, options); + } + + @Override + public InternalScanner preMemStoreCompactionCompact( + ObserverContext c, Store store, InternalScanner scanner) + throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preMemStoreCompactionCompact(c, store, scanner); + } + + @Override + public void preCompactSelection(ObserverContext c, + Store store, List candidates, CompactionLifeCycleTracker tracker) + throws IOException { + internalReadOnlyGuard(); + RegionObserver.super.preCompactSelection(c, store, candidates, tracker); + } + + @Override + public void preCompactScannerOpen(ObserverContext c, + Store store, ScanType scanType, ScanOptions options, CompactionLifeCycleTracker tracker, + CompactionRequest request) throws IOException { + internalReadOnlyGuard(); + RegionObserver.super.preCompactScannerOpen(c, store, scanType, options, tracker, request); + } + + @Override + public InternalScanner preCompact(ObserverContext c, + Store store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, + CompactionRequest request) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preCompact(c, store, scanner, scanType, tracker, request); + } + @Override public void prePut(ObserverContext c, Put put, - WALEdit edit) throws IOException { + WALEdit edit, Durability durability) throws IOException { TableName tableName = c.getEnvironment().getRegionInfo().getTable(); if (tableName.isSystemTable()) { return; } internalReadOnlyGuard(); + RegionObserver.super.prePut(c, put, edit, durability); } @Override - public void preDelete(ObserverContext c, Delete delete, + public void prePut(ObserverContext c, Put put, WALEdit edit) throws IOException { - if (c.getEnvironment().getRegionInfo().getTable().isSystemTable()) { + TableName tableName = c.getEnvironment().getRegionInfo().getTable(); + if (tableName.isSystemTable()) { return; } internalReadOnlyGuard(); + RegionObserver.super.prePut(c, put, edit); } @Override - public void preBatchMutate(ObserverContext c, - MiniBatchOperationInProgress miniBatchOp) throws IOException { - TableName tableName = c.getEnvironment().getRegionInfo().getTable(); - if (tableName.isSystemTable()) { + public void preDelete(ObserverContext c, Delete delete, + WALEdit edit, Durability durability) throws IOException { + if (c.getEnvironment().getRegionInfo().getTable().isSystemTable()) { return; } internalReadOnlyGuard(); + RegionObserver.super.preDelete(c, delete, edit, durability); } @Override - public void preFlush(final ObserverContext c, - FlushLifeCycleTracker tracker) throws IOException { + public void preDelete(ObserverContext c, Delete delete, + WALEdit edit) throws IOException { + if (c.getEnvironment().getRegionInfo().getTable().isSystemTable()) { + return; + } internalReadOnlyGuard(); + RegionObserver.super.preDelete(c, delete, edit); } @Override - public void preCompactSelection(ObserverContext c, - Store store, List candidates, CompactionLifeCycleTracker tracker) - throws IOException { + public void preBatchMutate(ObserverContext c, + MiniBatchOperationInProgress miniBatchOp) throws IOException { + TableName tableName = c.getEnvironment().getRegionInfo().getTable(); + if (tableName.isSystemTable()) { + return; + } internalReadOnlyGuard(); + RegionObserver.super.preBatchMutate(c, miniBatchOp); } @Override @@ -231,6 +331,22 @@ public boolean preCheckAndDeleteAfterRowLock( return RegionObserver.super.preCheckAndDeleteAfterRowLock(c, row, filter, delete, result); } + @Override + public CheckAndMutateResult preCheckAndMutate( + ObserverContext c, CheckAndMutate checkAndMutate, + CheckAndMutateResult result) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preCheckAndMutate(c, checkAndMutate, result); + } + + @Override + public CheckAndMutateResult preCheckAndMutateAfterRowLock( + ObserverContext c, CheckAndMutate checkAndMutate, + CheckAndMutateResult result) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preCheckAndMutateAfterRowLock(c, checkAndMutate, result); + } + @Override public Result preAppend(ObserverContext c, Append append) throws IOException { @@ -252,6 +368,34 @@ public Result preAppendAfterRowLock(ObserverContext c, + Increment increment) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preIncrement(c, increment); + } + + @Override + public Result preIncrement(ObserverContext c, + Increment increment, WALEdit edit) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preIncrement(c, increment, edit); + } + + @Override + public Result preIncrementAfterRowLock(ObserverContext c, + Increment increment) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preIncrementAfterRowLock(c, increment); + } + + @Override + public void preReplayWALs(ObserverContext ctx, + RegionInfo info, Path edits) throws IOException { + internalReadOnlyGuard(); + RegionObserver.super.preReplayWALs(ctx, info, edits); + } + @Override public void preBulkLoadHFile(ObserverContext ctx, List> familyPaths) throws IOException { @@ -259,12 +403,33 @@ public void preBulkLoadHFile(ObserverContext ctx, + byte[] family, List> pairs) throws IOException { + internalReadOnlyGuard(); + RegionObserver.super.preCommitStoreFile(ctx, family, pairs); + } + + @Override + public void preWALAppend(ObserverContext ctx, WALKey key, + WALEdit edit) throws IOException { + internalReadOnlyGuard(); + RegionObserver.super.preWALAppend(ctx, key, edit); + } + /* ---- MasterObserver Overrides ---- */ @Override public Optional getMasterObserver() { return Optional.of(this); } + @Override + public TableDescriptor preCreateTableRegionsInfos( + ObserverContext ctx, TableDescriptor desc) throws IOException { + internalReadOnlyGuard(); + return MasterObserver.super.preCreateTableRegionsInfos(ctx, desc); + } + @Override public void preCreateTable(ObserverContext ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException { @@ -272,6 +437,13 @@ public void preCreateTable(ObserverContext ctx, MasterObserver.super.preCreateTable(ctx, desc, regions); } + @Override + public void preCreateTableAction(ObserverContext ctx, + TableDescriptor desc, RegionInfo[] regions) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preCreateTableAction(ctx, desc, regions); + } + @Override public void preDeleteTable(ObserverContext ctx, TableName tableName) throws IOException { @@ -308,6 +480,96 @@ public TableDescriptor preModifyTable(ObserverContext ctx, + TableName tableName, String dstSFT) throws IOException { + internalReadOnlyGuard(); + return MasterObserver.super.preModifyTableStoreFileTracker(ctx, tableName, dstSFT); + } + + @Override + public String preModifyColumnFamilyStoreFileTracker( + ObserverContext ctx, TableName tableName, byte[] family, + String dstSFT) throws IOException { + internalReadOnlyGuard(); + return MasterObserver.super.preModifyColumnFamilyStoreFileTracker(ctx, tableName, family, + dstSFT); + } + + @Override + public void preModifyTableAction(ObserverContext ctx, + TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) + throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preModifyTableAction(ctx, tableName, currentDescriptor, newDescriptor); + } + + @Override + public void preSplitRegion(ObserverContext c, TableName tableName, + byte[] splitRow) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preSplitRegion(c, tableName, splitRow); + } + + @Override + public void preSplitRegionAction(ObserverContext c, + TableName tableName, byte[] splitRow) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preSplitRegionAction(c, tableName, splitRow); + } + + @Override + public void preSplitRegionBeforeMETAAction(ObserverContext ctx, + byte[] splitKey, List metaEntries) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preSplitRegionBeforeMETAAction(ctx, splitKey, metaEntries); + } + + @Override + public void preSplitRegionAfterMETAAction(ObserverContext ctx) + throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preSplitRegionAfterMETAAction(ctx); + } + + @Override + public void preTruncateRegion(ObserverContext c, + RegionInfo regionInfo) { + try { + internalReadOnlyGuard(); + } catch (IOException e) { + LOG.info("Region truncation of region {} not allowed in read-only mode", + regionInfo.getRegionNameAsString()); + } + MasterObserver.super.preTruncateRegion(c, regionInfo); + } + + @Override + public void preTruncateRegionAction(ObserverContext c, + RegionInfo regionInfo) { + try { + internalReadOnlyGuard(); + } catch (IOException e) { + LOG.info("Region truncation of region {} not allowed in read-only mode", + regionInfo.getRegionNameAsString()); + } + MasterObserver.super.preTruncateRegionAction(c, regionInfo); + } + + @Override + public void preMergeRegionsAction(final ObserverContext ctx, + final RegionInfo[] regionsToMerge) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preMergeRegionsAction(ctx, regionsToMerge); + } + + @Override + public void preMergeRegionsCommitAction(ObserverContext ctx, + RegionInfo[] regionsToMerge, List metaEntries) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preMergeRegionsCommitAction(ctx, regionsToMerge, metaEntries); + } + @Override public void preSnapshot(ObserverContext ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException { @@ -359,25 +621,187 @@ public void preDeleteNamespace(ObserverContext ctx } @Override - public void preMergeRegionsAction(ObserverContext ctx, - RegionInfo[] regionsToMerge) throws IOException { + public void preMasterStoreFlush(ObserverContext ctx) + throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preMasterStoreFlush(ctx); + } + + @Override + public void preSetUserQuota(ObserverContext ctx, String userName, + GlobalQuotaSettings quotas) throws IOException { internalReadOnlyGuard(); - MasterObserver.super.preMergeRegionsAction(ctx, regionsToMerge); + MasterObserver.super.preSetUserQuota(ctx, userName, quotas); } - /* ---- RegionServerObserver Overrides ---- */ @Override - public void preRollWALWriterRequest(ObserverContext ctx) + public void preSetUserQuota(ObserverContext ctx, String userName, + TableName tableName, GlobalQuotaSettings quotas) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preSetUserQuota(ctx, userName, tableName, quotas); + } + + @Override + public void preSetUserQuota(ObserverContext ctx, String userName, + String namespace, GlobalQuotaSettings quotas) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preSetUserQuota(ctx, userName, namespace, quotas); + } + + @Override + public void preSetTableQuota(ObserverContext ctx, + TableName tableName, GlobalQuotaSettings quotas) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preSetTableQuota(ctx, tableName, quotas); + } + + @Override + public void preSetNamespaceQuota(ObserverContext ctx, + String namespace, GlobalQuotaSettings quotas) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preSetNamespaceQuota(ctx, namespace, quotas); + } + + @Override + public void preSetRegionServerQuota(ObserverContext ctx, + String regionServer, GlobalQuotaSettings quotas) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preSetRegionServerQuota(ctx, regionServer, quotas); + } + + @Override + public void preMergeRegions(final ObserverContext ctx, + final RegionInfo[] regionsToMerge) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preMergeRegions(ctx, regionsToMerge); + } + + @Override + public void preMoveServersAndTables(ObserverContext ctx, + Set
servers, Set tables, String targetGroup) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preMoveServersAndTables(ctx, servers, tables, targetGroup); + } + + @Override + public void preMoveServers(ObserverContext ctx, + Set
servers, String targetGroup) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preMoveServers(ctx, servers, targetGroup); + } + + @Override + public void preMoveTables(ObserverContext ctx, + Set tables, String targetGroup) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preMoveTables(ctx, tables, targetGroup); + } + + @Override + public void preAddRSGroup(ObserverContext ctx, String name) throws IOException { internalReadOnlyGuard(); - RegionServerObserver.super.preRollWALWriterRequest(ctx); + MasterObserver.super.preAddRSGroup(ctx, name); } @Override - public void preClearCompactionQueues(ObserverContext ctx) + public void preRemoveRSGroup(ObserverContext ctx, String name) throws IOException { internalReadOnlyGuard(); - RegionServerObserver.super.preClearCompactionQueues(ctx); + MasterObserver.super.preRemoveRSGroup(ctx, name); + } + + @Override + public void preBalanceRSGroup(ObserverContext ctx, String groupName, + BalanceRequest request) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preBalanceRSGroup(ctx, groupName, request); + } + + @Override + public void preRemoveServers(ObserverContext ctx, + Set
servers) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preRemoveServers(ctx, servers); + } + + @Override + public void preRenameRSGroup(ObserverContext ctx, String oldName, + String newName) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preRenameRSGroup(ctx, oldName, newName); + } + + @Override + public void preUpdateRSGroupConfig(ObserverContext ctx, + String groupName, Map configuration) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preUpdateRSGroupConfig(ctx, groupName, configuration); + } + + @Override + public void preAddReplicationPeer(ObserverContext ctx, + String peerId, ReplicationPeerConfig peerConfig) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preAddReplicationPeer(ctx, peerId, peerConfig); + } + + @Override + public void preRemoveReplicationPeer(ObserverContext ctx, + String peerId) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preRemoveReplicationPeer(ctx, peerId); + } + + @Override + public void preEnableReplicationPeer(ObserverContext ctx, + String peerId) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preEnableReplicationPeer(ctx, peerId); + } + + @Override + public void preDisableReplicationPeer(ObserverContext ctx, + String peerId) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preDisableReplicationPeer(ctx, peerId); + } + + @Override + public void preUpdateReplicationPeerConfig(ObserverContext ctx, + String peerId, ReplicationPeerConfig peerConfig) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preUpdateReplicationPeerConfig(ctx, peerId, peerConfig); + } + + @Override + public void preTransitReplicationPeerSyncReplicationState( + ObserverContext ctx, String peerId, SyncReplicationState state) + throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preTransitReplicationPeerSyncReplicationState(ctx, peerId, state); + } + + @Override + public void preGrant(ObserverContext ctx, + UserPermission userPermission, boolean mergeExistingPermissions) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preGrant(ctx, userPermission, mergeExistingPermissions); + } + + @Override + public void preRevoke(ObserverContext ctx, + UserPermission userPermission) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preRevoke(ctx, userPermission); + } + + /* ---- RegionServerObserver Overrides ---- */ + @Override + public void preRollWALWriterRequest(ObserverContext ctx) + throws IOException { + internalReadOnlyGuard(); + RegionServerObserver.super.preRollWALWriterRequest(ctx); } @Override @@ -395,10 +819,10 @@ public void preReplicationSinkBatchMutate(ObserverContext ctx) + public void preReplicateLogEntries(ObserverContext ctx) throws IOException { internalReadOnlyGuard(); - RegionServerObserver.super.preClearRegionBlockCache(ctx); + RegionServerObserver.super.preReplicateLogEntries(ctx); } /* ---- EndpointObserver Overrides ---- */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerBulkLoadObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerBulkLoadObserver.java new file mode 100644 index 000000000000..cd6990a84b33 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerBulkLoadObserver.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.access; + +import static org.mockito.Mockito.mock; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +// Tests methods of BulkLoad Observer which are implemented in ReadOnlyController, +// by mocking the coprocessor environment and dependencies +@Category({ SecurityTests.class, SmallTests.class }) +public class TestReadOnlyControllerBulkLoadObserver { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReadOnlyControllerBulkLoadObserver.class); + + ReadOnlyController readOnlyController; + + // Region Server Coprocessor mocking variables + ObserverContext ctx; + + @Before + public void setup() throws Exception { + readOnlyController = new ReadOnlyController(); + + // mocking variables initialization + ctx = mock(ObserverContext.class); + } + + @After + public void tearDown() throws Exception { + + } + + @Test(expected = IOException.class) + public void testPrePrepareBulkLoadReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.prePrepareBulkLoad(ctx); + } + + @Test + public void testPrePrepareBulkLoadNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.prePrepareBulkLoad(ctx); + } + + @Test(expected = IOException.class) + public void testPreCleanupBulkLoadReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCleanupBulkLoad(ctx); + } + + @Test + public void testPreCleanupBulkLoadNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCleanupBulkLoad(ctx); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerEndpointObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerEndpointObserver.java new file mode 100644 index 000000000000..716eabf052b6 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerEndpointObserver.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.access; + +import static org.mockito.Mockito.mock; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hbase.thirdparty.com.google.protobuf.Message; +import org.apache.hbase.thirdparty.com.google.protobuf.Service; + +// Tests methods of Endpoint Observer which are implemented in ReadOnlyController, +// by mocking the coprocessor environment and dependencies +@Category({ SecurityTests.class, SmallTests.class }) +public class TestReadOnlyControllerEndpointObserver { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReadOnlyControllerEndpointObserver.class); + + ReadOnlyController readOnlyController; + + // Region Server Coprocessor mocking variables + ObserverContext ctx; + Service service; + String methodName; + Message request; + + @Before + public void setup() throws Exception { + readOnlyController = new ReadOnlyController(); + + // mocking variables initialization + ctx = mock(ObserverContext.class); + service = mock(Service.class); + methodName = "testMethod"; + request = mock(Message.class); + + // Linking the mocks: + + } + + @After + public void tearDown() throws Exception { + + } + + @Test(expected = IOException.class) + public void testPreEndpointInvocationReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preEndpointInvocation(ctx, service, methodName, request); + } + + @Test + public void testPreEndpointInvocationNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preEndpointInvocation(ctx, service, methodName, request); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerMasterObserver.java new file mode 100644 index 000000000000..70104653027e --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerMasterObserver.java @@ -0,0 +1,748 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.access; + +import static org.mockito.Mockito.mock; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.BalanceRequest; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.SnapshotDescription; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.net.Address; +import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +// Tests methods of Master Observer which are implemented in ReadOnlyController, +// by mocking the coprocessor environment and dependencies + +@Category({ SecurityTests.class, SmallTests.class }) +public class TestReadOnlyControllerMasterObserver { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReadOnlyControllerMasterObserver.class); + + ReadOnlyController readOnlyController; + + // Master Coprocessor mocking variables + ObserverContext c, ctx; + TableDescriptor desc; + RegionInfo[] regions; + TableName tableName; + TableDescriptor currentDescriptor, newDescriptor; + String dstSFT; + byte[] family; + byte[] splitRow; + byte[] splitKey; + List metaEntries; + RegionInfo[] regionsToMerge; + SnapshotDescription snapshot; + TableDescriptor tableDescriptor; + NamespaceDescriptor ns; + NamespaceDescriptor currentNsDescriptor, newNsDescriptor; + String namespace; + String userName; + GlobalQuotaSettings quotas; + String regionServer; + Set
servers; + Set tables; + String targetGroup; + String name; + String groupName; + BalanceRequest request; + String oldName, newName; + Map configuration; + String peerId; + ReplicationPeerConfig peerConfig; + SyncReplicationState state; + UserPermission userPermission; + boolean mergeExistingPermissions; + + @Before + public void setup() throws Exception { + readOnlyController = new ReadOnlyController(); + + // mocking variables initialization + c = mock(ObserverContext.class); + // ctx is created to make naming variable in sync with the Observer interface + // methods where 'ctx' is used as the ObserverContext variable name instead of 'c'. + // otherwise both are one and the same + ctx = c; + desc = mock(TableDescriptor.class); + regions = new RegionInfo[] {}; + tableName = TableName.valueOf("testTable"); + currentDescriptor = mock(TableDescriptor.class); + newDescriptor = mock(TableDescriptor.class); + dstSFT = "dstSFT"; + family = Bytes.toBytes("testFamily"); + splitRow = Bytes.toBytes("splitRow"); + splitKey = Bytes.toBytes("splitKey"); + metaEntries = List.of(); + regionsToMerge = new RegionInfo[] {}; + snapshot = mock(SnapshotDescription.class); + tableDescriptor = mock(TableDescriptor.class); + ns = mock(NamespaceDescriptor.class); + currentNsDescriptor = mock(NamespaceDescriptor.class); + newNsDescriptor = mock(NamespaceDescriptor.class); + namespace = "testNamespace"; + userName = "testUser"; + quotas = mock(GlobalQuotaSettings.class); + regionServer = "testRegionServer"; + servers = Set.of(); + tables = Set.of(); + targetGroup = "targetGroup"; + name = "testRSGroup"; + groupName = "testGroupName"; + request = BalanceRequest.newBuilder().build(); + oldName = "oldRSGroupName"; + newName = "newRSGroupName"; + configuration = Map.of(); + peerId = "testPeerId"; + peerConfig = mock(ReplicationPeerConfig.class); + state = SyncReplicationState.NONE; + userPermission = mock(UserPermission.class); + mergeExistingPermissions = false; + + // Linking the mocks: + + } + + @After + public void tearDown() throws Exception { + + } + + @Test(expected = IOException.class) + public void testPreCreateTableRegionsInfosReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCreateTableRegionsInfos(ctx, desc); + } + + @Test + public void testPreCreateTableRegionsInfosNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCreateTableRegionsInfos(ctx, desc); + } + + @Test(expected = IOException.class) + public void testPreCreateTableReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCreateTable(ctx, desc, regions); + } + + @Test + public void testPreCreateTableNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCreateTable(ctx, desc, regions); + } + + @Test(expected = IOException.class) + public void testPreCreateTableActionReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCreateTableAction(ctx, desc, regions); + } + + @Test + public void testPreCreateTableActionNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCreateTableAction(ctx, desc, regions); + } + + @Test(expected = IOException.class) + public void testPreDeleteTableReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preDeleteTable(ctx, tableName); + } + + @Test + public void testPreDeleteTableNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preDeleteTable(ctx, tableName); + } + + @Test(expected = IOException.class) + public void testPreDeleteTableActionReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preDeleteTableAction(ctx, tableName); + } + + @Test + public void testPreDeleteTableActionNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preDeleteTableAction(ctx, tableName); + } + + @Test(expected = IOException.class) + public void testPreTruncateTableReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preTruncateTable(ctx, tableName); + } + + @Test + public void testPreTruncateTableNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preTruncateTable(ctx, tableName); + } + + @Test(expected = IOException.class) + public void testPreTruncateTableActionReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preTruncateTableAction(ctx, tableName); + } + + @Test + public void testPreTruncateTableActionNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preTruncateTableAction(ctx, tableName); + } + + @Test(expected = IOException.class) + public void testPreModifyTableReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preModifyTable(ctx, tableName, currentDescriptor, newDescriptor); + } + + @Test + public void testPreModifyTableNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preModifyTable(ctx, tableName, currentDescriptor, newDescriptor); + } + + @Test(expected = IOException.class) + public void testPreModifyTableStoreFileTrackerReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preModifyTableStoreFileTracker(ctx, tableName, dstSFT); + } + + @Test + public void testPreModifyTableStoreFileTrackerNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preModifyTableStoreFileTracker(ctx, tableName, dstSFT); + } + + @Test(expected = IOException.class) + public void testPreModifyColumnFamilyStoreFileTrackerReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preModifyColumnFamilyStoreFileTracker(ctx, tableName, family, dstSFT); + } + + @Test + public void testPreModifyColumnFamilyStoreFileTrackerNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preModifyColumnFamilyStoreFileTracker(ctx, tableName, family, dstSFT); + } + + @Test(expected = IOException.class) + public void testPreModifyTableActionReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preModifyTableAction(ctx, tableName, currentDescriptor, newDescriptor); + } + + @Test + public void testPreModifyTableActionNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preModifyTableAction(ctx, tableName, currentDescriptor, newDescriptor); + } + + @Test(expected = IOException.class) + public void testPreSplitRegionReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preSplitRegion(c, tableName, splitRow); + } + + @Test + public void testPreSplitRegionNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preSplitRegion(c, tableName, splitRow); + } + + @Test(expected = IOException.class) + public void testPreSplitRegionActionReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preSplitRegionAction(c, tableName, splitRow); + } + + @Test + public void testPreSplitRegionActionNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preSplitRegionAction(c, tableName, splitRow); + } + + @Test(expected = IOException.class) + public void testPreSplitRegionBeforeMETAActionReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preSplitRegionBeforeMETAAction(ctx, splitKey, metaEntries); + } + + @Test + public void testPreSplitRegionBeforeMETAActionNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preSplitRegionBeforeMETAAction(ctx, splitKey, metaEntries); + } + + @Test(expected = IOException.class) + public void testPreSplitRegionAfterMETAActionReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preSplitRegionAfterMETAAction(ctx); + } + + @Test + public void testPreSplitRegionAfterMETAActionNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preSplitRegionAfterMETAAction(ctx); + } + + @Test(expected = IOException.class) + public void testPreMergeRegionsActionReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preMergeRegionsAction(ctx, regionsToMerge); + } + + @Test + public void testPreMergeRegionsActionNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preMergeRegionsAction(ctx, regionsToMerge); + } + + @Test(expected = IOException.class) + public void testPreMergeRegionsCommitActionReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preMergeRegionsCommitAction(ctx, regionsToMerge, metaEntries); + } + + @Test + public void testPreMergeRegionsCommitActionNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preMergeRegionsCommitAction(ctx, regionsToMerge, metaEntries); + } + + @Test(expected = IOException.class) + public void testPreSnapshotReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preSnapshot(ctx, snapshot, tableDescriptor); + } + + @Test + public void testPreSnapshotNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preSnapshot(ctx, snapshot, tableDescriptor); + } + + @Test(expected = IOException.class) + public void testPreCloneSnapshotReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCloneSnapshot(ctx, snapshot, tableDescriptor); + } + + @Test + public void testPreCloneSnapshotNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCloneSnapshot(ctx, snapshot, tableDescriptor); + } + + @Test(expected = IOException.class) + public void testPreRestoreSnapshotReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preRestoreSnapshot(ctx, snapshot, tableDescriptor); + } + + @Test + public void testPreRestoreSnapshotNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preRestoreSnapshot(ctx, snapshot, tableDescriptor); + } + + @Test(expected = IOException.class) + public void testPreDeleteSnapshotReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preDeleteSnapshot(ctx, snapshot); + } + + @Test + public void testPreDeleteSnapshotNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preDeleteSnapshot(ctx, snapshot); + } + + @Test(expected = IOException.class) + public void testPreCreateNamespaceReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCreateNamespace(ctx, ns); + } + + @Test + public void testPreCreateNamespaceNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCreateNamespace(ctx, ns); + } + + @Test(expected = IOException.class) + public void testPreModifyNamespaceReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preModifyNamespace(ctx, currentNsDescriptor, newNsDescriptor); + } + + @Test + public void testPreModifyNamespaceNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preModifyNamespace(ctx, currentNsDescriptor, newNsDescriptor); + } + + @Test(expected = IOException.class) + public void testPreDeleteNamespaceReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preDeleteNamespace(ctx, namespace); + } + + @Test + public void testPreDeleteNamespaceNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preDeleteNamespace(ctx, namespace); + } + + @Test(expected = IOException.class) + public void testPreMasterStoreFlushReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preMasterStoreFlush(ctx); + } + + @Test + public void testPreMasterStoreFlushNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preMasterStoreFlush(ctx); + } + + @Test(expected = IOException.class) + public void testPreSetUserQuotaReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preSetUserQuota(ctx, userName, quotas); + } + + @Test + public void testPreSetUserQuotaNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preSetUserQuota(ctx, userName, quotas); + } + + @Test(expected = IOException.class) + public void testPreSetUserQuotaOnTableReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preSetUserQuota(ctx, userName, tableName, quotas); + } + + @Test + public void testPreSetUserQuotaOnTableNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preSetUserQuota(ctx, userName, tableName, quotas); + } + + @Test(expected = IOException.class) + public void testPreSetUserQuotaOnNamespaceReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preSetUserQuota(ctx, userName, namespace, quotas); + } + + @Test + public void testPreSetUserQuotaOnNamespaceNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preSetUserQuota(ctx, userName, namespace, quotas); + } + + @Test(expected = IOException.class) + public void testPreSetTableQuotaReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preSetTableQuota(ctx, tableName, quotas); + } + + @Test + public void testPreSetTableQuotaNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preSetTableQuota(ctx, tableName, quotas); + } + + @Test(expected = IOException.class) + public void testPreSetNamespaceQuotaReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preSetNamespaceQuota(ctx, namespace, quotas); + } + + @Test + public void testPreSetNamespaceQuotaNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preSetNamespaceQuota(ctx, namespace, quotas); + } + + @Test(expected = IOException.class) + public void testPreSetRegionServerQuotaReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preSetRegionServerQuota(ctx, regionServer, quotas); + } + + @Test + public void testPreSetRegionServerQuotaNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preSetRegionServerQuota(ctx, regionServer, quotas); + } + + @Test(expected = IOException.class) + public void testPreMergeRegionsReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preMergeRegions(ctx, regionsToMerge); + } + + @Test + public void testPreMergeRegionsNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preMergeRegions(ctx, regionsToMerge); + } + + @Test(expected = IOException.class) + public void testPreMoveServersAndTablesReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preMoveServersAndTables(ctx, servers, tables, targetGroup); + } + + @Test + public void testPreMoveServersAndTablesNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preMoveServersAndTables(ctx, servers, tables, targetGroup); + } + + @Test(expected = IOException.class) + public void testPreMoveServersReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preMoveServers(ctx, servers, targetGroup); + } + + @Test + public void testPreMoveServersNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preMoveServers(ctx, servers, targetGroup); + } + + @Test(expected = IOException.class) + public void testPreMoveTablesReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preMoveTables(ctx, tables, targetGroup); + } + + @Test + public void testPreMoveTablesNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preMoveTables(ctx, tables, targetGroup); + } + + @Test(expected = IOException.class) + public void testPreAddRSGroupReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preAddRSGroup(ctx, name); + } + + @Test + public void testPreAddRSGroupNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preAddRSGroup(ctx, name); + } + + @Test(expected = IOException.class) + public void testPreRemoveRSGroupReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preRemoveRSGroup(ctx, name); + } + + @Test + public void testPreRemoveRSGroupNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preRemoveRSGroup(ctx, name); + } + + @Test(expected = IOException.class) + public void testPreBalanceRSGroupReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preBalanceRSGroup(ctx, groupName, request); + } + + @Test + public void testPreBalanceRSGroupNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preBalanceRSGroup(ctx, groupName, request); + } + + @Test(expected = IOException.class) + public void testPreRemoveServersReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preRemoveServers(ctx, servers); + } + + @Test + public void testPreRemoveServersNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preRemoveServers(ctx, servers); + } + + @Test(expected = IOException.class) + public void testPreRenameRSGroupReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preRenameRSGroup(ctx, oldName, newName); + } + + @Test + public void testPreRenameRSGroupNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preRenameRSGroup(ctx, oldName, newName); + } + + @Test(expected = IOException.class) + public void testPreUpdateRSGroupConfigReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preUpdateRSGroupConfig(ctx, groupName, configuration); + } + + @Test + public void testPreUpdateRSGroupConfigNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preUpdateRSGroupConfig(ctx, groupName, configuration); + } + + @Test(expected = IOException.class) + public void testPreAddReplicationPeerReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preAddReplicationPeer(ctx, peerId, peerConfig); + } + + @Test + public void testPreAddReplicationPeerNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preAddReplicationPeer(ctx, peerId, peerConfig); + } + + @Test(expected = IOException.class) + public void testPreRemoveReplicationPeerReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preRemoveReplicationPeer(ctx, peerId); + } + + @Test + public void testPreRemoveReplicationPeerNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preRemoveReplicationPeer(ctx, peerId); + } + + @Test(expected = IOException.class) + public void testPreEnableReplicationPeerReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preEnableReplicationPeer(ctx, peerId); + } + + @Test + public void testPreEnableReplicationPeerNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preEnableReplicationPeer(ctx, peerId); + } + + @Test(expected = IOException.class) + public void testPreDisableReplicationPeerReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preDisableReplicationPeer(ctx, peerId); + } + + @Test + public void testPreDisableReplicationPeerNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preDisableReplicationPeer(ctx, peerId); + } + + @Test(expected = IOException.class) + public void testPreUpdateReplicationPeerConfigReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preUpdateReplicationPeerConfig(ctx, peerId, peerConfig); + } + + @Test + public void testPreUpdateReplicationPeerConfigNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preUpdateReplicationPeerConfig(ctx, peerId, peerConfig); + } + + @Test(expected = IOException.class) + public void testPreTransitReplicationPeerSyncReplicationStateReadOnlyException() + throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preTransitReplicationPeerSyncReplicationState(ctx, peerId, state); + } + + @Test + public void testPreTransitReplicationPeerSyncReplicationStateNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preTransitReplicationPeerSyncReplicationState(ctx, peerId, state); + } + + @Test(expected = IOException.class) + public void testPreGrantReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preGrant(ctx, userPermission, mergeExistingPermissions); + } + + @Test + public void testPreGrantNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preGrant(ctx, userPermission, mergeExistingPermissions); + } + + @Test(expected = IOException.class) + public void testPreRevokeReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preRevoke(ctx, userPermission); + } + + @Test + public void testPreRevokeNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preRevoke(ctx, userPermission); + } + + // @Test(expected = IOException.class) + // public void testReadOnlyException() throws IOException { + // readOnlyController.setGlobalReadOnlyEnabled(true); + // readOnlyController.(); + // } + // + // @Test + // public void testNoException() throws IOException { + // readOnlyController.setGlobalReadOnlyEnabled(false); + // readOnlyController.(); + // } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerRegionObserver.java new file mode 100644 index 000000000000..7ccf48bf65cc --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerRegionObserver.java @@ -0,0 +1,588 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.access; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.List; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.CheckAndMutate; +import org.apache.hadoop.hbase.client.CheckAndMutateResult; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +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.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.filter.ByteArrayComparable; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; +import org.apache.hadoop.hbase.regionserver.ScanOptions; +import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +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.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +// Tests methods of Region Observer interface which are implemented in ReadOnlyController, +// by mocking the coprocessor environment and dependencies. +// V1 and V2 means version 1 and version 2 of the coprocessor method signature. +// For example, prePut has 2 versions: +// V1: prePut(ObserverContext c, Put put, WALEdit edit) +// V2: prePut(ObserverContext c, Put put, WALEdit edit, Durability durability) + +@Category({ SecurityTests.class, SmallTests.class }) +public class TestReadOnlyControllerRegionObserver { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReadOnlyControllerRegionObserver.class); + + ReadOnlyController readOnlyController; + + // Region Coprocessor mocking variables + ObserverContext c, ctx; + RegionCoprocessorEnvironment env; + RegionInfo regionInfo; + Store store; + InternalScanner scanner; + ScanOptions options; + FlushLifeCycleTracker flushLifeCycleTracker; + List candidates; + CompactionLifeCycleTracker compactionLifeCycleTracker; + ScanType scanType; + CompactionRequest compactionRequest; + TableName tableName; + Put put; + WALEdit edit; + Durability durability; + Delete delete; + MiniBatchOperationInProgress miniBatchOp; + byte[] row; + byte[] family; + byte[] qualifier; + Filter filter; + CompareOperator op; + ByteArrayComparable comparator; + boolean result; + CheckAndMutate checkAndMutate; + CheckAndMutateResult checkAndMutateResult; + Append append; + Increment increment; + RegionInfo info; + Path edits; + List> familyPaths; + List> pairs; + WALKey key; + + @Before + public void setup() throws Exception { + readOnlyController = new ReadOnlyController(); + + // mocking variables initialization + c = mock(ObserverContext.class); + // ctx is created to make naming variable in sync with the Observer interface + // methods where 'ctx' is used as the ObserverContext variable name instead of 'c'. + // otherwise both are one and the same + ctx = c; + env = mock(RegionCoprocessorEnvironment.class); + regionInfo = mock(RegionInfo.class); + store = mock(Store.class); + scanner = mock(InternalScanner.class); + options = mock(ScanOptions.class); + flushLifeCycleTracker = mock(FlushLifeCycleTracker.class); + compactionLifeCycleTracker = mock(CompactionLifeCycleTracker.class); + StoreFile sf1 = mock(StoreFile.class); + StoreFile sf2 = mock(StoreFile.class); + candidates = List.of(sf1, sf2); + scanType = ScanType.COMPACT_DROP_DELETES; + compactionRequest = mock(CompactionRequest.class); + tableName = TableName.valueOf("testTable"); + put = mock(Put.class); + edit = mock(WALEdit.class); + durability = Durability.USE_DEFAULT; + delete = mock(Delete.class); + miniBatchOp = mock(MiniBatchOperationInProgress.class); + row = Bytes.toBytes("test-row"); + family = Bytes.toBytes("test-family"); + qualifier = Bytes.toBytes("test-qualifier"); + filter = mock(Filter.class); + op = CompareOperator.NO_OP; + comparator = mock(ByteArrayComparable.class); + result = false; + checkAndMutate = CheckAndMutate + .newBuilder(Bytes.toBytes("test-row")).ifEquals(Bytes.toBytes("test-family"), + Bytes.toBytes("test-qualifier"), Bytes.toBytes("test-value")) + .build(new Put(Bytes.toBytes("test-row"))); + checkAndMutateResult = mock(CheckAndMutateResult.class); + append = mock(Append.class); + increment = mock(Increment.class); + edits = mock(Path.class); + familyPaths = List.of(new Pair<>(Bytes.toBytes("test-family"), "/path/to/hfile1"), + new Pair<>(Bytes.toBytes("test-family"), "/path/to/hfile2")); + pairs = List.of(new Pair<>(mock(Path.class), mock(Path.class)), + new Pair<>(mock(Path.class), mock(Path.class))); + key = mock(WALKey.class); + + // Linking the mocks: + when(c.getEnvironment()).thenReturn(env); + when(env.getRegionInfo()).thenReturn(regionInfo); + when(regionInfo.getTable()).thenReturn(tableName); + } + + @After + public void tearDown() throws Exception { + + } + + @Test(expected = IOException.class) + public void testPreFlushV1ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preFlush(c, flushLifeCycleTracker); + } + + @Test + public void testPreFlushV1NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preFlush(c, flushLifeCycleTracker); + } + + @Test(expected = IOException.class) + public void testPreFlushV2ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preFlush(c, store, scanner, flushLifeCycleTracker); + } + + @Test + public void testPreFlushV2NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preFlush(c, store, scanner, flushLifeCycleTracker); + } + + @Test(expected = IOException.class) + public void testPreFlushScannerOpenReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preFlushScannerOpen(c, store, options, flushLifeCycleTracker); + } + + @Test + public void testPreFlushScannerOpenNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preFlushScannerOpen(c, store, options, flushLifeCycleTracker); + } + + @Test(expected = IOException.class) + public void testPreMemStoreCompactionReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preMemStoreCompaction(c, store); + } + + @Test + public void testPreMemStoreCompactionNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preMemStoreCompaction(c, store); + } + + @Test(expected = IOException.class) + public void testPreMemStoreCompactionCompactScannerOpenReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preMemStoreCompactionCompactScannerOpen(c, store, options); + } + + @Test + public void testPreMemStoreCompactionCompactScannerOpenNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preMemStoreCompactionCompactScannerOpen(c, store, options); + } + + @Test(expected = IOException.class) + public void testPreMemStoreCompactionCompactReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preMemStoreCompactionCompact(c, store, scanner); + } + + @Test + public void testPreMemStoreCompactionCompactNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preMemStoreCompactionCompact(c, store, scanner); + } + + @Test(expected = IOException.class) + public void testPreCompactSelectionReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCompactSelection(c, store, candidates, compactionLifeCycleTracker); + } + + @Test + public void testPreCompactSelectionNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCompactSelection(c, store, candidates, compactionLifeCycleTracker); + } + + @Test(expected = IOException.class) + public void testPreCompactScannerOpenReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCompactScannerOpen(c, store, scanType, options, + compactionLifeCycleTracker, compactionRequest); + } + + @Test + public void testPreCompactScannerOpenNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCompactScannerOpen(c, store, scanType, options, + compactionLifeCycleTracker, compactionRequest); + } + + @Test(expected = IOException.class) + public void testPreCompactReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCompact(c, store, scanner, scanType, compactionLifeCycleTracker, + compactionRequest); + } + + @Test + public void testPreCompactNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCompact(c, store, scanner, scanType, compactionLifeCycleTracker, + compactionRequest); + } + + @Test(expected = IOException.class) + public void testPrePutV1ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.prePut(c, put, edit); + } + + @Test + public void testPrePutV1NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.prePut(c, put, edit); + } + + @Test(expected = IOException.class) + public void testPrePutV2ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.prePut(c, put, edit, durability); + } + + @Test + public void testPrePutV2NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.prePut(c, put, edit, durability); + } + + @Test(expected = IOException.class) + public void testPreDeleteV1ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preDelete(c, delete, edit); + } + + @Test + public void testPreDeleteV1NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preDelete(c, delete, edit); + } + + @Test(expected = IOException.class) + public void testPreDeleteV2ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preDelete(c, delete, edit, durability); + } + + @Test + public void testPreDeleteV2NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preDelete(c, delete, edit, durability); + } + + @Test + public void testPreBatchMutateNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preBatchMutate(c, miniBatchOp); + } + + @Test(expected = IOException.class) + public void testPreCheckAndPutV1ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCheckAndPut(c, row, family, qualifier, op, comparator, put, result); + } + + @Test + public void testPreCheckAndPutV1NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCheckAndPut(c, row, family, qualifier, op, comparator, put, result); + } + + @Test(expected = IOException.class) + public void testPreCheckAndPutV2ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCheckAndPut(c, row, filter, put, result); + } + + @Test + public void testPreCheckAndPutV2NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCheckAndPut(c, row, filter, put, result); + } + + @Test(expected = IOException.class) + public void testPreCheckAndPutAfterRowLockV1ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCheckAndPutAfterRowLock(c, row, family, qualifier, op, comparator, put, + result); + } + + @Test + public void testPreCheckAndPutAfterRowLockV1NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCheckAndPutAfterRowLock(c, row, family, qualifier, op, comparator, put, + result); + } + + @Test(expected = IOException.class) + public void testPreCheckAndPutAfterRowLockV2ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCheckAndPutAfterRowLock(c, row, filter, put, result); + } + + @Test + public void testPreCheckAndPutAfterRowLockV2NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCheckAndPutAfterRowLock(c, row, filter, put, result); + } + + @Test(expected = IOException.class) + public void testPreCheckAndDeleteV1ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCheckAndDelete(c, row, family, qualifier, op, comparator, delete, result); + } + + @Test + public void testPreCheckAndDeleteV1NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCheckAndDelete(c, row, family, qualifier, op, comparator, delete, result); + } + + @Test(expected = IOException.class) + public void testPreCheckAndDeleteV2ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCheckAndDelete(c, row, filter, delete, result); + } + + @Test + public void testPreCheckAndDeleteV2NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCheckAndDelete(c, row, filter, delete, result); + } + + @Test(expected = IOException.class) + public void testPreCheckAndDeleteAfterRowLockV1ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCheckAndDeleteAfterRowLock(c, row, family, qualifier, op, comparator, + delete, result); + } + + @Test + public void testPreCheckAndDeleteAfterRowLockV1NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCheckAndDeleteAfterRowLock(c, row, family, qualifier, op, comparator, + delete, result); + } + + @Test(expected = IOException.class) + public void testPreCheckAndDeleteAfterRowLockV2ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCheckAndDeleteAfterRowLock(c, row, filter, delete, result); + } + + @Test + public void testPreCheckAndDeleteAfterRowLockV2NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCheckAndDeleteAfterRowLock(c, row, filter, delete, result); + } + + @Test(expected = IOException.class) + public void testPreBatchMutateReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preBatchMutate(c, miniBatchOp); + } + + @Test(expected = IOException.class) + public void testPreCheckAndMutateReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCheckAndMutate(c, checkAndMutate, checkAndMutateResult); + } + + @Test + public void testPreCheckAndMutateNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCheckAndMutate(c, checkAndMutate, checkAndMutateResult); + } + + @Test(expected = IOException.class) + public void testPreCheckAndMutateAfterRowLockReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCheckAndMutateAfterRowLock(c, checkAndMutate, checkAndMutateResult); + } + + @Test + public void testPreCheckAndMutateAfterRowLockNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCheckAndMutateAfterRowLock(c, checkAndMutate, checkAndMutateResult); + } + + @Test(expected = IOException.class) + public void testPreAppendV1ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preAppend(c, append); + } + + @Test + public void testPreAppendV1NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preAppend(c, append); + } + + @Test(expected = IOException.class) + public void testPreAppendV2ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preAppend(c, append, edit); + } + + @Test + public void testPreAppendV2NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preAppend(c, append, edit); + } + + @Test(expected = IOException.class) + public void testPreAppendAfterRowLockReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preAppendAfterRowLock(c, append); + } + + @Test + public void testPreAppendAfterRowLockNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preAppendAfterRowLock(c, append); + } + + @Test(expected = IOException.class) + public void testPreIncrementV1ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preIncrement(c, increment); + } + + @Test + public void testPreIncrementV1NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preIncrement(c, increment); + } + + @Test(expected = IOException.class) + public void testPreIncrementV2ReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preIncrement(c, increment, edit); + } + + @Test + public void testPreIncrementV2NoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preIncrement(c, increment, edit); + } + + @Test(expected = IOException.class) + public void testPreIncrementAfterRowLockReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preIncrementAfterRowLock(c, increment); + } + + @Test + public void testPreIncrementAfterRowLockNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preIncrementAfterRowLock(c, increment); + } + + @Test(expected = IOException.class) + public void testPreReplayWALsReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preReplayWALs(ctx, info, edits); + } + + @Test + public void testPreReplayWALsNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preReplayWALs(ctx, info, edits); + } + + @Test(expected = IOException.class) + public void testPreBulkLoadHFileReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preBulkLoadHFile(ctx, familyPaths); + } + + @Test + public void testPreBulkLoadHFileNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preBulkLoadHFile(ctx, familyPaths); + } + + @Test(expected = IOException.class) + public void testPreCommitStoreFileReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preCommitStoreFile(ctx, family, pairs); + } + + @Test + public void testPreCommitStoreFileNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preCommitStoreFile(ctx, family, pairs); + } + + @Test(expected = IOException.class) + public void testPreWALAppendReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preWALAppend(ctx, key, edit); + } + + @Test + public void testPreWALAppendNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preWALAppend(ctx, key, edit); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerRegionServerObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerRegionServerObserver.java new file mode 100644 index 000000000000..d0adb6437458 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyControllerRegionServerObserver.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.access; + +import static org.mockito.Mockito.mock; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; + +// Tests methods of Region Server Observer which are implemented in ReadOnlyController, +// by mocking the coprocessor environment and dependencies +@Category({ SecurityTests.class, SmallTests.class }) +public class TestReadOnlyControllerRegionServerObserver { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReadOnlyControllerRegionServerObserver.class); + + ReadOnlyController readOnlyController; + + // Region Server Coprocessor mocking variables + ObserverContext ctx; + AdminProtos.WALEntry walEntry; + Mutation mutation; + + @Before + public void setup() throws Exception { + readOnlyController = new ReadOnlyController(); + + // mocking variables initialization + ctx = mock(ObserverContext.class); + walEntry = AdminProtos.WALEntry.newBuilder() + .setKey(WALProtos.WALKey.newBuilder().setTableName(ByteString.copyFromUtf8("test")) + .setEncodedRegionName(ByteString.copyFromUtf8("regionA")).setLogSequenceNumber(100) + .setWriteTime(2).build()) + .build(); + mutation = mock(Mutation.class); + } + + @After + public void tearDown() throws Exception { + + } + + @Test(expected = IOException.class) + public void testPreRollWALWriterRequestReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preRollWALWriterRequest(ctx); + } + + @Test + public void testPreRollWALWriterRequestNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preRollWALWriterRequest(ctx); + } + + @Test(expected = IOException.class) + public void testPreExecuteProceduresReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preExecuteProcedures(ctx); + } + + @Test + public void testPreExecuteProceduresNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preExecuteProcedures(ctx); + } + + @Test(expected = IOException.class) + public void testPreReplicationSinkBatchMutateReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preReplicationSinkBatchMutate(ctx, walEntry, mutation); + } + + @Test + public void testPreReplicationSinkBatchMutateNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preReplicationSinkBatchMutate(ctx, walEntry, mutation); + } + + @Test(expected = IOException.class) + public void testPreReplicateLogEntriesReadOnlyException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(true); + readOnlyController.preReplicateLogEntries(ctx); + } + + @Test + public void testPreReplicateLogEntriesNoException() throws IOException { + readOnlyController.setGlobalReadOnlyEnabled(false); + readOnlyController.preReplicateLogEntries(ctx); + } +}