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;
025
026import java.io.IOException;
027import java.security.PrivilegedExceptionAction;
028import java.util.Collections;
029import java.util.HashMap;
030import org.apache.hadoop.conf.Configuration;
031import org.apache.hadoop.hbase.AuthUtil;
032import org.apache.hadoop.hbase.Cell;
033import org.apache.hadoop.hbase.CellUtil;
034import org.apache.hadoop.hbase.HBaseClassTestRule;
035import org.apache.hadoop.hbase.HBaseTestingUtil;
036import org.apache.hadoop.hbase.ServerName;
037import org.apache.hadoop.hbase.TableName;
038import org.apache.hadoop.hbase.client.Admin;
039import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
040import org.apache.hadoop.hbase.client.Connection;
041import org.apache.hadoop.hbase.client.ConnectionFactory;
042import org.apache.hadoop.hbase.client.Get;
043import org.apache.hadoop.hbase.client.Put;
044import org.apache.hadoop.hbase.client.Result;
045import org.apache.hadoop.hbase.client.Table;
046import org.apache.hadoop.hbase.client.TableDescriptor;
047import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
048import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
049import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
050import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessor;
051import org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl;
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(TestRpcServiceProtos.TestProtobufRpcProto
117        .newReflectiveService(new TestProtobufRpcServiceImpl()));
118    }
119  }
120
121  private static void enableSecurity(Configuration conf) throws IOException {
122    conf.set("hadoop.security.authorization", "false");
123    conf.set("hadoop.security.authentication", "simple");
124    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
125      AccessController.class.getName() + "," + DummyCpService.class.getName());
126    conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName());
127    conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
128      AccessController.class.getName() + "," + DummyCpService.class.getName());
129    conf.set(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, "true");
130    SecureTestUtil.configureSuperuser(conf);
131  }
132
133  @BeforeClass
134  public static void setup() throws Exception {
135    conf = TEST_UTIL.getConfiguration();
136
137    // Enable security
138    enableSecurity(conf);
139
140    // Create users
141    // admin is superuser as well.
142    USER_ADMIN = User.createUserForTesting(conf, "admin", new String[0]);
143    USER_NON_ADMIN = User.createUserForTesting(conf, "non_admin", new String[0]);
144    USER_GROUP_ADMIN =
145      User.createUserForTesting(conf, "user_group_admin", new String[] { GROUP_ADMIN });
146    USER_IN_SUPERGROUPS =
147      User.createUserForTesting(conf, "user_in_supergroup", new String[] { "supergroup" });
148    USER_ADMIN_NOT_SUPER = User.createUserForTesting(conf, "normal_admin", new String[0]);
149
150    TEST_UTIL.startMiniCluster();
151    // Wait for the ACL table to become available
152    TEST_UTIL.waitUntilAllRegionsAssigned(PermissionStorage.ACL_TABLE_NAME);
153
154    // Assign permissions to groups
155    SecureTestUtil.grantGlobal(TEST_UTIL, toGroupEntry(GROUP_ADMIN), Permission.Action.ADMIN,
156      Permission.Action.CREATE);
157    SecureTestUtil.grantGlobal(TEST_UTIL, USER_ADMIN_NOT_SUPER.getShortName(),
158      Permission.Action.ADMIN);
159  }
160
161  interface Action {
162    void run(Admin admin) throws Exception;
163  }
164
165  private void verifyAllowed(User user, Action action) throws Exception {
166    user.runAs((PrivilegedExceptionAction<?>) () -> {
167      try (Connection conn = ConnectionFactory.createConnection(conf);
168        Admin admin = conn.getAdmin()) {
169        action.run(admin);
170      } catch (IOException e) {
171        fail(e.toString());
172      }
173      return null;
174    });
175  }
176
177  private void verifyDenied(User user, Action action) throws Exception {
178    user.runAs((PrivilegedExceptionAction<?>) () -> {
179      boolean accessDenied = false;
180      try (Connection conn = ConnectionFactory.createConnection(conf);
181        Admin admin = conn.getAdmin()) {
182        action.run(admin);
183      } catch (AccessDeniedException e) {
184        accessDenied = true;
185      }
186      assertTrue("Expected access to be denied", accessDenied);
187      return null;
188    });
189  }
190
191  private void verifiedDeniedServiceException(User user, Action action) throws Exception {
192    user.runAs((PrivilegedExceptionAction<?>) () -> {
193      boolean accessDenied = false;
194      try (Connection conn = ConnectionFactory.createConnection(conf);
195        Admin admin = conn.getAdmin()) {
196        action.run(admin);
197      } catch (ServiceException e) {
198        // For MasterRpcServices.execService.
199        if (e.getCause() instanceof AccessDeniedException) {
200          accessDenied = true;
201        }
202      }
203      assertTrue("Expected access to be denied", accessDenied);
204      return null;
205    });
206
207  }
208
209  private void verifyAdminCheckForAction(Action action) throws Exception {
210    verifyAllowed(USER_ADMIN, action);
211    verifyAllowed(USER_GROUP_ADMIN, action);
212    verifyDenied(USER_NON_ADMIN, action);
213  }
214
215  @Test
216  public void testEnableCatalogJanitor() throws Exception {
217    verifyAdminCheckForAction((admin) -> admin.catalogJanitorSwitch(true));
218  }
219
220  @Test
221  public void testRunCatalogJanitor() throws Exception {
222    verifyAdminCheckForAction((admin) -> admin.runCatalogJanitor());
223  }
224
225  @Test
226  public void testCleanerChoreRunning() throws Exception {
227    verifyAdminCheckForAction((admin) -> admin.cleanerChoreSwitch(true));
228  }
229
230  @Test
231  public void testRunCleanerChore() throws Exception {
232    verifyAdminCheckForAction((admin) -> admin.runCleanerChore());
233  }
234
235  @Test
236  public void testExecProcedure() throws Exception {
237    verifyAdminCheckForAction((admin) -> {
238      // Using existing table instead of creating a new one.
239      admin.execProcedure("flush-table-proc", TableName.META_TABLE_NAME.getNameAsString(),
240        new HashMap<>());
241    });
242  }
243
244  @Test
245  public void testExecService() throws Exception {
246    Action action = (admin) -> {
247      TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface service =
248        TestRpcServiceProtos.TestProtobufRpcProto.newBlockingStub(admin.coprocessorService());
249      service.ping(null, TestProtos.EmptyRequestProto.getDefaultInstance());
250    };
251
252    verifyAllowed(USER_ADMIN, action);
253    verifyAllowed(USER_GROUP_ADMIN, action);
254    // This is same as above verifyAccessDenied
255    verifiedDeniedServiceException(USER_NON_ADMIN, action);
256  }
257
258  @Test
259  public void testExecProcedureWithRet() throws Exception {
260    verifyAdminCheckForAction((admin) -> {
261      // Using existing table instead of creating a new one.
262      admin.execProcedureWithReturn("flush-table-proc", TableName.META_TABLE_NAME.getNameAsString(),
263        new HashMap<>());
264    });
265  }
266
267  @Test
268  public void testNormalize() throws Exception {
269    verifyAdminCheckForAction((admin) -> admin.normalize());
270  }
271
272  @Test
273  public void testSetNormalizerRunning() throws Exception {
274    verifyAdminCheckForAction((admin) -> admin.normalizerSwitch(true));
275  }
276
277  @Test
278  public void testExecRegionServerService() throws Exception {
279    Action action = (admin) -> {
280      ServerName serverName = TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName();
281      TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface service =
282        TestRpcServiceProtos.TestProtobufRpcProto
283          .newBlockingStub(admin.coprocessorService(serverName));
284      service.ping(null, TestProtos.EmptyRequestProto.getDefaultInstance());
285    };
286
287    verifyAllowed(USER_ADMIN, action);
288    verifyAllowed(USER_GROUP_ADMIN, action);
289    verifiedDeniedServiceException(USER_NON_ADMIN, action);
290  }
291
292  @Test
293  public void testTableFlush() throws Exception {
294    TableName tn = TableName.valueOf(TEST_NAME.getMethodName());
295    TableDescriptor desc = TableDescriptorBuilder.newBuilder(tn)
296      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("f1")).build();
297    Action adminAction = (admin) -> {
298      admin.createTable(desc);
299      // Avoid giving a global permission which may screw up other tests
300      SecureTestUtil.grantOnTable(TEST_UTIL, USER_NON_ADMIN.getShortName(), tn, null, null,
301        Permission.Action.READ, Permission.Action.WRITE, Permission.Action.CREATE);
302    };
303    verifyAllowed(USER_ADMIN, adminAction);
304
305    Action userAction = (admin) -> {
306      Connection conn = admin.getConnection();
307      final byte[] rowKey = Bytes.toBytes("row1");
308      final byte[] col = Bytes.toBytes("q1");
309      final byte[] val = Bytes.toBytes("v1");
310      try (Table table = conn.getTable(tn)) {
311        // Write a value
312        Put p = new Put(rowKey);
313        p.addColumn(Bytes.toBytes("f1"), col, val);
314        table.put(p);
315        // Flush should not require ADMIN permission
316        admin.flush(tn);
317        // Nb: ideally, we would verify snapshot permission too (as that was fixed in the
318        // regression HBASE-20185) but taking a snapshot requires ADMIN permission which
319        // masks the root issue.
320        // Make sure we read the value
321        Result result = table.get(new Get(rowKey));
322        assertFalse(result.isEmpty());
323        Cell c = result.getColumnLatestCell(Bytes.toBytes("f1"), col);
324        assertArrayEquals(val, CellUtil.cloneValue(c));
325      }
326    };
327    verifyAllowed(USER_NON_ADMIN, userAction);
328  }
329
330  @Test
331  public void testTableFlushAndSnapshot() throws Exception {
332    TableName tn = TableName.valueOf(TEST_NAME.getMethodName());
333    TableDescriptor desc = TableDescriptorBuilder.newBuilder(tn)
334      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("f1")).build();
335    Action adminAction = (admin) -> {
336      admin.createTable(desc);
337      // Giving ADMIN here, but only on this table, *not* globally
338      SecureTestUtil.grantOnTable(TEST_UTIL, USER_NON_ADMIN.getShortName(), tn, null, null,
339        Permission.Action.READ, Permission.Action.WRITE, Permission.Action.CREATE,
340        Permission.Action.ADMIN);
341    };
342    verifyAllowed(USER_ADMIN, adminAction);
343
344    Action userAction = (admin) -> {
345      Connection conn = admin.getConnection();
346      final byte[] rowKey = Bytes.toBytes("row1");
347      final byte[] col = Bytes.toBytes("q1");
348      final byte[] val = Bytes.toBytes("v1");
349      try (Table table = conn.getTable(tn)) {
350        // Write a value
351        Put p = new Put(rowKey);
352        p.addColumn(Bytes.toBytes("f1"), col, val);
353        table.put(p);
354        // Flush should not require ADMIN permission
355        admin.flush(tn);
356        // Table admin should be sufficient to snapshot this table
357        admin.snapshot(tn.getNameAsString() + "_snapshot1", tn);
358        // Read the value just because
359        Result result = table.get(new Get(rowKey));
360        assertFalse(result.isEmpty());
361        Cell c = result.getColumnLatestCell(Bytes.toBytes("f1"), col);
362        assertArrayEquals(val, CellUtil.cloneValue(c));
363      }
364    };
365    verifyAllowed(USER_NON_ADMIN, userAction);
366  }
367
368  @Test
369  public void testGrantDeniedOnSuperUsersGroups() {
370    /** User */
371    try {
372      // Global
373      SecureTestUtil.grantGlobal(USER_ADMIN_NOT_SUPER, TEST_UTIL, USER_ADMIN.getShortName(),
374        Permission.Action.ADMIN, Permission.Action.CREATE);
375      fail("Granting superuser's global permissions is not allowed.");
376    } catch (Exception e) {
377    }
378    try {
379      // Namespace
380      SecureTestUtil.grantOnNamespace(USER_ADMIN_NOT_SUPER, TEST_UTIL, USER_ADMIN.getShortName(),
381        TEST_NAME.getMethodName(), Permission.Action.ADMIN, Permission.Action.CREATE);
382      fail("Granting superuser's namespace permissions is not allowed.");
383    } catch (Exception e) {
384    }
385    try {
386      // Table
387      SecureTestUtil.grantOnTable(USER_ADMIN_NOT_SUPER, TEST_UTIL, USER_ADMIN.getName(),
388        TableName.valueOf(TEST_NAME.getMethodName()), null, null, Permission.Action.ADMIN,
389        Permission.Action.CREATE);
390      fail("Granting superuser's table permissions is not allowed.");
391    } catch (Exception e) {
392    }
393
394    /** Group */
395    try {
396      SecureTestUtil.grantGlobal(USER_ADMIN_NOT_SUPER, TEST_UTIL,
397        USER_IN_SUPERGROUPS.getShortName(), Permission.Action.ADMIN, Permission.Action.CREATE);
398      fail("Granting superuser's global permissions is not allowed.");
399    } catch (Exception e) {
400    }
401  }
402
403  @Test
404  public void testRevokeDeniedOnSuperUsersGroups() {
405    /** User */
406    try {
407      // Global
408      SecureTestUtil.revokeGlobal(USER_ADMIN_NOT_SUPER, TEST_UTIL, USER_ADMIN.getShortName(),
409        Permission.Action.ADMIN);
410      fail("Revoking superuser's global permissions is not allowed.");
411    } catch (Exception e) {
412    }
413    try {
414      // Namespace
415      SecureTestUtil.revokeFromNamespace(USER_ADMIN_NOT_SUPER, TEST_UTIL, USER_ADMIN.getShortName(),
416        TEST_NAME.getMethodName(), Permission.Action.ADMIN);
417      fail("Revoking superuser's namespace permissions is not allowed.");
418    } catch (Exception e) {
419    }
420    try {
421      // Table
422      SecureTestUtil.revokeFromTable(USER_ADMIN_NOT_SUPER, TEST_UTIL, USER_ADMIN.getName(),
423        TableName.valueOf(TEST_NAME.getMethodName()), null, null, Permission.Action.ADMIN);
424      fail("Revoking superuser's table permissions is not allowed.");
425    } catch (Exception e) {
426    }
427
428    /** Group */
429    try {
430      // Global revoke
431      SecureTestUtil.revokeGlobal(USER_ADMIN_NOT_SUPER, TEST_UTIL,
432        AuthUtil.toGroupEntry("supergroup"), Permission.Action.ADMIN, Permission.Action.CREATE);
433      fail("Revoking supergroup's permissions is not allowed.");
434    } catch (Exception e) {
435    }
436  }
437}