001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.security.access;
019
020import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry;
021import static org.junit.Assert.assertArrayEquals;
022import static org.junit.Assert.assertFalse;
023import static org.junit.Assert.assertTrue;
024import static org.junit.Assert.fail;
025import static org.mockito.Mockito.mock;
026
027import java.io.IOException;
028import java.security.PrivilegedExceptionAction;
029import java.util.Collections;
030import java.util.HashMap;
031import org.apache.hadoop.conf.Configuration;
032import org.apache.hadoop.hbase.AuthUtil;
033import org.apache.hadoop.hbase.Cell;
034import org.apache.hadoop.hbase.CellUtil;
035import org.apache.hadoop.hbase.HBaseClassTestRule;
036import org.apache.hadoop.hbase.HBaseTestingUtil;
037import org.apache.hadoop.hbase.ServerName;
038import org.apache.hadoop.hbase.TableName;
039import org.apache.hadoop.hbase.client.Admin;
040import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
041import org.apache.hadoop.hbase.client.Connection;
042import org.apache.hadoop.hbase.client.ConnectionFactory;
043import org.apache.hadoop.hbase.client.Get;
044import org.apache.hadoop.hbase.client.Put;
045import org.apache.hadoop.hbase.client.Result;
046import org.apache.hadoop.hbase.client.Table;
047import org.apache.hadoop.hbase.client.TableDescriptor;
048import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
049import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
050import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
051import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessor;
052import org.apache.hadoop.hbase.security.AccessDeniedException;
053import org.apache.hadoop.hbase.security.User;
054import org.apache.hadoop.hbase.testclassification.MediumTests;
055import org.apache.hadoop.hbase.testclassification.SecurityTests;
056import org.apache.hadoop.hbase.util.Bytes;
057import org.junit.BeforeClass;
058import org.junit.ClassRule;
059import org.junit.Rule;
060import org.junit.Test;
061import org.junit.experimental.categories.Category;
062import org.junit.rules.TestName;
063
064import org.apache.hbase.thirdparty.com.google.protobuf.Service;
065import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
066
067import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos;
068import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos;
069
070/**
071 * This class tests operations in MasterRpcServices which require ADMIN access. It doesn't test all
072 * operations which require ADMIN access, only those which get vetted within MasterRpcServices at
073 * the point of entry itself (unlike old approach of using hooks in AccessController). Sidenote:
074 * There is one big difference between how security tests for AccessController hooks work, and how
075 * the tests in this class for security in MasterRpcServices work. The difference arises because of
076 * the way AC & MasterRpcServices get the user. In AccessController, it first checks if there is an
077 * active rpc user in ObserverContext. If not, it uses UserProvider for current user. This *might*
078 * make sense in the context of coprocessors, because they can be called outside the context of
079 * RPCs. But in the context of MasterRpcServices, only one way makes sense -
080 * RPCServer.getRequestUser(). In AC tests, when we do FooUser.runAs on AccessController instance
081 * directly, it bypasses the rpc framework completely, but works because UserProvider provides the
082 * correct user, i.e. FooUser in this case. But this doesn't work for the tests here, so we go
083 * around by doing complete RPCs.
084 */
085@Category({ SecurityTests.class, MediumTests.class })
086public class TestRpcAccessChecks {
087  @ClassRule
088  public static final HBaseClassTestRule CLASS_RULE =
089    HBaseClassTestRule.forClass(TestRpcAccessChecks.class);
090
091  @Rule
092  public final TestName TEST_NAME = new TestName();
093
094  private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
095  private static Configuration conf;
096
097  // user granted with all global permission
098  private static User USER_ADMIN;
099  // user without admin permissions
100  private static User USER_NON_ADMIN;
101  // user in supergroup
102  private static User USER_IN_SUPERGROUPS;
103  // user with global permission but not a superuser
104  private static User USER_ADMIN_NOT_SUPER;
105
106  private static final String GROUP_ADMIN = "admin_group";
107  private static User USER_GROUP_ADMIN;
108
109  // Dummy service to test execService calls. Needs to be public so can be loaded as Coprocessor.
110  public static class DummyCpService implements MasterCoprocessor, RegionServerCoprocessor {
111    public DummyCpService() {
112    }
113
114    @Override
115    public Iterable<Service> getServices() {
116      return Collections.singleton(mock(TestRpcServiceProtos.TestProtobufRpcProto.class));
117    }
118  }
119
120  private static void enableSecurity(Configuration conf) throws IOException {
121    conf.set("hadoop.security.authorization", "false");
122    conf.set("hadoop.security.authentication", "simple");
123    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
124      AccessController.class.getName() + "," + DummyCpService.class.getName());
125    conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName());
126    conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
127      AccessController.class.getName() + "," + DummyCpService.class.getName());
128    conf.set(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, "true");
129    SecureTestUtil.configureSuperuser(conf);
130  }
131
132  @BeforeClass
133  public static void setup() throws Exception {
134    conf = TEST_UTIL.getConfiguration();
135
136    // Enable security
137    enableSecurity(conf);
138
139    // Create users
140    // admin is superuser as well.
141    USER_ADMIN = User.createUserForTesting(conf, "admin", new String[0]);
142    USER_NON_ADMIN = User.createUserForTesting(conf, "non_admin", new String[0]);
143    USER_GROUP_ADMIN =
144      User.createUserForTesting(conf, "user_group_admin", new String[] { GROUP_ADMIN });
145    USER_IN_SUPERGROUPS =
146      User.createUserForTesting(conf, "user_in_supergroup", new String[] { "supergroup" });
147    USER_ADMIN_NOT_SUPER = User.createUserForTesting(conf, "normal_admin", new String[0]);
148
149    TEST_UTIL.startMiniCluster();
150    // Wait for the ACL table to become available
151    TEST_UTIL.waitUntilAllRegionsAssigned(PermissionStorage.ACL_TABLE_NAME);
152
153    // Assign permissions to groups
154    SecureTestUtil.grantGlobal(TEST_UTIL, toGroupEntry(GROUP_ADMIN), Permission.Action.ADMIN,
155      Permission.Action.CREATE);
156    SecureTestUtil.grantGlobal(TEST_UTIL, USER_ADMIN_NOT_SUPER.getShortName(),
157      Permission.Action.ADMIN);
158  }
159
160  interface Action {
161    void run(Admin admin) throws Exception;
162  }
163
164  private void verifyAllowed(User user, Action action) throws Exception {
165    user.runAs((PrivilegedExceptionAction<?>) () -> {
166      try (Connection conn = ConnectionFactory.createConnection(conf);
167        Admin admin = conn.getAdmin()) {
168        action.run(admin);
169      } catch (IOException e) {
170        fail(e.toString());
171      }
172      return null;
173    });
174  }
175
176  private void verifyDenied(User user, Action action) throws Exception {
177    user.runAs((PrivilegedExceptionAction<?>) () -> {
178      boolean accessDenied = false;
179      try (Connection conn = ConnectionFactory.createConnection(conf);
180        Admin admin = conn.getAdmin()) {
181        action.run(admin);
182      } catch (AccessDeniedException e) {
183        accessDenied = true;
184      }
185      assertTrue("Expected access to be denied", accessDenied);
186      return null;
187    });
188  }
189
190  private void verifiedDeniedServiceException(User user, Action action) throws Exception {
191    user.runAs((PrivilegedExceptionAction<?>) () -> {
192      boolean accessDenied = false;
193      try (Connection conn = ConnectionFactory.createConnection(conf);
194        Admin admin = conn.getAdmin()) {
195        action.run(admin);
196      } catch (ServiceException e) {
197        // For MasterRpcServices.execService.
198        if (e.getCause() instanceof AccessDeniedException) {
199          accessDenied = true;
200        }
201      }
202      assertTrue("Expected access to be denied", accessDenied);
203      return null;
204    });
205
206  }
207
208  private void verifyAdminCheckForAction(Action action) throws Exception {
209    verifyAllowed(USER_ADMIN, action);
210    verifyAllowed(USER_GROUP_ADMIN, action);
211    verifyDenied(USER_NON_ADMIN, action);
212  }
213
214  @Test
215  public void testEnableCatalogJanitor() throws Exception {
216    verifyAdminCheckForAction((admin) -> admin.catalogJanitorSwitch(true));
217  }
218
219  @Test
220  public void testRunCatalogJanitor() throws Exception {
221    verifyAdminCheckForAction((admin) -> admin.runCatalogJanitor());
222  }
223
224  @Test
225  public void testCleanerChoreRunning() throws Exception {
226    verifyAdminCheckForAction((admin) -> admin.cleanerChoreSwitch(true));
227  }
228
229  @Test
230  public void testRunCleanerChore() throws Exception {
231    verifyAdminCheckForAction((admin) -> admin.runCleanerChore());
232  }
233
234  @Test
235  public void testExecProcedure() throws Exception {
236    verifyAdminCheckForAction((admin) -> {
237      // Using existing table instead of creating a new one.
238      admin.execProcedure("flush-table-proc", TableName.META_TABLE_NAME.getNameAsString(),
239        new HashMap<>());
240    });
241  }
242
243  @Test
244  public void testExecService() throws Exception {
245    Action action = (admin) -> {
246      TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface service =
247        TestRpcServiceProtos.TestProtobufRpcProto.newBlockingStub(admin.coprocessorService());
248      service.ping(null, TestProtos.EmptyRequestProto.getDefaultInstance());
249    };
250
251    verifyAllowed(USER_ADMIN, action);
252    verifyAllowed(USER_GROUP_ADMIN, action);
253    // This is same as above verifyAccessDenied
254    verifiedDeniedServiceException(USER_NON_ADMIN, action);
255  }
256
257  @Test
258  public void testExecProcedureWithRet() throws Exception {
259    verifyAdminCheckForAction((admin) -> {
260      // Using existing table instead of creating a new one.
261      admin.execProcedureWithReturn("flush-table-proc", TableName.META_TABLE_NAME.getNameAsString(),
262        new HashMap<>());
263    });
264  }
265
266  @Test
267  public void testNormalize() throws Exception {
268    verifyAdminCheckForAction((admin) -> admin.normalize());
269  }
270
271  @Test
272  public void testSetNormalizerRunning() throws Exception {
273    verifyAdminCheckForAction((admin) -> admin.normalizerSwitch(true));
274  }
275
276  @Test
277  public void testExecRegionServerService() throws Exception {
278    Action action = (admin) -> {
279      ServerName serverName = TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName();
280      TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface service =
281        TestRpcServiceProtos.TestProtobufRpcProto
282          .newBlockingStub(admin.coprocessorService(serverName));
283      service.ping(null, TestProtos.EmptyRequestProto.getDefaultInstance());
284    };
285
286    verifyAllowed(USER_ADMIN, action);
287    verifyAllowed(USER_GROUP_ADMIN, action);
288    verifiedDeniedServiceException(USER_NON_ADMIN, action);
289  }
290
291  @Test
292  public void testTableFlush() throws Exception {
293    TableName tn = TableName.valueOf(TEST_NAME.getMethodName());
294    TableDescriptor desc = TableDescriptorBuilder.newBuilder(tn)
295      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("f1")).build();
296    Action adminAction = (admin) -> {
297      admin.createTable(desc);
298      // Avoid giving a global permission which may screw up other tests
299      SecureTestUtil.grantOnTable(TEST_UTIL, USER_NON_ADMIN.getShortName(), tn, null, null,
300        Permission.Action.READ, Permission.Action.WRITE, Permission.Action.CREATE);
301    };
302    verifyAllowed(USER_ADMIN, adminAction);
303
304    Action userAction = (admin) -> {
305      Connection conn = admin.getConnection();
306      final byte[] rowKey = Bytes.toBytes("row1");
307      final byte[] col = Bytes.toBytes("q1");
308      final byte[] val = Bytes.toBytes("v1");
309      try (Table table = conn.getTable(tn)) {
310        // Write a value
311        Put p = new Put(rowKey);
312        p.addColumn(Bytes.toBytes("f1"), col, val);
313        table.put(p);
314        // Flush should not require ADMIN permission
315        admin.flush(tn);
316        // Nb: ideally, we would verify snapshot permission too (as that was fixed in the
317        // regression HBASE-20185) but taking a snapshot requires ADMIN permission which
318        // masks the root issue.
319        // Make sure we read the value
320        Result result = table.get(new Get(rowKey));
321        assertFalse(result.isEmpty());
322        Cell c = result.getColumnLatestCell(Bytes.toBytes("f1"), col);
323        assertArrayEquals(val, CellUtil.cloneValue(c));
324      }
325    };
326    verifyAllowed(USER_NON_ADMIN, userAction);
327  }
328
329  @Test
330  public void testTableFlushAndSnapshot() throws Exception {
331    TableName tn = TableName.valueOf(TEST_NAME.getMethodName());
332    TableDescriptor desc = TableDescriptorBuilder.newBuilder(tn)
333      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("f1")).build();
334    Action adminAction = (admin) -> {
335      admin.createTable(desc);
336      // Giving ADMIN here, but only on this table, *not* globally
337      SecureTestUtil.grantOnTable(TEST_UTIL, USER_NON_ADMIN.getShortName(), tn, null, null,
338        Permission.Action.READ, Permission.Action.WRITE, Permission.Action.CREATE,
339        Permission.Action.ADMIN);
340    };
341    verifyAllowed(USER_ADMIN, adminAction);
342
343    Action userAction = (admin) -> {
344      Connection conn = admin.getConnection();
345      final byte[] rowKey = Bytes.toBytes("row1");
346      final byte[] col = Bytes.toBytes("q1");
347      final byte[] val = Bytes.toBytes("v1");
348      try (Table table = conn.getTable(tn)) {
349        // Write a value
350        Put p = new Put(rowKey);
351        p.addColumn(Bytes.toBytes("f1"), col, val);
352        table.put(p);
353        // Flush should not require ADMIN permission
354        admin.flush(tn);
355        // Table admin should be sufficient to snapshot this table
356        admin.snapshot(tn.getNameAsString() + "_snapshot1", tn);
357        // Read the value just because
358        Result result = table.get(new Get(rowKey));
359        assertFalse(result.isEmpty());
360        Cell c = result.getColumnLatestCell(Bytes.toBytes("f1"), col);
361        assertArrayEquals(val, CellUtil.cloneValue(c));
362      }
363    };
364    verifyAllowed(USER_NON_ADMIN, userAction);
365  }
366
367  @Test
368  public void testGrantDeniedOnSuperUsersGroups() {
369    /** User */
370    try {
371      // Global
372      SecureTestUtil.grantGlobal(USER_ADMIN_NOT_SUPER, TEST_UTIL, USER_ADMIN.getShortName(),
373        Permission.Action.ADMIN, Permission.Action.CREATE);
374      fail("Granting superuser's global permissions is not allowed.");
375    } catch (Exception e) {
376    }
377    try {
378      // Namespace
379      SecureTestUtil.grantOnNamespace(USER_ADMIN_NOT_SUPER, TEST_UTIL, USER_ADMIN.getShortName(),
380        TEST_NAME.getMethodName(), Permission.Action.ADMIN, Permission.Action.CREATE);
381      fail("Granting superuser's namespace permissions is not allowed.");
382    } catch (Exception e) {
383    }
384    try {
385      // Table
386      SecureTestUtil.grantOnTable(USER_ADMIN_NOT_SUPER, TEST_UTIL, USER_ADMIN.getName(),
387        TableName.valueOf(TEST_NAME.getMethodName()), null, null, Permission.Action.ADMIN,
388        Permission.Action.CREATE);
389      fail("Granting superuser's table permissions is not allowed.");
390    } catch (Exception e) {
391    }
392
393    /** Group */
394    try {
395      SecureTestUtil.grantGlobal(USER_ADMIN_NOT_SUPER, TEST_UTIL,
396        USER_IN_SUPERGROUPS.getShortName(), Permission.Action.ADMIN, Permission.Action.CREATE);
397      fail("Granting superuser's global permissions is not allowed.");
398    } catch (Exception e) {
399    }
400  }
401
402  @Test
403  public void testRevokeDeniedOnSuperUsersGroups() {
404    /** User */
405    try {
406      // Global
407      SecureTestUtil.revokeGlobal(USER_ADMIN_NOT_SUPER, TEST_UTIL, USER_ADMIN.getShortName(),
408        Permission.Action.ADMIN);
409      fail("Revoking superuser's global permissions is not allowed.");
410    } catch (Exception e) {
411    }
412    try {
413      // Namespace
414      SecureTestUtil.revokeFromNamespace(USER_ADMIN_NOT_SUPER, TEST_UTIL, USER_ADMIN.getShortName(),
415        TEST_NAME.getMethodName(), Permission.Action.ADMIN);
416      fail("Revoking superuser's namespace permissions is not allowed.");
417    } catch (Exception e) {
418    }
419    try {
420      // Table
421      SecureTestUtil.revokeFromTable(USER_ADMIN_NOT_SUPER, TEST_UTIL, USER_ADMIN.getName(),
422        TableName.valueOf(TEST_NAME.getMethodName()), null, null, Permission.Action.ADMIN);
423      fail("Revoking superuser's table permissions is not allowed.");
424    } catch (Exception e) {
425    }
426
427    /** Group */
428    try {
429      // Global revoke
430      SecureTestUtil.revokeGlobal(USER_ADMIN_NOT_SUPER, TEST_UTIL,
431        AuthUtil.toGroupEntry("supergroup"), Permission.Action.ADMIN, Permission.Action.CREATE);
432      fail("Revoking supergroup's permissions is not allowed.");
433    } catch (Exception e) {
434    }
435  }
436}